From 3fc45e20cc1b08f1446e27e07da32fb83cd57836 Mon Sep 17 00:00:00 2001 From: Peng Junzhi <201250214@smail.nju.edu.cn> Date: Tue, 17 Mar 2026 14:57:32 +0800 Subject: [PATCH 1/4] introduce consistency check and repair --- .../IoTDBIoTConsensusV23C3DBasicITBase.java | 421 ++++++-- .../IoTDBIoTConsensusV2Batch3C3DBasicIT.java | 18 + .../IoTDBIoTConsensusV2Stream3C3DBasicIT.java | 18 + .../client/sync/CnToDnSyncRequestType.java | 2 + .../client/sync/SyncDataNodeClientPool.java | 9 + .../confignode/manager/ConfigManager.java | 8 + .../confignode/manager/ProcedureManager.java | 37 + .../LiveDataRegionRepairExecutionContext.java | 549 ++++++++++ .../consistency/RepairRegionProcedure.java | 982 ++++++++++++++++++ .../state/consistency/RepairState.java | 44 + .../procedure/store/ProcedureFactory.java | 6 + .../procedure/store/ProcedureType.java | 3 + .../thrift/ConfigNodeRPCServiceProcessor.java | 7 + .../RepairRegionProcedureTest.java | 828 +++++++++++++++ .../db/protocol/client/ConfigNodeClient.java | 7 + .../impl/DataNodeInternalRPCServiceImpl.java | 17 + .../DataRegionConsistencyRepairService.java | 352 +++++++ .../consistency/MerkleCompactionHook.java | 162 +++ .../consistency/MerkleDeletionHook.java | 97 ++ .../consistency/MerkleFlushListener.java | 95 ++ .../consistency/MerkleHashComputer.java | 232 +++++ .../ConsistencyCheckScheduler.java | 217 ++++ .../consistency/ConsistencyMerkleTree.java | 238 +++++ .../consistency/ConsistencyViewBuilder.java | 133 +++ .../iotv2/consistency/DualDigest.java | 105 ++ .../consistency/RepairProgressTable.java | 229 ++++ .../consistency/StaleSnapshotException.java | 31 + .../consistency/TimePartitionMerkleNode.java | 129 +++ .../consistency/ibf/CompositeKeyCodec.java | 88 ++ .../consistency/ibf/DataPointLocator.java | 71 ++ .../iotv2/consistency/ibf/DiffEntry.java | 59 ++ .../iotv2/consistency/ibf/IBFCell.java | 98 ++ .../consistency/ibf/IBFDecodeResult.java | 58 ++ .../ibf/InvertibleBloomFilter.java | 210 ++++ .../iotv2/consistency/ibf/RowRefIndex.java | 160 +++ .../consistency/ibf/StrataEstimator.java | 108 ++ .../iotv2/consistency/merkle/MerkleEntry.java | 73 ++ .../consistency/merkle/MerkleFileCache.java | 124 +++ .../consistency/merkle/MerkleFileContent.java | 82 ++ .../consistency/merkle/MerkleFileReader.java | 148 +++ .../consistency/merkle/MerkleFileWriter.java | 136 +++ .../repair/AdaptiveRepairExecutor.java | 204 ++++ .../consistency/repair/DiffAttribution.java | 111 ++ .../consistency/repair/ModEntrySummary.java | 97 ++ .../consistency/repair/RepairAction.java | 38 + .../repair/RepairConflictResolver.java | 145 +++ .../consistency/repair/RepairCostModel.java | 119 +++ .../iotv2/consistency/repair/RepairPlan.java | 80 ++ .../consistency/repair/RepairRecord.java | 132 +++ .../consistency/repair/RepairSession.java | 195 ++++ .../consistency/repair/RepairStrategy.java | 28 + .../ConsistencyMerkleTreeTest.java | 59 ++ .../repair/DiffAttributionTest.java | 70 ++ .../repair/RepairConflictResolverTest.java | 76 ++ .../consistency/repair/RepairSessionTest.java | 80 ++ .../src/main/thrift/confignode.thrift | 8 +- .../src/main/thrift/datanode.thrift | 34 +- replica_consistency_check_repair_review.md | 43 + 58 files changed, 7839 insertions(+), 71 deletions(-) create mode 100644 iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/procedure/impl/consistency/LiveDataRegionRepairExecutionContext.java create mode 100644 iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/procedure/impl/consistency/RepairRegionProcedure.java create mode 100644 iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/procedure/state/consistency/RepairState.java create mode 100644 iotdb-core/confignode/src/test/java/org/apache/iotdb/confignode/procedure/impl/consistency/RepairRegionProcedureTest.java create mode 100644 iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/consistency/DataRegionConsistencyRepairService.java create mode 100644 iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/consistency/MerkleCompactionHook.java create mode 100644 iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/consistency/MerkleDeletionHook.java create mode 100644 iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/consistency/MerkleFlushListener.java create mode 100644 iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/consistency/MerkleHashComputer.java create mode 100644 iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/consensus/iotv2/consistency/ConsistencyCheckScheduler.java create mode 100644 iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/consensus/iotv2/consistency/ConsistencyMerkleTree.java create mode 100644 iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/consensus/iotv2/consistency/ConsistencyViewBuilder.java create mode 100644 iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/consensus/iotv2/consistency/DualDigest.java create mode 100644 iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/consensus/iotv2/consistency/RepairProgressTable.java create mode 100644 iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/consensus/iotv2/consistency/StaleSnapshotException.java create mode 100644 iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/consensus/iotv2/consistency/TimePartitionMerkleNode.java create mode 100644 iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/consensus/iotv2/consistency/ibf/CompositeKeyCodec.java create mode 100644 iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/consensus/iotv2/consistency/ibf/DataPointLocator.java create mode 100644 iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/consensus/iotv2/consistency/ibf/DiffEntry.java create mode 100644 iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/consensus/iotv2/consistency/ibf/IBFCell.java create mode 100644 iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/consensus/iotv2/consistency/ibf/IBFDecodeResult.java create mode 100644 iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/consensus/iotv2/consistency/ibf/InvertibleBloomFilter.java create mode 100644 iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/consensus/iotv2/consistency/ibf/RowRefIndex.java create mode 100644 iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/consensus/iotv2/consistency/ibf/StrataEstimator.java create mode 100644 iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/consensus/iotv2/consistency/merkle/MerkleEntry.java create mode 100644 iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/consensus/iotv2/consistency/merkle/MerkleFileCache.java create mode 100644 iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/consensus/iotv2/consistency/merkle/MerkleFileContent.java create mode 100644 iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/consensus/iotv2/consistency/merkle/MerkleFileReader.java create mode 100644 iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/consensus/iotv2/consistency/merkle/MerkleFileWriter.java create mode 100644 iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/consensus/iotv2/consistency/repair/AdaptiveRepairExecutor.java create mode 100644 iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/consensus/iotv2/consistency/repair/DiffAttribution.java create mode 100644 iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/consensus/iotv2/consistency/repair/ModEntrySummary.java create mode 100644 iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/consensus/iotv2/consistency/repair/RepairAction.java create mode 100644 iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/consensus/iotv2/consistency/repair/RepairConflictResolver.java create mode 100644 iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/consensus/iotv2/consistency/repair/RepairCostModel.java create mode 100644 iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/consensus/iotv2/consistency/repair/RepairPlan.java create mode 100644 iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/consensus/iotv2/consistency/repair/RepairRecord.java create mode 100644 iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/consensus/iotv2/consistency/repair/RepairSession.java create mode 100644 iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/consensus/iotv2/consistency/repair/RepairStrategy.java create mode 100644 iotdb-core/node-commons/src/test/java/org/apache/iotdb/commons/consensus/iotv2/consistency/ConsistencyMerkleTreeTest.java create mode 100644 iotdb-core/node-commons/src/test/java/org/apache/iotdb/commons/consensus/iotv2/consistency/repair/DiffAttributionTest.java create mode 100644 iotdb-core/node-commons/src/test/java/org/apache/iotdb/commons/consensus/iotv2/consistency/repair/RepairConflictResolverTest.java create mode 100644 iotdb-core/node-commons/src/test/java/org/apache/iotdb/commons/consensus/iotv2/consistency/repair/RepairSessionTest.java create mode 100644 replica_consistency_check_repair_review.md diff --git a/integration-test/src/test/java/org/apache/iotdb/db/it/iotconsensusv2/IoTDBIoTConsensusV23C3DBasicITBase.java b/integration-test/src/test/java/org/apache/iotdb/db/it/iotconsensusv2/IoTDBIoTConsensusV23C3DBasicITBase.java index 9544ac5cf2b31..d2394eac57249 100644 --- a/integration-test/src/test/java/org/apache/iotdb/db/it/iotconsensusv2/IoTDBIoTConsensusV23C3DBasicITBase.java +++ b/integration-test/src/test/java/org/apache/iotdb/db/it/iotconsensusv2/IoTDBIoTConsensusV23C3DBasicITBase.java @@ -19,12 +19,20 @@ package org.apache.iotdb.db.it.iotconsensusv2; +import org.apache.iotdb.common.rpc.thrift.TConsensusGroupId; +import org.apache.iotdb.common.rpc.thrift.TConsensusGroupType; +import org.apache.iotdb.common.rpc.thrift.TSStatus; +import org.apache.iotdb.commons.client.sync.SyncConfigNodeIServiceClient; import org.apache.iotdb.confignode.it.regionmigration.IoTDBRegionOperationReliabilityITFramework; +import org.apache.iotdb.confignode.rpc.thrift.TTriggerRegionConsistencyRepairReq; import org.apache.iotdb.consensus.ConsensusFactory; import org.apache.iotdb.isession.SessionConfig; import org.apache.iotdb.it.env.EnvFactory; import org.apache.iotdb.it.env.cluster.node.DataNodeWrapper; import org.apache.iotdb.itbase.env.BaseEnv; +import org.apache.iotdb.db.storageengine.dataregion.modification.ModificationFile; +import org.apache.iotdb.db.storageengine.dataregion.modification.v1.ModificationFileV1; +import org.apache.iotdb.db.storageengine.dataregion.tsfile.TsFileResource; import org.apache.tsfile.utils.Pair; import org.awaitility.Awaitility; @@ -33,15 +41,21 @@ import org.slf4j.Logger; import org.slf4j.LoggerFactory; +import java.io.File; import java.sql.Connection; import java.sql.ResultSet; import java.sql.Statement; +import java.nio.file.Files; +import java.nio.file.Path; +import java.nio.file.Paths; import java.util.Collections; import java.util.Map; +import java.util.Optional; import java.util.Set; import java.util.concurrent.TimeUnit; import java.util.regex.Matcher; import java.util.regex.Pattern; +import java.util.stream.Stream; import static org.apache.iotdb.util.MagicUtils.makeItCloseQuietly; @@ -76,6 +90,10 @@ public abstract class IoTDBIoTConsensusV23C3DBasicITBase "INSERT INTO root.sg.d1(timestamp,speed,temperature) values(102, 5, 6)"; protected static final String FLUSH_COMMAND = "flush on cluster"; protected static final String COUNT_QUERY = "select count(*) from root.sg.**"; + protected static final String DELETE_SPEED_UP_TO_101 = + "DELETE FROM root.sg.d1.speed WHERE time <= 101"; + protected static final String COUNT_AFTER_DELETE_QUERY = + "select count(speed), count(temperature) from root.sg.d1"; protected static final String SELECT_ALL_QUERY = "select speed, temperature from root.sg.d1"; /** @@ -108,17 +126,32 @@ public void test3C3DWriteFlushAndQuery() throws Exception { try (Connection connection = makeItCloseQuietly(EnvFactory.getEnv().getConnection()); Statement statement = makeItCloseQuietly(connection.createStatement())) { - LOGGER.info("Writing data to 3C3D cluster (mode: {})...", getIoTConsensusV2Mode()); - statement.execute(INSERTION1); - statement.execute(INSERTION2); - statement.execute(INSERTION3); + insertAndFlushTestData(statement); + verifyDataConsistency(statement); - LOGGER.info("Executing flush on cluster..."); - statement.execute(FLUSH_COMMAND); + LOGGER.info("3C3D IoTConsensusV2 {} basic test passed", getIoTConsensusV2Mode()); + } + } + + /** + * Test that a follower can observe the same logical view after the leader reports replication + * catch-up. + */ + public void testFollowerCanReadConsistentDataAfterCatchUp() throws Exception { + try (Connection connection = makeItCloseQuietly(EnvFactory.getEnv().getConnection()); + Statement statement = makeItCloseQuietly(connection.createStatement())) { + insertAndFlushTestData(statement); verifyDataConsistency(statement); - LOGGER.info("3C3D IoTConsensusV2 {} basic test passed", getIoTConsensusV2Mode()); + RegionReplicaSelection regionReplicaSelection = selectReplicatedDataRegion(statement); + waitForReplicationComplete(regionReplicaSelection.leaderNode); + + LOGGER.info( + "Verifying logical view from follower DataNode {} for region {} after catch-up", + regionReplicaSelection.followerDataNodeId, + regionReplicaSelection.regionId); + verifyDataConsistencyOnNode(regionReplicaSelection.followerNode); } } @@ -127,86 +160,205 @@ public void test3C3DWriteFlushAndQuery() throws Exception { * follower serves the same data. */ public void testReplicaConsistencyAfterLeaderStop() throws Exception { + RegionReplicaSelection regionReplicaSelection; + try (Connection connection = makeItCloseQuietly(EnvFactory.getEnv().getConnection()); + Statement statement = makeItCloseQuietly(connection.createStatement())) { + + insertAndFlushTestData(statement); + verifyDataConsistency(statement); + + regionReplicaSelection = selectReplicatedDataRegion(statement); + waitForReplicationComplete(regionReplicaSelection.leaderNode); + } + + LOGGER.info( + "Stopping leader DataNode {} (region {}) for replica consistency test", + regionReplicaSelection.leaderDataNodeId, + regionReplicaSelection.regionId); + + regionReplicaSelection.leaderNode.stopForcibly(); + Assert.assertFalse("Leader should be stopped", regionReplicaSelection.leaderNode.isAlive()); + + LOGGER.info( + "Waiting for follower DataNode {} to be elected as new leader and verifying replica consistency...", + regionReplicaSelection.followerDataNodeId); + Awaitility.await() + .pollDelay(2, TimeUnit.SECONDS) + .atMost(2, TimeUnit.MINUTES) + .untilAsserted(() -> verifyDataConsistencyOnNode(regionReplicaSelection.followerNode)); + + LOGGER.info( + "Replica consistency verified: follower has same data as former leader after failover"); + } + + /** + * Test replica consistency for a delete path: after deletion is replicated, stopping the leader + * must not change the surviving logical view. + */ + public void testReplicaConsistencyAfterDeleteAndLeaderStop() throws Exception { + RegionReplicaSelection regionReplicaSelection; try (Connection connection = makeItCloseQuietly(EnvFactory.getEnv().getConnection()); Statement statement = makeItCloseQuietly(connection.createStatement())) { - LOGGER.info("Writing data to 3C3D cluster (mode: {})...", getIoTConsensusV2Mode()); - statement.execute(INSERTION1); - statement.execute(INSERTION2); - statement.execute(INSERTION3); + insertAndFlushTestData(statement); + regionReplicaSelection = selectReplicatedDataRegion(statement); + + LOGGER.info( + "Deleting replicated data on leader DataNode {} for region {}", + regionReplicaSelection.leaderDataNodeId, + regionReplicaSelection.regionId); + statement.execute(DELETE_SPEED_UP_TO_101); statement.execute(FLUSH_COMMAND); + verifyPostDeleteConsistency(statement); + waitForReplicationComplete(regionReplicaSelection.leaderNode); + verifyPostDeleteConsistencyOnNode(regionReplicaSelection.followerNode); + } + + LOGGER.info( + "Stopping leader DataNode {} after replicated delete", + regionReplicaSelection.leaderDataNodeId); + regionReplicaSelection.leaderNode.stopForcibly(); + Assert.assertFalse("Leader should be stopped", regionReplicaSelection.leaderNode.isAlive()); + + Awaitility.await() + .pollDelay(2, TimeUnit.SECONDS) + .atMost(2, TimeUnit.MINUTES) + .untilAsserted(() -> verifyPostDeleteConsistencyOnNode(regionReplicaSelection.followerNode)); + + LOGGER.info( + "Replica consistency verified after delete and failover on follower DataNode {}", + regionReplicaSelection.followerDataNodeId); + } + + /** + * Simulate a follower missing a sealed TsFile, trigger replica consistency repair through + * ConfigNode, and verify the repaired follower still serves the correct data after the leader is + * stopped. + */ + public void testReplicaConsistencyRepairAfterFollowerLosesSealedTsFile() throws Exception { + RegionReplicaSelection regionReplicaSelection; + Path deletedTsFile; + + try (Connection connection = makeItCloseQuietly(EnvFactory.getEnv().getConnection()); + Statement statement = makeItCloseQuietly(connection.createStatement())) { + insertAndFlushTestData(statement); verifyDataConsistency(statement); - Map>> dataRegionMap = - getDataRegionMapWithLeader(statement); - - int targetRegionId = -1; - int leaderDataNodeId = -1; - int followerDataNodeId = -1; - for (Map.Entry>> entry : dataRegionMap.entrySet()) { - Pair> leaderAndReplicas = entry.getValue(); - if (leaderAndReplicas.getRight().size() > 1 - && leaderAndReplicas.getRight().size() <= DATA_REPLICATION_FACTOR - && leaderAndReplicas.getLeft() > 0) { - targetRegionId = entry.getKey(); - leaderDataNodeId = leaderAndReplicas.getLeft(); - final int lambdaLeaderDataNodeId = leaderDataNodeId; - followerDataNodeId = - leaderAndReplicas.getRight().stream() - .filter(i -> i != lambdaLeaderDataNodeId) - .findAny() - .orElse(-1); - break; - } - } + regionReplicaSelection = selectReplicatedDataRegion(statement); + waitForReplicationComplete(regionReplicaSelection.leaderNode); + deletedTsFile = + findLatestSealedTsFile(regionReplicaSelection.followerNode, regionReplicaSelection.regionId); + } - Assert.assertTrue( - "Should find a data region with leader for root.sg", - targetRegionId > 0 && leaderDataNodeId > 0 && followerDataNodeId > 0); + LOGGER.info( + "Stopping follower DataNode {} and deleting sealed TsFile {} for region {}", + regionReplicaSelection.followerDataNodeId, + deletedTsFile, + regionReplicaSelection.regionId); + regionReplicaSelection.followerNode.stopForcibly(); + Assert.assertFalse("Follower should be stopped", regionReplicaSelection.followerNode.isAlive()); + deleteTsFileArtifacts(deletedTsFile); + + regionReplicaSelection.followerNode.start(); + Awaitility.await() + .pollDelay(2, TimeUnit.SECONDS) + .atMost(2, TimeUnit.MINUTES) + .untilAsserted(() -> assertDataInconsistentOnNode(regionReplicaSelection.followerNode)); - DataNodeWrapper leaderNode = - EnvFactory.getEnv() - .dataNodeIdToWrapper(leaderDataNodeId) - .orElseThrow(() -> new AssertionError("DataNode not found in cluster")); + triggerRegionConsistencyRepair(regionReplicaSelection.regionId); - waitForReplicationComplete(leaderNode); + Awaitility.await() + .pollDelay(2, TimeUnit.SECONDS) + .atMost(2, TimeUnit.MINUTES) + .untilAsserted(() -> verifyDataConsistencyOnNode(regionReplicaSelection.followerNode)); - LOGGER.info( - "Stopping leader DataNode {} (region {}) for replica consistency test", - leaderDataNodeId, - targetRegionId); + LOGGER.info( + "Stopping leader DataNode {} after repair to verify repaired follower serves local data", + regionReplicaSelection.leaderDataNodeId); + regionReplicaSelection.leaderNode.stopForcibly(); + Assert.assertFalse("Leader should be stopped", regionReplicaSelection.leaderNode.isAlive()); + + Awaitility.await() + .pollDelay(2, TimeUnit.SECONDS) + .atMost(2, TimeUnit.MINUTES) + .untilAsserted(() -> verifyDataConsistencyOnNode(regionReplicaSelection.followerNode)); + } - leaderNode.stopForcibly(); - Assert.assertFalse("Leader should be stopped", leaderNode.isAlive()); + protected void insertAndFlushTestData(Statement statement) throws Exception { + LOGGER.info("Writing data to 3C3D cluster (mode: {})...", getIoTConsensusV2Mode()); + statement.execute(INSERTION1); + statement.execute(INSERTION2); + statement.execute(INSERTION3); + LOGGER.info("Executing flush on cluster..."); + statement.execute(FLUSH_COMMAND); + } + + protected RegionReplicaSelection selectReplicatedDataRegion(Statement statement) throws Exception { + Map>> dataRegionMap = getDataRegionMapWithLeader(statement); + + for (Map.Entry>> entry : dataRegionMap.entrySet()) { + Pair> leaderAndReplicas = entry.getValue(); + if (leaderAndReplicas.getLeft() <= 0 || leaderAndReplicas.getRight().size() <= 1) { + continue; + } + + int leaderDataNodeId = leaderAndReplicas.getLeft(); + int followerDataNodeId = + leaderAndReplicas.getRight().stream() + .filter(dataNodeId -> dataNodeId != leaderDataNodeId) + .findFirst() + .orElse(-1); + if (followerDataNodeId <= 0) { + continue; + } + + DataNodeWrapper leaderNode = + EnvFactory.getEnv() + .dataNodeIdToWrapper(leaderDataNodeId) + .orElseThrow(() -> new AssertionError("Leader DataNode not found in cluster")); DataNodeWrapper followerNode = EnvFactory.getEnv() .dataNodeIdToWrapper(followerDataNodeId) .orElseThrow(() -> new AssertionError("Follower DataNode not found in cluster")); - LOGGER.info( - "Waiting for follower DataNode {} to be elected as new leader and verifying replica consistency...", - followerDataNodeId); - Awaitility.await() - .pollDelay(2, TimeUnit.SECONDS) - .atMost(2, TimeUnit.MINUTES) - .untilAsserted( - () -> { - try (Connection followerConn = - makeItCloseQuietly( - EnvFactory.getEnv() - .getConnection( - followerNode, - SessionConfig.DEFAULT_USER, - SessionConfig.DEFAULT_PASSWORD, - BaseEnv.TREE_SQL_DIALECT)); - Statement followerStmt = makeItCloseQuietly(followerConn.createStatement())) { - verifyDataConsistency(followerStmt); - } - }); + return new RegionReplicaSelection( + entry.getKey(), + leaderDataNodeId, + followerDataNodeId, + leaderNode, + followerNode); + } - LOGGER.info( - "Replica consistency verified: follower has same data as former leader after failover"); + Assert.fail("Should find a replicated data region with a leader for root.sg"); + throw new AssertionError("unreachable"); + } + + protected void verifyDataConsistencyOnNode(DataNodeWrapper targetNode) throws Exception { + try (Connection targetConnection = + makeItCloseQuietly( + EnvFactory.getEnv() + .getConnection( + targetNode, + SessionConfig.DEFAULT_USER, + SessionConfig.DEFAULT_PASSWORD, + BaseEnv.TREE_SQL_DIALECT)); + Statement targetStatement = makeItCloseQuietly(targetConnection.createStatement())) { + verifyDataConsistency(targetStatement); + } + } + + protected void verifyPostDeleteConsistencyOnNode(DataNodeWrapper targetNode) throws Exception { + try (Connection targetConnection = + makeItCloseQuietly( + EnvFactory.getEnv() + .getConnection( + targetNode, + SessionConfig.DEFAULT_USER, + SessionConfig.DEFAULT_PASSWORD, + BaseEnv.TREE_SQL_DIALECT)); + Statement targetStatement = makeItCloseQuietly(targetConnection.createStatement())) { + verifyPostDeleteConsistency(targetStatement); } } @@ -284,6 +436,69 @@ protected void verifyDataConsistency(Statement statement) throws Exception { Assert.assertEquals("Expected 3 rows from select *", 3, rowCount); } + protected void verifyPostDeleteConsistency(Statement statement) throws Exception { + LOGGER.info("Querying data to verify replicated delete success..."); + try (ResultSet countResult = statement.executeQuery(COUNT_AFTER_DELETE_QUERY)) { + Assert.assertTrue("Delete count query should return results", countResult.next()); + Assert.assertEquals( + "Expected only one surviving speed value after delete", + 1, + parseLongFromString(countResult.getString(1))); + Assert.assertEquals( + "Expected all temperature values to remain after delete", + 3, + parseLongFromString(countResult.getString(2))); + } + + int rowCount = 0; + try (ResultSet selectResult = statement.executeQuery(SELECT_ALL_QUERY)) { + while (selectResult.next()) { + rowCount++; + long timestamp = parseLongFromString(selectResult.getString(1)); + String speed = selectResult.getString(2); + long temperature = parseLongFromString(selectResult.getString(3)); + if (timestamp == 100) { + assertNullValue(speed); + Assert.assertEquals(2, temperature); + } else if (timestamp == 101) { + assertNullValue(speed); + Assert.assertEquals(4, temperature); + } else if (timestamp == 102) { + Assert.assertEquals(5, parseLongFromString(speed)); + Assert.assertEquals(6, temperature); + } else { + Assert.fail("Unexpected timestamp after delete: " + timestamp); + } + } + } + Assert.assertEquals("Expected 3 logical rows from select after delete", 3, rowCount); + } + + protected void assertDataInconsistentOnNode(DataNodeWrapper targetNode) throws Exception { + try (Connection targetConnection = + makeItCloseQuietly( + EnvFactory.getEnv() + .getConnection( + targetNode, + SessionConfig.DEFAULT_USER, + SessionConfig.DEFAULT_PASSWORD, + BaseEnv.TREE_SQL_DIALECT)); + Statement targetStatement = makeItCloseQuietly(targetConnection.createStatement())) { + try { + verifyDataConsistency(targetStatement); + Assert.fail("Expected inconsistent data on DataNode " + targetNode.getId()); + } catch (AssertionError expected) { + LOGGER.info("Observed expected inconsistency on DataNode {}", targetNode.getId()); + } + } + } + + protected static void assertNullValue(String value) { + Assert.assertTrue( + "Expected deleted value to be null, but was " + value, + value == null || "null".equalsIgnoreCase(value)); + } + /** Parse long from IoTDB result string (handles both "1" and "1.0" formats). */ protected static long parseLongFromString(String s) { if (s == null || s.isEmpty()) { @@ -295,4 +510,72 @@ protected static long parseLongFromString(String s) { return (long) Double.parseDouble(s); } } + + protected static final class RegionReplicaSelection { + private final int regionId; + private final int leaderDataNodeId; + private final int followerDataNodeId; + private final DataNodeWrapper leaderNode; + private final DataNodeWrapper followerNode; + + private RegionReplicaSelection( + int regionId, + int leaderDataNodeId, + int followerDataNodeId, + DataNodeWrapper leaderNode, + DataNodeWrapper followerNode) { + this.regionId = regionId; + this.leaderDataNodeId = leaderDataNodeId; + this.followerDataNodeId = followerDataNodeId; + this.leaderNode = leaderNode; + this.followerNode = followerNode; + } + } + + private Path findLatestSealedTsFile(DataNodeWrapper dataNodeWrapper, int regionId) + throws Exception { + try (Stream tsFiles = Files.walk(Paths.get(dataNodeWrapper.getDataPath()))) { + Optional candidate = + tsFiles + .filter(Files::isRegularFile) + .filter(path -> path.getFileName().toString().endsWith(".tsfile")) + .filter(path -> path.toString().contains(File.separator + "root.sg" + File.separator)) + .filter(path -> belongsToRegion(path, regionId)) + .max((left, right) -> Long.compare(left.toFile().lastModified(), right.toFile().lastModified())); + if (candidate.isPresent()) { + return candidate.get(); + } + } + throw new AssertionError("No sealed TsFile found for region " + regionId); + } + + private boolean belongsToRegion(Path tsFile, int regionId) { + Path timePartitionDir = tsFile.getParent(); + Path regionDir = timePartitionDir == null ? null : timePartitionDir.getParent(); + return regionDir != null && String.valueOf(regionId).equals(regionDir.getFileName().toString()); + } + + private void deleteTsFileArtifacts(Path tsFile) throws Exception { + Files.deleteIfExists(tsFile); + Files.deleteIfExists(Paths.get(tsFile.toString() + TsFileResource.RESOURCE_SUFFIX)); + Files.deleteIfExists(Paths.get(tsFile.toString() + ModificationFile.FILE_SUFFIX)); + Files.deleteIfExists(Paths.get(tsFile.toString() + ModificationFile.COMPACTION_FILE_SUFFIX)); + Files.deleteIfExists(Paths.get(tsFile.toString() + ModificationFileV1.FILE_SUFFIX)); + Files.deleteIfExists( + Paths.get(tsFile.toString() + ModificationFileV1.COMPACTION_FILE_SUFFIX)); + } + + private void triggerRegionConsistencyRepair(int regionId) throws Exception { + TConsensusGroupId consensusGroupId = new TConsensusGroupId(TConsensusGroupType.DataRegion, regionId); + try (SyncConfigNodeIServiceClient client = + (SyncConfigNodeIServiceClient) EnvFactory.getEnv().getLeaderConfigNodeConnection()) { + TSStatus status = + client.triggerRegionConsistencyRepair( + new TTriggerRegionConsistencyRepairReq(consensusGroupId)); + Assert.assertEquals( + "Replica consistency repair should succeed", + 200, + status.getCode()); + } + } } diff --git a/integration-test/src/test/java/org/apache/iotdb/db/it/iotconsensusv2/batch/IoTDBIoTConsensusV2Batch3C3DBasicIT.java b/integration-test/src/test/java/org/apache/iotdb/db/it/iotconsensusv2/batch/IoTDBIoTConsensusV2Batch3C3DBasicIT.java index f71462fa470af..2302378924e0d 100644 --- a/integration-test/src/test/java/org/apache/iotdb/db/it/iotconsensusv2/batch/IoTDBIoTConsensusV2Batch3C3DBasicIT.java +++ b/integration-test/src/test/java/org/apache/iotdb/db/it/iotconsensusv2/batch/IoTDBIoTConsensusV2Batch3C3DBasicIT.java @@ -44,6 +44,24 @@ public void testReplicaConsistencyAfterLeaderStop() throws Exception { super.testReplicaConsistencyAfterLeaderStop(); } + @Override + @Test + public void testFollowerCanReadConsistentDataAfterCatchUp() throws Exception { + super.testFollowerCanReadConsistentDataAfterCatchUp(); + } + + @Override + @Test + public void testReplicaConsistencyAfterDeleteAndLeaderStop() throws Exception { + super.testReplicaConsistencyAfterDeleteAndLeaderStop(); + } + + @Override + @Test + public void testReplicaConsistencyRepairAfterFollowerLosesSealedTsFile() throws Exception { + super.testReplicaConsistencyRepairAfterFollowerLosesSealedTsFile(); + } + @Override @Test public void test3C3DWriteFlushAndQuery() throws Exception { diff --git a/integration-test/src/test/java/org/apache/iotdb/db/it/iotconsensusv2/stream/IoTDBIoTConsensusV2Stream3C3DBasicIT.java b/integration-test/src/test/java/org/apache/iotdb/db/it/iotconsensusv2/stream/IoTDBIoTConsensusV2Stream3C3DBasicIT.java index 856d3624bf189..2c19b642d21a1 100644 --- a/integration-test/src/test/java/org/apache/iotdb/db/it/iotconsensusv2/stream/IoTDBIoTConsensusV2Stream3C3DBasicIT.java +++ b/integration-test/src/test/java/org/apache/iotdb/db/it/iotconsensusv2/stream/IoTDBIoTConsensusV2Stream3C3DBasicIT.java @@ -44,6 +44,24 @@ public void testReplicaConsistencyAfterLeaderStop() throws Exception { super.testReplicaConsistencyAfterLeaderStop(); } + @Override + @Test + public void testFollowerCanReadConsistentDataAfterCatchUp() throws Exception { + super.testFollowerCanReadConsistentDataAfterCatchUp(); + } + + @Override + @Test + public void testReplicaConsistencyAfterDeleteAndLeaderStop() throws Exception { + super.testReplicaConsistencyAfterDeleteAndLeaderStop(); + } + + @Override + @Test + public void testReplicaConsistencyRepairAfterFollowerLosesSealedTsFile() throws Exception { + super.testReplicaConsistencyRepairAfterFollowerLosesSealedTsFile(); + } + @Override @Test public void test3C3DWriteFlushAndQuery() throws Exception { diff --git a/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/client/sync/CnToDnSyncRequestType.java b/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/client/sync/CnToDnSyncRequestType.java index 4055398ddb7ec..858afcd602289 100644 --- a/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/client/sync/CnToDnSyncRequestType.java +++ b/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/client/sync/CnToDnSyncRequestType.java @@ -36,6 +36,8 @@ public enum CnToDnSyncRequestType { REMOVE_REGION_PEER, DELETE_OLD_REGION_PEER, RESET_PEER_LIST, + GET_DATA_REGION_CONSISTENCY_SNAPSHOT, + REPAIR_TRANSFER_TSFILE, // PartitionCache INVALIDATE_PARTITION_CACHE, diff --git a/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/client/sync/SyncDataNodeClientPool.java b/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/client/sync/SyncDataNodeClientPool.java index d63d5a74f6095..588046c1f6af2 100644 --- a/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/client/sync/SyncDataNodeClientPool.java +++ b/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/client/sync/SyncDataNodeClientPool.java @@ -32,6 +32,8 @@ import org.apache.iotdb.mpp.rpc.thrift.TCreateDataRegionReq; import org.apache.iotdb.mpp.rpc.thrift.TCreatePeerReq; import org.apache.iotdb.mpp.rpc.thrift.TCreateSchemaRegionReq; +import org.apache.iotdb.mpp.rpc.thrift.TDataRegionConsistencySnapshotReq; +import org.apache.iotdb.mpp.rpc.thrift.TRepairTransferTsFileReq; import org.apache.iotdb.mpp.rpc.thrift.TInvalidateCacheReq; import org.apache.iotdb.mpp.rpc.thrift.TInvalidatePermissionCacheReq; import org.apache.iotdb.mpp.rpc.thrift.TKillQueryInstanceReq; @@ -98,6 +100,10 @@ private void buildActionMap() { actionMapBuilder.put( CnToDnSyncRequestType.DELETE_REGION, (req, client) -> client.deleteRegion((TConsensusGroupId) req)); + actionMapBuilder.put( + CnToDnSyncRequestType.GET_DATA_REGION_CONSISTENCY_SNAPSHOT, + (req, client) -> + client.getDataRegionConsistencySnapshot((TDataRegionConsistencySnapshotReq) req)); actionMapBuilder.put( CnToDnSyncRequestType.INVALIDATE_PERMISSION_CACHE, (req, client) -> client.invalidatePermissionCache((TInvalidatePermissionCacheReq) req)); @@ -134,6 +140,9 @@ private void buildActionMap() { actionMapBuilder.put( CnToDnSyncRequestType.RESET_PEER_LIST, (req, client) -> client.resetPeerList((TResetPeerListReq) req)); + actionMapBuilder.put( + CnToDnSyncRequestType.REPAIR_TRANSFER_TSFILE, + (req, client) -> client.repairTransferTsFile((TRepairTransferTsFileReq) req)); actionMapBuilder.put( CnToDnSyncRequestType.SHOW_CONFIGURATION, (req, client) -> client.showConfiguration()); actionMapBuilder.put( diff --git a/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/manager/ConfigManager.java b/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/manager/ConfigManager.java index f455edb26b8b1..cc20be5f61462 100644 --- a/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/manager/ConfigManager.java +++ b/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/manager/ConfigManager.java @@ -246,6 +246,7 @@ import org.apache.iotdb.confignode.rpc.thrift.TSubscribeReq; import org.apache.iotdb.confignode.rpc.thrift.TThrottleQuotaResp; import org.apache.iotdb.confignode.rpc.thrift.TTimeSlotList; +import org.apache.iotdb.confignode.rpc.thrift.TTriggerRegionConsistencyRepairReq; import org.apache.iotdb.confignode.rpc.thrift.TUnsetSchemaTemplateReq; import org.apache.iotdb.confignode.rpc.thrift.TUnsubscribeReq; import org.apache.iotdb.consensus.common.DataSet; @@ -2601,6 +2602,13 @@ public TSStatus removeRegion(TRemoveRegionReq req) { : status; } + public TSStatus triggerRegionConsistencyRepair(TTriggerRegionConsistencyRepairReq req) { + TSStatus status = confirmLeader(); + return status.getCode() == TSStatusCode.SUCCESS_STATUS.getStatusCode() + ? procedureManager.triggerRegionConsistencyRepair(req.getConsensusGroupId()) + : status; + } + @Override public TSStatus createCQ(TCreateCQReq req) { TSStatus status = confirmLeader(); diff --git a/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/manager/ProcedureManager.java b/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/manager/ProcedureManager.java index 646aaf66daf4f..e5d70c308666d 100644 --- a/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/manager/ProcedureManager.java +++ b/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/manager/ProcedureManager.java @@ -62,6 +62,7 @@ import org.apache.iotdb.confignode.procedure.env.ConfigNodeProcedureEnv; import org.apache.iotdb.confignode.procedure.env.RegionMaintainHandler; import org.apache.iotdb.confignode.procedure.env.RemoveDataNodeHandler; +import org.apache.iotdb.confignode.procedure.impl.consistency.LiveDataRegionRepairExecutionContext; import org.apache.iotdb.confignode.procedure.impl.cq.CreateCQProcedure; import org.apache.iotdb.confignode.procedure.impl.node.AddConfigNodeProcedure; import org.apache.iotdb.confignode.procedure.impl.node.RemoveAINodeProcedure; @@ -84,6 +85,7 @@ import org.apache.iotdb.confignode.procedure.impl.region.RegionMigrationPlan; import org.apache.iotdb.confignode.procedure.impl.region.RegionOperationProcedure; import org.apache.iotdb.confignode.procedure.impl.region.RemoveRegionPeerProcedure; +import org.apache.iotdb.confignode.procedure.impl.consistency.RepairRegionProcedure; import org.apache.iotdb.confignode.procedure.impl.schema.AlterEncodingCompressorProcedure; import org.apache.iotdb.confignode.procedure.impl.schema.AlterLogicalViewProcedure; import org.apache.iotdb.confignode.procedure.impl.schema.AlterTimeSeriesDataTypeProcedure; @@ -1374,6 +1376,41 @@ public TSStatus createRegionGroups( } } + public TSStatus triggerRegionConsistencyRepair(final TConsensusGroupId consensusGroupId) { + if (consensusGroupId == null || consensusGroupId.getType() != TConsensusGroupType.DataRegion) { + return new TSStatus(TSStatusCode.ILLEGAL_PARAMETER.getStatusCode()) + .setMessage("Replica consistency repair currently only supports DataRegion"); + } + + synchronized (this) { + for (Procedure procedure : executor.getProcedures().values()) { + if (procedure instanceof RepairRegionProcedure + && !procedure.isFinished() + && consensusGroupId.equals(((RepairRegionProcedure) procedure).getConsensusGroupId())) { + return new TSStatus(TSStatusCode.OVERLAP_WITH_EXISTING_TASK.getStatusCode()) + .setMessage( + "Replica consistency repair is already running for region " + consensusGroupId); + } + } + + try { + RepairRegionProcedure procedure = + new RepairRegionProcedure( + consensusGroupId, + new LiveDataRegionRepairExecutionContext(configManager, consensusGroupId)); + executor.submitProcedure(procedure); + return waitingProcedureFinished(procedure); + } catch (Exception e) { + LOGGER.warn( + "Failed to trigger replica consistency repair for region {}", + consensusGroupId, + e); + return new TSStatus(TSStatusCode.EXECUTE_STATEMENT_ERROR.getStatusCode()) + .setMessage(e.getMessage()); + } + } + } + /** * Generate {@link CreateTriggerProcedure} and wait until it finished. * diff --git a/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/procedure/impl/consistency/LiveDataRegionRepairExecutionContext.java b/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/procedure/impl/consistency/LiveDataRegionRepairExecutionContext.java new file mode 100644 index 0000000000000..dd46f4cf7e462 --- /dev/null +++ b/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/procedure/impl/consistency/LiveDataRegionRepairExecutionContext.java @@ -0,0 +1,549 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.iotdb.confignode.procedure.impl.consistency; + +import org.apache.iotdb.common.rpc.thrift.TConsensusGroupId; +import org.apache.iotdb.common.rpc.thrift.TConsensusGroupType; +import org.apache.iotdb.common.rpc.thrift.TDataNodeLocation; +import org.apache.iotdb.common.rpc.thrift.TRegionReplicaSet; +import org.apache.iotdb.common.rpc.thrift.TSStatus; +import org.apache.iotdb.commons.consensus.iotv2.consistency.RepairProgressTable; +import org.apache.iotdb.commons.consensus.iotv2.consistency.ibf.DiffEntry; +import org.apache.iotdb.commons.consensus.iotv2.consistency.ibf.RowRefIndex; +import org.apache.iotdb.commons.consensus.iotv2.consistency.merkle.MerkleFileContent; +import org.apache.iotdb.commons.consensus.iotv2.consistency.repair.RepairConflictResolver; +import org.apache.iotdb.commons.consensus.iotv2.consistency.repair.RepairRecord; +import org.apache.iotdb.confignode.client.sync.CnToDnSyncRequestType; +import org.apache.iotdb.confignode.client.sync.SyncDataNodeClientPool; +import org.apache.iotdb.confignode.manager.ConfigManager; +import org.apache.iotdb.mpp.rpc.thrift.TConsistencyMerkleFile; +import org.apache.iotdb.mpp.rpc.thrift.TDataRegionConsistencySnapshotReq; +import org.apache.iotdb.mpp.rpc.thrift.TDataRegionConsistencySnapshotResp; +import org.apache.iotdb.mpp.rpc.thrift.TRepairTransferTsFileReq; +import org.apache.iotdb.mpp.rpc.thrift.TTimePartitionConsistencyView; +import org.apache.iotdb.rpc.TSStatusCode; + +import java.util.ArrayList; +import java.util.Collection; +import java.util.Collections; +import java.util.Comparator; +import java.util.HashMap; +import java.util.LinkedHashMap; +import java.util.LinkedHashSet; +import java.util.List; +import java.util.Map; +import java.util.Objects; +import java.util.Set; +import java.util.TreeMap; +import java.util.stream.Collectors; + +/** + * Live runtime bridge for {@link RepairRegionProcedure}. The first production path only supports + * leader-authoritative repair when one or more followers are missing sealed TsFiles that still + * exist on the leader. + */ +public class LiveDataRegionRepairExecutionContext + implements RepairRegionProcedure.RepairExecutionContext { + + private final TConsensusGroupId consensusGroupId; + private final TDataNodeLocation leaderLocation; + private final List followerLocations; + private final Map partitionContexts; + private final Map> transferTargetsByTsFile; + + public LiveDataRegionRepairExecutionContext( + ConfigManager configManager, TConsensusGroupId consensusGroupId) { + this.consensusGroupId = requireDataRegion(consensusGroupId); + + TRegionReplicaSet replicaSet = + configManager + .getPartitionManager() + .getAllReplicaSetsMap(TConsensusGroupType.DataRegion) + .get(this.consensusGroupId); + if (replicaSet == null) { + throw new IllegalStateException("DataRegion " + consensusGroupId + " does not exist"); + } + if (replicaSet.getDataNodeLocations() == null || replicaSet.getDataNodeLocations().size() < 2) { + throw new IllegalStateException( + "DataRegion " + consensusGroupId + " has fewer than 2 replicas, no repair target exists"); + } + + int leaderId = + configManager.getLoadManager().getRegionLeaderMap().getOrDefault(this.consensusGroupId, -1); + if (leaderId <= 0) { + throw new IllegalStateException("Cannot determine leader for DataRegion " + consensusGroupId); + } + + this.leaderLocation = + replicaSet.getDataNodeLocations().stream() + .filter(location -> location.getDataNodeId() == leaderId) + .findFirst() + .orElseThrow( + () -> + new IllegalStateException( + "Leader " + + leaderId + + " is not part of replica set for " + + consensusGroupId)); + + this.followerLocations = + replicaSet.getDataNodeLocations().stream() + .filter(location -> location.getDataNodeId() != leaderId) + .sorted(Comparator.comparingInt(TDataNodeLocation::getDataNodeId)) + .collect(Collectors.toList()); + + Map> leaderSnapshot = fetchSnapshot(leaderLocation); + Map>> followerSnapshots = new LinkedHashMap<>(); + for (TDataNodeLocation followerLocation : followerLocations) { + followerSnapshots.put(followerLocation.getDataNodeId(), fetchSnapshot(followerLocation)); + } + + this.partitionContexts = new TreeMap<>(); + this.transferTargetsByTsFile = new LinkedHashMap<>(); + initializePartitionContexts(leaderSnapshot, followerSnapshots); + } + + @Override + public boolean isReplicationComplete() { + return true; + } + + @Override + public long computeSafeWatermark() { + return Long.MAX_VALUE; + } + + @Override + public List collectPendingPartitions( + long globalRepairedWatermark, long safeWatermark, RepairProgressTable repairProgressTable) { + return new ArrayList<>(partitionContexts.keySet()); + } + + @Override + public RepairRegionProcedure.PartitionRepairContext getPartitionContext(long partitionId) { + return partitionContexts.get(partitionId); + } + + @Override + public RepairRecord buildRepairRecord( + RepairRegionProcedure.PartitionRepairContext partitionContext, + DiffEntry diffEntry, + RowRefIndex rowRefIndex, + RepairConflictResolver conflictResolver) { + return null; + } + + @Override + public void transferTsFile(String tsFilePath) { + List targetDataNodes = transferTargetsByTsFile.get(tsFilePath); + if (targetDataNodes == null || targetDataNodes.isEmpty()) { + throw new IllegalStateException("No repair target found for TsFile " + tsFilePath); + } + + TRepairTransferTsFileReq request = + new TRepairTransferTsFileReq(consensusGroupId, tsFilePath, targetDataNodes); + Object response = + SyncDataNodeClientPool.getInstance() + .sendSyncRequestToDataNodeWithRetry( + leaderLocation.getInternalEndPoint(), + request, + CnToDnSyncRequestType.REPAIR_TRANSFER_TSFILE); + + TSStatus status = + response instanceof TSStatus + ? (TSStatus) response + : new TSStatus(TSStatusCode.INTERNAL_SERVER_ERROR.getStatusCode()) + .setMessage("Unexpected response type for repair TsFile transfer: " + response); + if (status.getCode() != TSStatusCode.SUCCESS_STATUS.getStatusCode()) { + throw new IllegalStateException( + "Repair transfer of TsFile " + tsFilePath + " failed: " + status.getMessage()); + } + } + + private void initializePartitionContexts( + Map> leaderSnapshot, + Map>> followerSnapshots) { + Set partitionIds = new LinkedHashSet<>(leaderSnapshot.keySet()); + followerSnapshots.values().forEach(snapshot -> partitionIds.addAll(snapshot.keySet())); + + for (Long partitionId : partitionIds.stream().sorted().collect(Collectors.toList())) { + List leaderFiles = + leaderSnapshot.getOrDefault(partitionId, Collections.emptyList()); + if (leaderFiles.isEmpty()) { + for (TDataNodeLocation followerLocation : followerLocations) { + List followerFiles = + followerSnapshots + .getOrDefault(followerLocation.getDataNodeId(), Collections.emptyMap()) + .getOrDefault(partitionId, Collections.emptyList()); + if (!followerFiles.isEmpty()) { + throw new IllegalStateException( + String.format( + "Partition %d of region %s has follower-only data on DataNode %d", + partitionId, consensusGroupId, followerLocation.getDataNodeId())); + } + } + continue; + } + + Map leaderCounts = buildDigestCounts(leaderFiles); + Map> leaderFilesByDigest = groupFilesByDigest(leaderFiles); + Map> transferTargets = new LinkedHashMap<>(); + boolean matched = true; + + for (TDataNodeLocation followerLocation : followerLocations) { + List followerFiles = + followerSnapshots + .getOrDefault(followerLocation.getDataNodeId(), Collections.emptyMap()) + .getOrDefault(partitionId, Collections.emptyList()); + Map followerCounts = buildDigestCounts(followerFiles); + + for (Map.Entry followerEntry : followerCounts.entrySet()) { + int leaderCount = leaderCounts.getOrDefault(followerEntry.getKey(), 0); + if (followerEntry.getValue() > leaderCount) { + throw new IllegalStateException( + String.format( + "Partition %d of region %s has follower-only digest %s on DataNode %d", + partitionId, + consensusGroupId, + followerEntry.getKey(), + followerLocation.getDataNodeId())); + } + } + + for (Map.Entry leaderEntry : leaderCounts.entrySet()) { + int followerCount = followerCounts.getOrDefault(leaderEntry.getKey(), 0); + int missingCount = leaderEntry.getValue() - followerCount; + if (missingCount <= 0) { + continue; + } + + matched = false; + List candidateFiles = leaderFilesByDigest.get(leaderEntry.getKey()); + for (int i = 0; i < missingCount; i++) { + SnapshotFile snapshotFile = candidateFiles.get(i); + transferTargets + .computeIfAbsent(snapshotFile.getSourceTsFilePath(), ignored -> new ArrayList<>()) + .add(followerLocation); + } + } + } + + if (matched) { + continue; + } + + LivePartitionRepairContext partitionContext = + new LivePartitionRepairContext(partitionId, leaderFiles, transferTargets); + partitionContexts.put(partitionId, partitionContext); + transferTargets.forEach( + (tsFilePath, targets) -> + transferTargetsByTsFile + .computeIfAbsent(tsFilePath, ignored -> new ArrayList<>()) + .addAll(targets)); + } + } + + private Map> fetchSnapshot(TDataNodeLocation dataNodeLocation) { + Object response = + SyncDataNodeClientPool.getInstance() + .sendSyncRequestToDataNodeWithRetry( + dataNodeLocation.getInternalEndPoint(), + new TDataRegionConsistencySnapshotReq(consensusGroupId), + CnToDnSyncRequestType.GET_DATA_REGION_CONSISTENCY_SNAPSHOT); + + if (response instanceof TSStatus) { + TSStatus status = (TSStatus) response; + throw new IllegalStateException( + "Failed to fetch consistency snapshot from DataNode " + + dataNodeLocation.getDataNodeId() + + ": " + + status.getMessage()); + } + if (!(response instanceof TDataRegionConsistencySnapshotResp)) { + throw new IllegalStateException( + "Unexpected snapshot response type from DataNode " + + dataNodeLocation.getDataNodeId() + + ": " + + response); + } + + TDataRegionConsistencySnapshotResp snapshotResp = (TDataRegionConsistencySnapshotResp) response; + if (snapshotResp.getStatus().getCode() != TSStatusCode.SUCCESS_STATUS.getStatusCode()) { + throw new IllegalStateException( + "Failed to fetch consistency snapshot from DataNode " + + dataNodeLocation.getDataNodeId() + + ": " + + snapshotResp.getStatus().getMessage()); + } + + Map> snapshot = new TreeMap<>(); + if (!snapshotResp.isSetTimePartitionViews()) { + return snapshot; + } + + for (TTimePartitionConsistencyView partitionView : snapshotResp.getTimePartitionViews()) { + List files = new ArrayList<>(); + if (partitionView.isSetMerkleFiles()) { + for (TConsistencyMerkleFile merkleFile : partitionView.getMerkleFiles()) { + files.add( + new SnapshotFile( + merkleFile.getSourceTsFilePath(), + merkleFile.getTsFileSize(), + new DigestKey(merkleFile.getFileXorHash(), merkleFile.getFileAddHash()))); + } + } + files.sort(Comparator.comparing(SnapshotFile::getSourceTsFilePath)); + snapshot.put(partitionView.getTimePartitionId(), files); + } + return snapshot; + } + + private boolean verifyPartition(long partitionId) { + Map> leaderSnapshot = fetchSnapshot(leaderLocation); + Map leaderCounts = + buildDigestCounts(leaderSnapshot.getOrDefault(partitionId, Collections.emptyList())); + + for (TDataNodeLocation followerLocation : followerLocations) { + Map> followerSnapshot = fetchSnapshot(followerLocation); + Map followerCounts = + buildDigestCounts(followerSnapshot.getOrDefault(partitionId, Collections.emptyList())); + if (!leaderCounts.equals(followerCounts)) { + return false; + } + } + return true; + } + + private static TConsensusGroupId requireDataRegion(TConsensusGroupId consensusGroupId) { + if (consensusGroupId == null + || consensusGroupId.getType() == null + || consensusGroupId.getType() != TConsensusGroupType.DataRegion) { + throw new IllegalArgumentException( + "Replica consistency repair currently only supports DataRegion"); + } + return consensusGroupId; + } + + private static Map buildDigestCounts(Collection files) { + Map counts = new HashMap<>(); + for (SnapshotFile file : files) { + counts.merge(file.getDigestKey(), 1, Integer::sum); + } + return counts; + } + + private static Map> groupFilesByDigest(List files) { + Map> grouped = new LinkedHashMap<>(); + for (SnapshotFile file : files) { + grouped.computeIfAbsent(file.getDigestKey(), ignored -> new ArrayList<>()).add(file); + } + return grouped; + } + + private final class LivePartitionRepairContext + implements RepairRegionProcedure.PartitionRepairContext { + + private final long partitionId; + private final List leaderMerkleFiles; + private final Map leaderMerkleFileByPath; + private final Map leaderTsFileSizeByPath; + private final Map> transferTargets; + + private LivePartitionRepairContext( + long partitionId, + List leaderFiles, + Map> transferTargets) { + this.partitionId = partitionId; + this.leaderMerkleFiles = + leaderFiles.stream() + .map( + file -> + new MerkleFileContent( + file.getDigestKey().getFileXorHash(), + file.getDigestKey().getFileAddHash(), + Collections.emptyList(), + file.getSourceTsFilePath())) + .collect(Collectors.toList()); + this.leaderMerkleFileByPath = + this.leaderMerkleFiles.stream() + .collect( + Collectors.toMap( + MerkleFileContent::getSourceTsFilePath, + file -> file, + (left, right) -> left, + LinkedHashMap::new)); + this.leaderTsFileSizeByPath = + leaderFiles.stream() + .collect( + Collectors.toMap( + SnapshotFile::getSourceTsFilePath, + SnapshotFile::getTsFileSize, + (left, right) -> left, + LinkedHashMap::new)); + this.transferTargets = + transferTargets.entrySet().stream() + .collect( + Collectors.toMap( + Map.Entry::getKey, + entry -> Collections.unmodifiableList(new ArrayList<>(entry.getValue())), + (left, right) -> left, + LinkedHashMap::new)); + } + + @Override + public long getPartitionId() { + return partitionId; + } + + @Override + public boolean isRootHashMatched() { + return false; + } + + @Override + public List getLeaderMerkleFiles() { + return leaderMerkleFiles; + } + + @Override + public List getMismatchedLeaderMerkleFiles() { + return transferTargets.keySet().stream() + .map(leaderMerkleFileByPath::get) + .filter(Objects::nonNull) + .collect(Collectors.toList()); + } + + @Override + public RowRefIndex getRowRefIndex() { + return null; + } + + @Override + public List decodeDiffs() { + return Collections.emptyList(); + } + + @Override + public boolean isDiffDecodeSuccessful() { + return false; + } + + @Override + public long estimateDiffCount() { + return 0L; + } + + @Override + public List getFallbackTsFiles() { + return new ArrayList<>(transferTargets.keySet()); + } + + @Override + public long getTsFileSize(String tsFilePath) { + Long tsFileSize = leaderTsFileSizeByPath.get(tsFilePath); + if (tsFileSize == null) { + throw new IllegalArgumentException("Unknown leader TsFile " + tsFilePath); + } + return tsFileSize; + } + + @Override + public int getTotalPointCount(String tsFilePath) { + return 0; + } + + @Override + public boolean shouldForceDirectTsFileTransfer() { + return true; + } + + @Override + public boolean verify( + Map + repairPlans, + boolean rootHashMatched) { + return verifyPartition(partitionId); + } + } + + private static final class SnapshotFile { + private final String sourceTsFilePath; + private final long tsFileSize; + private final DigestKey digestKey; + + private SnapshotFile(String sourceTsFilePath, long tsFileSize, DigestKey digestKey) { + this.sourceTsFilePath = sourceTsFilePath; + this.tsFileSize = tsFileSize; + this.digestKey = digestKey; + } + + private String getSourceTsFilePath() { + return sourceTsFilePath; + } + + private long getTsFileSize() { + return tsFileSize; + } + + private DigestKey getDigestKey() { + return digestKey; + } + } + + private static final class DigestKey { + private final long fileXorHash; + private final long fileAddHash; + + private DigestKey(long fileXorHash, long fileAddHash) { + this.fileXorHash = fileXorHash; + this.fileAddHash = fileAddHash; + } + + private long getFileXorHash() { + return fileXorHash; + } + + private long getFileAddHash() { + return fileAddHash; + } + + @Override + public boolean equals(Object o) { + if (this == o) { + return true; + } + if (!(o instanceof DigestKey)) { + return false; + } + DigestKey digestKey = (DigestKey) o; + return fileXorHash == digestKey.fileXorHash && fileAddHash == digestKey.fileAddHash; + } + + @Override + public int hashCode() { + return Objects.hash(fileXorHash, fileAddHash); + } + + @Override + public String toString() { + return "DigestKey{" + "xor=" + fileXorHash + ", add=" + fileAddHash + '}'; + } + } +} diff --git a/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/procedure/impl/consistency/RepairRegionProcedure.java b/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/procedure/impl/consistency/RepairRegionProcedure.java new file mode 100644 index 0000000000000..e3405e98040af --- /dev/null +++ b/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/procedure/impl/consistency/RepairRegionProcedure.java @@ -0,0 +1,982 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.iotdb.confignode.procedure.impl.consistency; + +import org.apache.iotdb.common.rpc.thrift.TConsensusGroupId; +import org.apache.iotdb.commons.consensus.iotv2.consistency.RepairProgressTable; +import org.apache.iotdb.commons.consensus.iotv2.consistency.RepairProgressTable.RegionRepairStatus; +import org.apache.iotdb.commons.consensus.iotv2.consistency.ibf.DiffEntry; +import org.apache.iotdb.commons.consensus.iotv2.consistency.ibf.RowRefIndex; +import org.apache.iotdb.commons.consensus.iotv2.consistency.merkle.MerkleFileContent; +import org.apache.iotdb.commons.consensus.iotv2.consistency.repair.DiffAttribution; +import org.apache.iotdb.commons.consensus.iotv2.consistency.repair.ModEntrySummary; +import org.apache.iotdb.commons.consensus.iotv2.consistency.repair.RepairConflictResolver; +import org.apache.iotdb.commons.consensus.iotv2.consistency.repair.RepairCostModel; +import org.apache.iotdb.commons.consensus.iotv2.consistency.repair.RepairPlan; +import org.apache.iotdb.commons.consensus.iotv2.consistency.repair.RepairRecord; +import org.apache.iotdb.commons.consensus.iotv2.consistency.repair.RepairSession; +import org.apache.iotdb.commons.consensus.iotv2.consistency.repair.RepairStrategy; +import org.apache.iotdb.confignode.procedure.env.ConfigNodeProcedureEnv; +import org.apache.iotdb.confignode.procedure.impl.StateMachineProcedure; +import org.apache.iotdb.confignode.procedure.state.consistency.RepairState; +import org.apache.iotdb.confignode.procedure.store.ProcedureType; + +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.io.DataOutputStream; +import java.io.IOException; +import java.nio.ByteBuffer; +import java.nio.charset.StandardCharsets; +import java.util.ArrayList; +import java.util.Collections; +import java.util.Comparator; +import java.util.HashSet; +import java.util.LinkedHashMap; +import java.util.List; +import java.util.Map; +import java.util.Objects; +import java.util.Set; +import java.util.UUID; +import java.util.concurrent.ConcurrentHashMap; + +/** + * RepairRegionProcedure orchestrates the consistency check and repair lifecycle for a single + * consensus group. The procedure is intentionally isolated from transport details: all external + * interactions are supplied by a {@link RepairExecutionContext}, while the procedure itself + * persists the state-machine and applies the repair policy. + */ +public class RepairRegionProcedure + extends StateMachineProcedure { + + private static final Logger LOGGER = LoggerFactory.getLogger(RepairRegionProcedure.class); + + private static final RepairCostModel REPAIR_COST_MODEL = new RepairCostModel(); + private static final ConcurrentHashMap EXECUTION_CONTEXTS = + new ConcurrentHashMap<>(); + + private TConsensusGroupId consensusGroupId; + private long tSafe; + private long globalRepairedWatermark; + private List pendingPartitions; + private int currentPartitionIndex; + private boolean hashMatched; + private String executionContextId; + private String lastFailureReason; + + private transient RepairExecutionContext executionContext; + private transient RepairProgressTable repairProgressTable; + private transient PartitionRepairContext currentPartitionContext; + private transient DiffAttribution diffAttribution; + private transient RowRefIndex currentRowRefIndex; + private transient Map> attributedDiffs; + private transient Map repairPlans; + private transient Set executedTsFileTransfers; + private transient Set executedPointStreamingPlans; + private transient List decodedDiffs; + private transient RepairSession repairSession; + private transient long estimatedDiffCount; + + /** Required for deserialization. */ + public RepairRegionProcedure() { + this(null, (String) null); + } + + public RepairRegionProcedure(TConsensusGroupId consensusGroupId) { + this(consensusGroupId, (String) null); + } + + public RepairRegionProcedure( + TConsensusGroupId consensusGroupId, RepairExecutionContext executionContext) { + this(consensusGroupId, registerExecutionContext(executionContext)); + this.executionContext = executionContext; + } + + public RepairRegionProcedure(TConsensusGroupId consensusGroupId, String executionContextId) { + this.consensusGroupId = consensusGroupId; + this.executionContextId = executionContextId; + this.pendingPartitions = new ArrayList<>(); + this.currentPartitionIndex = 0; + this.hashMatched = false; + initializeTransientState(); + } + + @Override + protected Flow executeFromState(ConfigNodeProcedureEnv env, RepairState state) + throws InterruptedException { + try { + switch (state) { + case INIT: + LOGGER.info("RepairRegionProcedure: INIT for group {}", consensusGroupId); + setNextState(RepairState.CHECK_SYNC_LAG); + break; + + case CHECK_SYNC_LAG: + LOGGER.info("RepairRegionProcedure: CHECK_SYNC_LAG for group {}", consensusGroupId); + if (!checkSyncLagCompleted(env)) { + LOGGER.info( + "RepairRegionProcedure: skipping group {} because replication is not complete", + consensusGroupId); + finishWithoutRepair(); + return Flow.NO_MORE_STATE; + } + setNextState(RepairState.COMPUTE_WATERMARK); + break; + + case COMPUTE_WATERMARK: + LOGGER.info("RepairRegionProcedure: COMPUTE_WATERMARK for group {}", consensusGroupId); + computeWatermarkAndPartitions(env); + if (pendingPartitions.isEmpty()) { + LOGGER.info("No pending partitions to check for group {}", consensusGroupId); + finishWithoutRepair(); + return Flow.NO_MORE_STATE; + } + setNextState(RepairState.BUILD_MERKLE_VIEW); + break; + + case BUILD_MERKLE_VIEW: + LOGGER.info( + "RepairRegionProcedure: BUILD_MERKLE_VIEW for partition {} of group {}", + getCurrentPartitionId(), + consensusGroupId); + buildMerkleView(env); + setNextState(RepairState.COMPARE_ROOT_HASH); + break; + + case COMPARE_ROOT_HASH: + hashMatched = compareRootHash(env); + if (hashMatched) { + LOGGER.info("Root hash matched for partition {}", getCurrentPartitionId()); + setNextState(RepairState.COMMIT_PARTITION); + } else { + LOGGER.info("Root hash mismatched for partition {}", getCurrentPartitionId()); + setNextState(RepairState.DRILL_DOWN); + } + break; + + case DRILL_DOWN: + LOGGER.info( + "RepairRegionProcedure: DRILL_DOWN for partition {}", getCurrentPartitionId()); + drillDown(env); + setNextState(RepairState.SMALL_TSFILE_SHORT_CIRCUIT); + break; + + case SMALL_TSFILE_SHORT_CIRCUIT: + if (prepareDirectTransferOnlyPlans()) { + setNextState(RepairState.EXECUTE_TSFILE_TRANSFER); + } else { + boolean hasSmallFiles = handleSmallTsFileShortCircuit(env); + if (hasLargeFilesNeedingIBF()) { + setNextState(RepairState.NEGOTIATE_KEY_MAPPING); + } else if (hasSmallFiles) { + setNextState(RepairState.EXECUTE_TSFILE_TRANSFER); + } else { + setNextState(RepairState.COMMIT_PARTITION); + } + } + break; + + case NEGOTIATE_KEY_MAPPING: + LOGGER.info("RepairRegionProcedure: NEGOTIATE_KEY_MAPPING"); + negotiateKeyMapping(env); + setNextState(RepairState.ESTIMATE_DIFF); + break; + + case ESTIMATE_DIFF: + estimateDiff(env); + setNextState(RepairState.EXCHANGE_IBF); + break; + + case EXCHANGE_IBF: + exchangeIBF(env); + setNextState(RepairState.DECODE_DIFF); + break; + + case DECODE_DIFF: + boolean decodeSuccess = decodeDiff(env); + if (decodeSuccess) { + setNextState(RepairState.ATTRIBUTE_DIFFS); + } else { + LOGGER.warn( + "IBF decode failed for partition {}, falling back to direct transfer", + getCurrentPartitionId()); + setNextState(RepairState.EXECUTE_TSFILE_TRANSFER); + } + break; + + case ATTRIBUTE_DIFFS: + attributeDiffs(env); + setNextState(RepairState.SELECT_REPAIR_STRATEGY); + break; + + case SELECT_REPAIR_STRATEGY: + selectRepairStrategy(env); + if (hasPendingTsFileTransfers()) { + setNextState(RepairState.EXECUTE_TSFILE_TRANSFER); + } else if (hasPendingPointStreaming()) { + setNextState(RepairState.EXECUTE_POINT_STREAMING); + } else { + setNextState(RepairState.VERIFY_REPAIR); + } + break; + + case EXECUTE_TSFILE_TRANSFER: + LOGGER.info("RepairRegionProcedure: EXECUTE_TSFILE_TRANSFER"); + executeTsFileTransfer(env); + if (hasPendingPointStreaming()) { + setNextState(RepairState.EXECUTE_POINT_STREAMING); + } else { + setNextState(RepairState.VERIFY_REPAIR); + } + break; + + case EXECUTE_POINT_STREAMING: + LOGGER.info("RepairRegionProcedure: EXECUTE_POINT_STREAMING"); + executePointStreaming(env); + setNextState(RepairState.VERIFY_REPAIR); + break; + + case VERIFY_REPAIR: + boolean verified = verifyRepair(env); + if (verified) { + setNextState(RepairState.COMMIT_PARTITION); + } else { + lastFailureReason = + "Verification failed for partition " + + getCurrentPartitionId() + + " in group " + + consensusGroupId; + LOGGER.warn(lastFailureReason); + setNextState(RepairState.ROLLBACK); + } + break; + + case COMMIT_PARTITION: + commitPartition(env); + currentPartitionIndex++; + if (currentPartitionIndex < pendingPartitions.size()) { + setNextState(RepairState.BUILD_MERKLE_VIEW); + } else { + setNextState(RepairState.ADVANCE_WATERMARK); + } + break; + + case ADVANCE_WATERMARK: + advanceWatermark(env); + return Flow.NO_MORE_STATE; + + case ROLLBACK: + LOGGER.warn("RepairRegionProcedure: ROLLBACK for group {}", consensusGroupId); + rollback(env); + return Flow.NO_MORE_STATE; + + case DONE: + finishWithoutRepair(); + return Flow.NO_MORE_STATE; + + default: + LOGGER.error("Unknown state: {}", state); + finishWithoutRepair(); + return Flow.NO_MORE_STATE; + } + } catch (Exception e) { + lastFailureReason = e.getMessage(); + LOGGER.error( + "Error in RepairRegionProcedure state {} for group {}: {}", + state, + consensusGroupId, + e.getMessage(), + e); + setNextState(RepairState.ROLLBACK); + } + return Flow.HAS_MORE_STATE; + } + + private boolean checkSyncLagCompleted(ConfigNodeProcedureEnv env) { + return requireExecutionContext().isReplicationComplete(); + } + + private void computeWatermarkAndPartitions(ConfigNodeProcedureEnv env) { + RepairExecutionContext context = requireExecutionContext(); + RepairProgressTable progressTable = getOrCreateRepairProgressTable(); + progressTable.setRegionStatus(RegionRepairStatus.RUNNING); + globalRepairedWatermark = + Math.max(globalRepairedWatermark, progressTable.getGlobalRepairedWatermark()); + tSafe = Math.max(globalRepairedWatermark, context.computeSafeWatermark()); + + pendingPartitions.clear(); + if (tSafe <= globalRepairedWatermark) { + return; + } + + List candidatePartitions = + safeList(context.collectPendingPartitions(globalRepairedWatermark, tSafe, progressTable)); + candidatePartitions.stream() + .filter(Objects::nonNull) + .filter(partitionId -> partitionId <= tSafe) + .distinct() + .sorted(Comparator.naturalOrder()) + .forEach( + partitionId -> { + pendingPartitions.add(partitionId); + progressTable.getOrCreatePartition(partitionId); + }); + persistRepairProgressTable(progressTable); + } + + private void buildMerkleView(ConfigNodeProcedureEnv env) { + resetCurrentPartitionState(); + long partitionId = getCurrentPartitionId(); + currentPartitionContext = requireExecutionContext().getPartitionContext(partitionId); + if (currentPartitionContext == null) { + throw new IllegalStateException( + "Missing partition context for partition " + + partitionId + + " in group " + + consensusGroupId); + } + if (currentPartitionContext.getPartitionId() != partitionId) { + throw new IllegalStateException( + "Partition context mismatch: expected " + + partitionId + + ", actual " + + currentPartitionContext.getPartitionId()); + } + } + + private boolean compareRootHash(ConfigNodeProcedureEnv env) { + return requireCurrentPartitionContext().isRootHashMatched(); + } + + private void drillDown(ConfigNodeProcedureEnv env) { + PartitionRepairContext partitionContext = requireCurrentPartitionContext(); + List mismatchedFiles = + safeList(partitionContext.getMismatchedLeaderMerkleFiles()); + LOGGER.info( + "Partition {} has {} mismatched TsFiles after Merkle drill-down", + partitionContext.getPartitionId(), + mismatchedFiles.size()); + if (mismatchedFiles.isEmpty()) { + LOGGER.warn( + "Partition {} reported root mismatch but returned no mismatched TsFiles", + partitionContext.getPartitionId()); + } + } + + private boolean prepareDirectTransferOnlyPlans() { + PartitionRepairContext partitionContext = requireCurrentPartitionContext(); + if (!partitionContext.shouldForceDirectTsFileTransfer()) { + return false; + } + + List fallbackTsFiles = safeList(partitionContext.getFallbackTsFiles()); + if (fallbackTsFiles.isEmpty()) { + throw new IllegalStateException( + "Partition " + + partitionContext.getPartitionId() + + " requested direct TsFile transfer but provided no fallback files"); + } + + for (String tsFilePath : fallbackTsFiles) { + repairPlans.putIfAbsent( + tsFilePath, + RepairPlan.directTransfer(tsFilePath, partitionContext.getTsFileSize(tsFilePath))); + } + return true; + } + + private boolean handleSmallTsFileShortCircuit(ConfigNodeProcedureEnv env) { + boolean hasSmallFiles = false; + for (MerkleFileContent content : + safeList(requireCurrentPartitionContext().getMismatchedLeaderMerkleFiles())) { + String tsFilePath = content.getSourceTsFilePath(); + long tsFileSize = requireCurrentPartitionContext().getTsFileSize(tsFilePath); + if (REPAIR_COST_MODEL.shouldBypassIBF(tsFileSize)) { + hasSmallFiles = true; + repairPlans.put(tsFilePath, RepairPlan.directTransfer(tsFilePath, tsFileSize)); + } + } + return hasSmallFiles; + } + + private boolean hasLargeFilesNeedingIBF() { + for (MerkleFileContent content : + safeList(requireCurrentPartitionContext().getMismatchedLeaderMerkleFiles())) { + String tsFilePath = content.getSourceTsFilePath(); + if (repairPlans.containsKey(tsFilePath)) { + continue; + } + long tsFileSize = requireCurrentPartitionContext().getTsFileSize(tsFilePath); + if (!REPAIR_COST_MODEL.shouldBypassIBF(tsFileSize)) { + return true; + } + } + return false; + } + + private void negotiateKeyMapping(ConfigNodeProcedureEnv env) { + currentRowRefIndex = requireCurrentPartitionContext().getRowRefIndex(); + if (currentRowRefIndex == null) { + throw new IllegalStateException( + "Large-file diff localization requires a RowRefIndex for partition " + + getCurrentPartitionId()); + } + } + + private void estimateDiff(ConfigNodeProcedureEnv env) { + estimatedDiffCount = requireCurrentPartitionContext().estimateDiffCount(); + if (estimatedDiffCount <= 0 && requireCurrentPartitionContext().isDiffDecodeSuccessful()) { + estimatedDiffCount = safeList(requireCurrentPartitionContext().decodeDiffs()).size(); + } + LOGGER.info( + "Estimated diff count for partition {} is {}", getCurrentPartitionId(), estimatedDiffCount); + } + + private void exchangeIBF(ConfigNodeProcedureEnv env) { + decodedDiffs.clear(); + decodedDiffs.addAll(safeList(requireCurrentPartitionContext().decodeDiffs())); + } + + private boolean decodeDiff(ConfigNodeProcedureEnv env) { + PartitionRepairContext partitionContext = requireCurrentPartitionContext(); + if (!partitionContext.isDiffDecodeSuccessful()) { + for (String tsFilePath : safeList(partitionContext.getFallbackTsFiles())) { + repairPlans.put( + tsFilePath, + RepairPlan.directTransfer(tsFilePath, partitionContext.getTsFileSize(tsFilePath))); + } + return false; + } + if (decodedDiffs.isEmpty()) { + decodedDiffs.addAll(safeList(partitionContext.decodeDiffs())); + } + return true; + } + + private void attributeDiffs(ConfigNodeProcedureEnv env) { + attributedDiffs = + diffAttribution.attributeToSourceTsFiles( + decodedDiffs, + currentRowRefIndex, + safeList(requireCurrentPartitionContext().getLeaderMerkleFiles())); + } + + private void selectRepairStrategy(ConfigNodeProcedureEnv env) { + PartitionRepairContext partitionContext = requireCurrentPartitionContext(); + for (Map.Entry> entry : attributedDiffs.entrySet()) { + String tsFilePath = entry.getKey(); + if (repairPlans.containsKey(tsFilePath)) { + continue; + } + List diffs = entry.getValue(); + long tsFileSize = partitionContext.getTsFileSize(tsFilePath); + int totalPointCount = partitionContext.getTotalPointCount(tsFilePath); + RepairStrategy strategy = + REPAIR_COST_MODEL.selectStrategy(tsFileSize, totalPointCount, diffs.size()); + if (strategy == RepairStrategy.DIRECT_TSFILE_TRANSFER) { + repairPlans.put(tsFilePath, RepairPlan.directTransfer(tsFilePath, tsFileSize)); + } else { + repairPlans.put(tsFilePath, RepairPlan.pointStreaming(tsFilePath, tsFileSize, diffs)); + } + } + } + + private boolean hasPendingTsFileTransfers() { + return repairPlans.values().stream() + .anyMatch( + plan -> + plan.getStrategy() == RepairStrategy.DIRECT_TSFILE_TRANSFER + && !executedTsFileTransfers.contains(plan.getTsFilePath())); + } + + private boolean hasPendingPointStreaming() { + return repairPlans.values().stream() + .anyMatch( + plan -> + plan.getStrategy() == RepairStrategy.POINT_STREAMING + && !executedPointStreamingPlans.contains(plan.getTsFilePath())); + } + + private void executeTsFileTransfer(ConfigNodeProcedureEnv env) { + for (RepairPlan plan : repairPlans.values()) { + if (plan.getStrategy() != RepairStrategy.DIRECT_TSFILE_TRANSFER + || executedTsFileTransfers.contains(plan.getTsFilePath())) { + continue; + } + try { + requireExecutionContext().transferTsFile(plan.getTsFilePath()); + executedTsFileTransfers.add(plan.getTsFilePath()); + } catch (Exception e) { + throw new IllegalStateException( + "Failed to transfer TsFile " + plan.getTsFilePath() + ": " + e.getMessage(), e); + } + } + } + + private void executePointStreaming(ConfigNodeProcedureEnv env) { + RepairConflictResolver conflictResolver = + new RepairConflictResolver( + safeList(requireCurrentPartitionContext().getLeaderDeletions()), + safeList(requireCurrentPartitionContext().getFollowerDeletions())); + repairSession = requireExecutionContext().createRepairSession(getCurrentPartitionId()); + + for (RepairPlan plan : repairPlans.values()) { + if (plan.getStrategy() != RepairStrategy.POINT_STREAMING + || executedPointStreamingPlans.contains(plan.getTsFilePath())) { + continue; + } + for (DiffEntry diff : plan.getDiffs()) { + RepairRecord record = + requireExecutionContext() + .buildRepairRecord( + requireCurrentPartitionContext(), diff, currentRowRefIndex, conflictResolver); + if (record != null) { + repairSession.stage(record); + } + } + executedPointStreamingPlans.add(plan.getTsFilePath()); + } + + if (repairSession.getStagedCount() > 0 && !repairSession.promoteAtomically()) { + throw new IllegalStateException( + "Failed to atomically promote repair session " + repairSession.getSessionId()); + } + } + + private boolean verifyRepair(ConfigNodeProcedureEnv env) { + return hashMatched || requireCurrentPartitionContext().verify(repairPlans, hashMatched); + } + + private void commitPartition(ConfigNodeProcedureEnv env) { + long partitionId = getCurrentPartitionId(); + RepairProgressTable progressTable = getOrCreateRepairProgressTable(); + long repairedTo = + Math.max(tSafe, progressTable.getOrCreatePartition(partitionId).getRepairedTo()); + progressTable.commitPartition(partitionId, repairedTo); + requireExecutionContext().onPartitionCommitted(partitionId, repairedTo, progressTable); + if (repairSession != null) { + repairSession.cleanup(); + } + persistRepairProgressTable(progressTable); + LOGGER.info("Committed partition {} for group {}", partitionId, consensusGroupId); + resetCurrentPartitionState(); + } + + private void advanceWatermark(ConfigNodeProcedureEnv env) { + RepairProgressTable progressTable = getOrCreateRepairProgressTable(); + globalRepairedWatermark = progressTable.advanceGlobalWatermark(); + progressTable.setRegionStatus(RegionRepairStatus.IDLE); + requireExecutionContext().onWatermarkAdvanced(globalRepairedWatermark, progressTable); + persistRepairProgressTable(progressTable); + LOGGER.info( + "Advanced repair watermark for group {} to {}", consensusGroupId, globalRepairedWatermark); + cleanupExecutionContext(); + } + + protected void rollback(ConfigNodeProcedureEnv env) { + long partitionId = getCurrentPartitionId(); + RepairProgressTable progressTable = getOrCreateRepairProgressTable(); + if (partitionId >= 0) { + progressTable.failPartition( + partitionId, lastFailureReason == null ? "Unknown repair failure" : lastFailureReason); + } + progressTable.setRegionStatus(RegionRepairStatus.FAILED); + if (repairSession != null) { + repairSession.abort(); + } + RepairExecutionContext context = getExecutionContextIfPresent(); + if (context != null && partitionId >= 0) { + context.rollbackPartition(partitionId, repairSession, progressTable); + } + persistRepairProgressTable(progressTable); + LOGGER.warn("Rolled back repair for group {}", consensusGroupId); + cleanupExecutionContext(); + } + + private long getCurrentPartitionId() { + if (pendingPartitions.isEmpty() || currentPartitionIndex >= pendingPartitions.size()) { + return -1; + } + return pendingPartitions.get(currentPartitionIndex); + } + + @Override + protected RepairState getState(int stateId) { + return RepairState.values()[stateId]; + } + + @Override + protected int getStateId(RepairState repairState) { + return repairState.ordinal(); + } + + @Override + protected RepairState getInitialState() { + return RepairState.INIT; + } + + @Override + protected void rollbackState(ConfigNodeProcedureEnv env, RepairState state) { + LOGGER.warn("Rollback requested for state {} in group {}", state, consensusGroupId); + } + + @Override + public void serialize(DataOutputStream stream) throws IOException { + stream.writeShort(ProcedureType.REPAIR_REGION_PROCEDURE.getTypeCode()); + super.serialize(stream); + + stream.writeBoolean(consensusGroupId != null); + if (consensusGroupId != null) { + stream.writeInt(consensusGroupId.getId()); + stream.writeInt(consensusGroupId.getType().getValue()); + } + stream.writeLong(tSafe); + stream.writeLong(globalRepairedWatermark); + stream.writeInt(currentPartitionIndex); + stream.writeBoolean(hashMatched); + writeString(stream, executionContextId); + writeString(stream, lastFailureReason); + + stream.writeInt(pendingPartitions.size()); + for (long partitionId : pendingPartitions) { + stream.writeLong(partitionId); + } + } + + @Override + public void deserialize(ByteBuffer byteBuffer) { + super.deserialize(byteBuffer); + + if (byteBuffer.get() != 0) { + int groupId = byteBuffer.getInt(); + int groupType = byteBuffer.getInt(); + this.consensusGroupId = new TConsensusGroupId(); + this.consensusGroupId.setId(groupId); + this.consensusGroupId.setType( + org.apache.iotdb.common.rpc.thrift.TConsensusGroupType.findByValue(groupType)); + } else { + this.consensusGroupId = null; + } + + this.tSafe = byteBuffer.getLong(); + this.globalRepairedWatermark = byteBuffer.getLong(); + this.currentPartitionIndex = byteBuffer.getInt(); + this.hashMatched = byteBuffer.get() != 0; + this.executionContextId = readString(byteBuffer); + this.lastFailureReason = readString(byteBuffer); + + int partitionCount = byteBuffer.getInt(); + this.pendingPartitions = new ArrayList<>(partitionCount); + for (int i = 0; i < partitionCount; i++) { + pendingPartitions.add(byteBuffer.getLong()); + } + initializeTransientState(); + } + + public TConsensusGroupId getConsensusGroupId() { + return consensusGroupId; + } + + public long getGlobalRepairedWatermark() { + return globalRepairedWatermark; + } + + public String getExecutionContextId() { + return executionContextId; + } + + public static String registerExecutionContext(RepairExecutionContext executionContext) { + if (executionContext == null) { + return null; + } + String contextId = UUID.randomUUID().toString(); + EXECUTION_CONTEXTS.put(contextId, executionContext); + return contextId; + } + + public static void unregisterExecutionContext(String contextId) { + if (contextId == null) { + return; + } + EXECUTION_CONTEXTS.remove(contextId); + } + + @Override + public boolean equals(Object o) { + if (this == o) { + return true; + } + if (!(o instanceof RepairRegionProcedure)) { + return false; + } + RepairRegionProcedure that = (RepairRegionProcedure) o; + return tSafe == that.tSafe + && globalRepairedWatermark == that.globalRepairedWatermark + && currentPartitionIndex == that.currentPartitionIndex + && hashMatched == that.hashMatched + && Objects.equals(consensusGroupId, that.consensusGroupId) + && Objects.equals(pendingPartitions, that.pendingPartitions) + && Objects.equals(executionContextId, that.executionContextId) + && Objects.equals(lastFailureReason, that.lastFailureReason); + } + + @Override + public int hashCode() { + return Objects.hash( + consensusGroupId, + tSafe, + globalRepairedWatermark, + pendingPartitions, + currentPartitionIndex, + hashMatched, + executionContextId, + lastFailureReason); + } + + private void initializeTransientState() { + this.executionContext = getExecutionContextIfPresent(); + this.currentPartitionContext = null; + this.repairProgressTable = null; + this.diffAttribution = new DiffAttribution(); + this.currentRowRefIndex = null; + this.attributedDiffs = new LinkedHashMap<>(); + this.repairPlans = new LinkedHashMap<>(); + this.executedTsFileTransfers = new HashSet<>(); + this.executedPointStreamingPlans = new HashSet<>(); + this.decodedDiffs = new ArrayList<>(); + this.repairSession = null; + this.estimatedDiffCount = 0L; + } + + private void resetCurrentPartitionState() { + this.currentPartitionContext = null; + this.currentRowRefIndex = null; + this.attributedDiffs.clear(); + this.repairPlans.clear(); + this.executedTsFileTransfers.clear(); + this.executedPointStreamingPlans.clear(); + this.decodedDiffs.clear(); + this.repairSession = null; + this.hashMatched = false; + this.estimatedDiffCount = 0L; + this.lastFailureReason = null; + } + + private RepairExecutionContext requireExecutionContext() { + RepairExecutionContext context = getExecutionContextIfPresent(); + if (context == null) { + throw new IllegalStateException( + "No execution context registered for repair procedure of group " + + consensusGroupId + + ". Expected context id: " + + executionContextId); + } + return context; + } + + private RepairExecutionContext getExecutionContextIfPresent() { + if (executionContext == null && executionContextId != null) { + executionContext = EXECUTION_CONTEXTS.get(executionContextId); + } + return executionContext; + } + + private PartitionRepairContext requireCurrentPartitionContext() { + if (currentPartitionContext == null) { + throw new IllegalStateException( + "Partition context has not been prepared for partition " + getCurrentPartitionId()); + } + return currentPartitionContext; + } + + private RepairProgressTable getOrCreateRepairProgressTable() { + if (repairProgressTable == null) { + RepairExecutionContext context = getExecutionContextIfPresent(); + if (context != null) { + repairProgressTable = context.loadRepairProgressTable(toConsensusGroupKey(consensusGroupId)); + } + if (repairProgressTable == null) { + repairProgressTable = new RepairProgressTable(toConsensusGroupKey(consensusGroupId)); + } + repairProgressTable.setRegionStatus(RegionRepairStatus.RUNNING); + for (int i = 0; i < pendingPartitions.size(); i++) { + long partitionId = pendingPartitions.get(i); + RepairProgressTable.PartitionProgress progress = + repairProgressTable.getOrCreatePartition(partitionId); + if (i < currentPartitionIndex) { + progress.markVerified(tSafe); + } + } + repairProgressTable.advanceGlobalWatermark(); + } + return repairProgressTable; + } + + private void finishWithoutRepair() { + if (repairProgressTable != null) { + repairProgressTable.setRegionStatus(RegionRepairStatus.IDLE); + persistRepairProgressTable(repairProgressTable); + } + cleanupExecutionContext(); + } + + private void persistRepairProgressTable(RepairProgressTable progressTable) { + RepairExecutionContext context = getExecutionContextIfPresent(); + if (context != null && progressTable != null) { + context.persistRepairProgressTable(progressTable); + } + } + + private void cleanupExecutionContext() { + RepairExecutionContext context = getExecutionContextIfPresent(); + if (context != null) { + try { + context.close(); + } catch (Exception e) { + LOGGER.warn( + "Failed to close repair execution context {}: {}", + executionContextId, + e.getMessage(), + e); + } + } + unregisterExecutionContext(executionContextId); + executionContext = null; + } + + private static List safeList(List list) { + return list == null ? Collections.emptyList() : list; + } + + private static String toConsensusGroupKey(TConsensusGroupId consensusGroupId) { + if (consensusGroupId == null) { + return "unknown"; + } + return consensusGroupId.getType() + "-" + consensusGroupId.getId(); + } + + private static void writeString(DataOutputStream stream, String value) throws IOException { + if (value == null) { + stream.writeInt(-1); + return; + } + byte[] bytes = value.getBytes(StandardCharsets.UTF_8); + stream.writeInt(bytes.length); + stream.write(bytes); + } + + private static String readString(ByteBuffer byteBuffer) { + int length = byteBuffer.getInt(); + if (length < 0) { + return null; + } + byte[] bytes = new byte[length]; + byteBuffer.get(bytes); + return new String(bytes, StandardCharsets.UTF_8); + } + + /** + * Bridge between the state machine and the transport/runtime-specific implementation that + * provides Merkle snapshots, decoded diffs and repair primitives. + */ + public interface RepairExecutionContext extends AutoCloseable { + + boolean isReplicationComplete(); + + long computeSafeWatermark(); + + List collectPendingPartitions( + long globalRepairedWatermark, long safeWatermark, RepairProgressTable repairProgressTable); + + PartitionRepairContext getPartitionContext(long partitionId); + + RepairRecord buildRepairRecord( + PartitionRepairContext partitionContext, + DiffEntry diffEntry, + RowRefIndex rowRefIndex, + RepairConflictResolver conflictResolver); + + void transferTsFile(String tsFilePath) throws Exception; + + default RepairSession createRepairSession(long partitionId) { + return new RepairSession(partitionId); + } + + default RepairProgressTable loadRepairProgressTable(String consensusGroupKey) { + return null; + } + + default void persistRepairProgressTable(RepairProgressTable repairProgressTable) {} + + default void onPartitionCommitted( + long partitionId, long repairedTo, RepairProgressTable repairProgressTable) {} + + default void onWatermarkAdvanced( + long globalWatermark, RepairProgressTable repairProgressTable) {} + + default void rollbackPartition( + long partitionId, RepairSession repairSession, RepairProgressTable repairProgressTable) {} + + @Override + default void close() {} + } + + /** Immutable repair inputs for a single time partition. */ + public interface PartitionRepairContext { + + long getPartitionId(); + + boolean isRootHashMatched(); + + List getLeaderMerkleFiles(); + + List getMismatchedLeaderMerkleFiles(); + + RowRefIndex getRowRefIndex(); + + List decodeDiffs(); + + boolean isDiffDecodeSuccessful(); + + long estimateDiffCount(); + + List getFallbackTsFiles(); + + long getTsFileSize(String tsFilePath); + + int getTotalPointCount(String tsFilePath); + + default List getLeaderDeletions() { + return Collections.emptyList(); + } + + default List getFollowerDeletions() { + return Collections.emptyList(); + } + + default boolean shouldForceDirectTsFileTransfer() { + return false; + } + + default boolean verify(Map repairPlans, boolean rootHashMatched) { + return true; + } + } +} diff --git a/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/procedure/state/consistency/RepairState.java b/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/procedure/state/consistency/RepairState.java new file mode 100644 index 0000000000000..32062e55a6099 --- /dev/null +++ b/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/procedure/state/consistency/RepairState.java @@ -0,0 +1,44 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.iotdb.confignode.procedure.state.consistency; + +/** State machine states for the RepairRegionProcedure. */ +public enum RepairState { + INIT, + CHECK_SYNC_LAG, + COMPUTE_WATERMARK, + BUILD_MERKLE_VIEW, + COMPARE_ROOT_HASH, + DRILL_DOWN, + SMALL_TSFILE_SHORT_CIRCUIT, + NEGOTIATE_KEY_MAPPING, + ESTIMATE_DIFF, + EXCHANGE_IBF, + DECODE_DIFF, + ATTRIBUTE_DIFFS, + SELECT_REPAIR_STRATEGY, + EXECUTE_TSFILE_TRANSFER, + EXECUTE_POINT_STREAMING, + VERIFY_REPAIR, + COMMIT_PARTITION, + ADVANCE_WATERMARK, + ROLLBACK, + DONE +} diff --git a/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/procedure/store/ProcedureFactory.java b/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/procedure/store/ProcedureFactory.java index dd15558608718..474ca6f618171 100644 --- a/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/procedure/store/ProcedureFactory.java +++ b/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/procedure/store/ProcedureFactory.java @@ -21,6 +21,7 @@ import org.apache.iotdb.commons.exception.runtime.ThriftSerDeException; import org.apache.iotdb.confignode.procedure.Procedure; +import org.apache.iotdb.confignode.procedure.impl.consistency.RepairRegionProcedure; import org.apache.iotdb.confignode.procedure.impl.cq.CreateCQProcedure; import org.apache.iotdb.confignode.procedure.impl.node.AddConfigNodeProcedure; import org.apache.iotdb.confignode.procedure.impl.node.RemoveAINodeProcedure; @@ -395,6 +396,9 @@ public Procedure create(ByteBuffer buffer) throws IOException { case CONSUMER_GROUP_META_SYNC_PROCEDURE: procedure = new ConsumerGroupMetaSyncProcedure(); break; + case REPAIR_REGION_PROCEDURE: + procedure = new RepairRegionProcedure(); + break; case CREATE_MANY_DATABASES_PROCEDURE: procedure = new CreateManyDatabasesProcedure(); break; @@ -540,6 +544,8 @@ public static ProcedureType getProcedureType(final Procedure procedure) { return ProcedureType.ALTER_CONSUMER_GROUP_PROCEDURE; } else if (procedure instanceof ConsumerGroupMetaSyncProcedure) { return ProcedureType.CONSUMER_GROUP_META_SYNC_PROCEDURE; + } else if (procedure instanceof RepairRegionProcedure) { + return ProcedureType.REPAIR_REGION_PROCEDURE; } else if (procedure instanceof DeleteLogicalViewProcedure) { return ProcedureType.DELETE_LOGICAL_VIEW_PROCEDURE; } else if (procedure instanceof AlterLogicalViewProcedure) { diff --git a/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/procedure/store/ProcedureType.java b/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/procedure/store/ProcedureType.java index 820a90f7ebfb9..c693c110a676f 100644 --- a/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/procedure/store/ProcedureType.java +++ b/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/procedure/store/ProcedureType.java @@ -168,6 +168,9 @@ public enum ProcedureType { TOPIC_META_SYNC_PROCEDURE((short) 1508), CONSUMER_GROUP_META_SYNC_PROCEDURE((short) 1509), + /** Consistency */ + REPAIR_REGION_PROCEDURE((short) 1600), + /** Other */ @TestOnly NEVER_FINISH_PROCEDURE((short) 30000), diff --git a/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/service/thrift/ConfigNodeRPCServiceProcessor.java b/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/service/thrift/ConfigNodeRPCServiceProcessor.java index 5d6aa8da9f5df..962a2e6caa8fe 100644 --- a/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/service/thrift/ConfigNodeRPCServiceProcessor.java +++ b/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/service/thrift/ConfigNodeRPCServiceProcessor.java @@ -223,6 +223,7 @@ import org.apache.iotdb.confignode.rpc.thrift.TSystemConfigurationResp; import org.apache.iotdb.confignode.rpc.thrift.TTestOperation; import org.apache.iotdb.confignode.rpc.thrift.TThrottleQuotaResp; +import org.apache.iotdb.confignode.rpc.thrift.TTriggerRegionConsistencyRepairReq; import org.apache.iotdb.confignode.rpc.thrift.TUnsetSchemaTemplateReq; import org.apache.iotdb.confignode.rpc.thrift.TUnsubscribeReq; import org.apache.iotdb.confignode.service.ConfigNode; @@ -1353,6 +1354,12 @@ public TSStatus removeRegion(TRemoveRegionReq req) throws TException { return configManager.removeRegion(req); } + @Override + public TSStatus triggerRegionConsistencyRepair(TTriggerRegionConsistencyRepairReq req) + throws TException { + return configManager.triggerRegionConsistencyRepair(req); + } + @Override public TSStatus createCQ(TCreateCQReq req) { return configManager.createCQ(req); diff --git a/iotdb-core/confignode/src/test/java/org/apache/iotdb/confignode/procedure/impl/consistency/RepairRegionProcedureTest.java b/iotdb-core/confignode/src/test/java/org/apache/iotdb/confignode/procedure/impl/consistency/RepairRegionProcedureTest.java new file mode 100644 index 0000000000000..7237bcf6586a2 --- /dev/null +++ b/iotdb-core/confignode/src/test/java/org/apache/iotdb/confignode/procedure/impl/consistency/RepairRegionProcedureTest.java @@ -0,0 +1,828 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.iotdb.confignode.procedure.impl.consistency; + +import org.apache.iotdb.common.rpc.thrift.TConsensusGroupId; +import org.apache.iotdb.common.rpc.thrift.TConsensusGroupType; +import org.apache.iotdb.commons.consensus.iotv2.consistency.RepairProgressTable; +import org.apache.iotdb.commons.consensus.iotv2.consistency.ibf.CompositeKeyCodec; +import org.apache.iotdb.commons.consensus.iotv2.consistency.ibf.DataPointLocator; +import org.apache.iotdb.commons.consensus.iotv2.consistency.ibf.DiffEntry; +import org.apache.iotdb.commons.consensus.iotv2.consistency.ibf.RowRefIndex; +import org.apache.iotdb.commons.consensus.iotv2.consistency.merkle.MerkleEntry; +import org.apache.iotdb.commons.consensus.iotv2.consistency.merkle.MerkleFileContent; +import org.apache.iotdb.commons.consensus.iotv2.consistency.repair.RepairAction; +import org.apache.iotdb.commons.consensus.iotv2.consistency.repair.RepairConflictResolver; +import org.apache.iotdb.commons.consensus.iotv2.consistency.repair.ModEntrySummary; +import org.apache.iotdb.commons.consensus.iotv2.consistency.repair.RepairPlan; +import org.apache.iotdb.commons.consensus.iotv2.consistency.repair.RepairRecord; +import org.apache.iotdb.commons.consensus.iotv2.consistency.repair.RepairSession; +import org.apache.iotdb.commons.consensus.iotv2.consistency.repair.RepairStrategy; +import org.apache.iotdb.confignode.procedure.Procedure; +import org.apache.iotdb.confignode.procedure.store.ProcedureFactory; + +import org.apache.tsfile.utils.PublicBAOS; +import org.junit.Assert; +import org.junit.Test; + +import java.io.DataOutputStream; +import java.nio.ByteBuffer; +import java.util.ArrayList; +import java.util.Arrays; +import java.util.Collections; +import java.util.HashMap; +import java.util.LinkedHashMap; +import java.util.LinkedHashSet; +import java.util.List; +import java.util.Map; +import java.util.Objects; +import java.util.Set; + +public class RepairRegionProcedureTest { + + @Test + public void serDeTest() throws Exception { + TConsensusGroupId groupId = new TConsensusGroupId(TConsensusGroupType.DataRegion, 10); + RepairRegionProcedure procedure = + new RepairRegionProcedure( + groupId, + new TestExecutionContext(Collections.emptyMap(), new SimulatedReplicaState())); + try (PublicBAOS byteArrayOutputStream = new PublicBAOS(); + DataOutputStream outputStream = new DataOutputStream(byteArrayOutputStream)) { + procedure.serialize(outputStream); + ByteBuffer buffer = + ByteBuffer.wrap(byteArrayOutputStream.getBuf(), 0, byteArrayOutputStream.size()); + Assert.assertEquals(procedure, ProcedureFactory.getInstance().create(buffer)); + } finally { + RepairRegionProcedure.unregisterExecutionContext(procedure.getExecutionContextId()); + } + } + + @Test + public void executeRepairFlowTest() throws Exception { + TConsensusGroupId groupId = new TConsensusGroupId(TConsensusGroupType.DataRegion, 1); + RowRefIndex rowRefIndex = + new RowRefIndex.Builder() + .addDevice("root.sg.d1", Arrays.asList("s1", "s2")) + .setTimeBucketStart(0L) + .build(); + long compositeKey = CompositeKeyCodec.encode(0, 0, 100L, 0L, 1L); + DiffEntry diffEntry = new DiffEntry(compositeKey, 42L, DiffEntry.DiffType.LEADER_HAS); + DataPointLocator largeLocator = new DataPointLocator("root.sg.d1", "s1", 100L); + DataPointLocator smallLocator = new DataPointLocator("root.sg.d1", "s2", 200L); + + MerkleFileContent smallFile = + new MerkleFileContent( + 1L, + 1L, + Collections.singletonList(new MerkleEntry("root.sg.d1", "s2", 0L, 3_600_000L, 10, 1L)), + "small.tsfile"); + MerkleFileContent largeFile = + new MerkleFileContent( + 2L, + 2L, + Collections.singletonList( + new MerkleEntry("root.sg.d1", "s1", 0L, 3_600_000L, 1000, 2L)), + "large.tsfile"); + + Map tsFileSizes = new HashMap<>(); + tsFileSizes.put("small.tsfile", 8L * 1024 * 1024); + tsFileSizes.put("large.tsfile", 128L * 1024 * 1024); + + Map totalPointCounts = new HashMap<>(); + totalPointCounts.put("small.tsfile", 10); + totalPointCounts.put("large.tsfile", 1000); + + SimulatedReplicaState replicaState = new SimulatedReplicaState(); + replicaState.addLeaderPoint("small.tsfile", smallLocator, 1L, "small-value"); + replicaState.addLeaderPoint("large.tsfile", largeLocator, 10L, "value"); + replicaState.addFollowerTsFile("large.tsfile"); + + TestPartitionRepairContext partitionContext = + new TestPartitionRepairContext( + 0L, + false, + Arrays.asList(smallFile, largeFile), + Arrays.asList(smallFile, largeFile), + rowRefIndex, + Collections.singletonList(diffEntry), + true, + 1L, + Collections.emptyList(), + tsFileSizes, + totalPointCounts, + Collections.emptyList(), + Collections.emptyList(), + (repairPlans, rootHashMatched) -> { + RepairPlan smallPlan = repairPlans.get("small.tsfile"); + RepairPlan largePlan = repairPlans.get("large.tsfile"); + return !rootHashMatched + && smallPlan != null + && largePlan != null + && smallPlan.getStrategy() == RepairStrategy.DIRECT_TSFILE_TRANSFER + && largePlan.getStrategy() == RepairStrategy.POINT_STREAMING + && replicaState.followerHasTsFile("small.tsfile") + && replicaState.isConsistentWithLeader(); + }); + + TestExecutionContext executionContext = + new TestExecutionContext(Collections.singletonMap(0L, partitionContext), replicaState); + ExposedRepairRegionProcedure procedure = + new ExposedRepairRegionProcedure(groupId, executionContext); + + executeProcedureToCompletion(procedure, 32); + Assert.assertEquals( + Collections.singletonList("small.tsfile"), executionContext.transferredTsFiles); + Assert.assertEquals(1, executionContext.generatedRecords.size()); + Assert.assertEquals(1, executionContext.appliedInsertRecords.size()); + Assert.assertTrue(executionContext.appliedDeleteRecords.isEmpty()); + Assert.assertEquals( + Arrays.asList("append:INSERT", "commit", "delete"), executionContext.repairJournalEvents); + Assert.assertEquals(1, executionContext.committedPartitions.size()); + Assert.assertEquals(Long.valueOf(0L), executionContext.committedPartitions.get(0)); + Assert.assertEquals(1000L, executionContext.advancedWatermark); + Assert.assertTrue(executionContext.closed); + Assert.assertFalse(executionContext.rolledBack); + Assert.assertEquals(1000L, procedure.getGlobalRepairedWatermark()); + Assert.assertTrue(replicaState.isConsistentWithLeader()); + } + + @Test + public void executeRepairFlowDeletesFollowerOnlyDataWhenLeaderDeletionWinsTest() throws Exception { + TConsensusGroupId groupId = new TConsensusGroupId(TConsensusGroupType.DataRegion, 2); + RowRefIndex rowRefIndex = + new RowRefIndex.Builder() + .addDevice("root.sg.d1", Collections.singletonList("s1")) + .setTimeBucketStart(0L) + .build(); + long extraCompositeKey = CompositeKeyCodec.encode(0, 0, 100L, 0L, 1L); + DiffEntry extraFollowerPoint = + new DiffEntry(extraCompositeKey, 84L, DiffEntry.DiffType.FOLLOWER_HAS); + DataPointLocator stableLocator = new DataPointLocator("root.sg.d1", "s1", 200L); + DataPointLocator extraLocator = new DataPointLocator("root.sg.d1", "s1", 100L); + + MerkleFileContent largeFile = + new MerkleFileContent( + 2L, + 2L, + Collections.singletonList( + new MerkleEntry("root.sg.d1", "s1", 0L, 3_600_000L, 1000, 2L)), + "large.tsfile"); + + Map tsFileSizes = Collections.singletonMap("large.tsfile", 128L * 1024 * 1024); + Map totalPointCounts = Collections.singletonMap("large.tsfile", 1000); + + SimulatedReplicaState replicaState = new SimulatedReplicaState(); + replicaState.addLeaderPoint("large.tsfile", stableLocator, 20L, "stable-value"); + replicaState.addFollowerPoint("large.tsfile", stableLocator, 20L, "stable-value"); + replicaState.addFollowerPoint("large.tsfile", extraLocator, 5L, "stale-value"); + + ModEntrySummary leaderDeletion = new ModEntrySummary("root.sg.d1", "s1", 0L, 150L, 10L); + TestPartitionRepairContext partitionContext = + new TestPartitionRepairContext( + 0L, + false, + Collections.singletonList(largeFile), + Collections.singletonList(largeFile), + rowRefIndex, + Collections.singletonList(extraFollowerPoint), + true, + 1L, + Collections.emptyList(), + tsFileSizes, + totalPointCounts, + Collections.singletonList(leaderDeletion), + Collections.emptyList(), + (repairPlans, rootHashMatched) -> { + RepairPlan largePlan = repairPlans.get("large.tsfile"); + return !rootHashMatched + && largePlan != null + && largePlan.getStrategy() == RepairStrategy.POINT_STREAMING + && replicaState.isConsistentWithLeader(); + }); + + TestExecutionContext executionContext = + new TestExecutionContext(Collections.singletonMap(0L, partitionContext), replicaState); + ExposedRepairRegionProcedure procedure = + new ExposedRepairRegionProcedure(groupId, executionContext); + + executeProcedureToCompletion(procedure, 32); + Assert.assertTrue(executionContext.transferredTsFiles.isEmpty()); + Assert.assertEquals(1, executionContext.generatedRecords.size()); + Assert.assertTrue(executionContext.appliedInsertRecords.isEmpty()); + Assert.assertEquals(1, executionContext.appliedDeleteRecords.size()); + Assert.assertEquals( + Arrays.asList("append:DELETE", "commit", "delete"), executionContext.repairJournalEvents); + Assert.assertTrue(replicaState.isConsistentWithLeader()); + Assert.assertFalse(replicaState.hasFollowerPoint(extraLocator)); + Assert.assertFalse(executionContext.rolledBack); + } + + @Test + public void executeRepairFlowDeletesLeaderDataWhenFollowerTombstoneWinsTest() throws Exception { + TConsensusGroupId groupId = new TConsensusGroupId(TConsensusGroupType.DataRegion, 3); + RowRefIndex rowRefIndex = + new RowRefIndex.Builder() + .addDevice("root.sg.d1", Collections.singletonList("s1")) + .setTimeBucketStart(0L) + .build(); + long staleCompositeKey = CompositeKeyCodec.encode(0, 0, 100L, 0L, 1L); + DiffEntry staleLeaderPoint = + new DiffEntry(staleCompositeKey, 126L, DiffEntry.DiffType.LEADER_HAS); + DataPointLocator stableLocator = new DataPointLocator("root.sg.d1", "s1", 200L); + DataPointLocator staleLocator = new DataPointLocator("root.sg.d1", "s1", 100L); + + MerkleFileContent largeFile = + new MerkleFileContent( + 2L, + 2L, + Collections.singletonList( + new MerkleEntry("root.sg.d1", "s1", 0L, 3_600_000L, 1000, 2L)), + "large.tsfile"); + + Map tsFileSizes = Collections.singletonMap("large.tsfile", 128L * 1024 * 1024); + Map totalPointCounts = Collections.singletonMap("large.tsfile", 1000); + + SimulatedReplicaState replicaState = new SimulatedReplicaState(); + replicaState.addLeaderPoint("large.tsfile", stableLocator, 20L, "stable-value"); + replicaState.addFollowerPoint("large.tsfile", stableLocator, 20L, "stable-value"); + replicaState.addLeaderPoint("large.tsfile", staleLocator, 5L, "stale-leader-value"); + + ModEntrySummary followerDeletion = new ModEntrySummary("root.sg.d1", "s1", 0L, 150L, 10L); + TestPartitionRepairContext partitionContext = + new TestPartitionRepairContext( + 0L, + false, + Collections.singletonList(largeFile), + Collections.singletonList(largeFile), + rowRefIndex, + Collections.singletonList(staleLeaderPoint), + true, + 1L, + Collections.emptyList(), + tsFileSizes, + totalPointCounts, + Collections.emptyList(), + Collections.singletonList(followerDeletion), + (repairPlans, rootHashMatched) -> { + RepairPlan largePlan = repairPlans.get("large.tsfile"); + return !rootHashMatched + && largePlan != null + && largePlan.getStrategy() == RepairStrategy.POINT_STREAMING + && replicaState.isConsistentWithLeader(); + }); + + TestExecutionContext executionContext = + new TestExecutionContext(Collections.singletonMap(0L, partitionContext), replicaState); + ExposedRepairRegionProcedure procedure = + new ExposedRepairRegionProcedure(groupId, executionContext); + + executeProcedureToCompletion(procedure, 32); + Assert.assertTrue(executionContext.transferredTsFiles.isEmpty()); + Assert.assertEquals(1, executionContext.generatedRecords.size()); + Assert.assertTrue(executionContext.appliedInsertRecords.isEmpty()); + Assert.assertEquals(1, executionContext.appliedDeleteRecords.size()); + Assert.assertEquals( + Arrays.asList("append:DELETE", "commit", "delete"), executionContext.repairJournalEvents); + Assert.assertTrue(replicaState.isConsistentWithLeader()); + Assert.assertFalse(replicaState.hasLeaderPoint(staleLocator)); + Assert.assertFalse(executionContext.rolledBack); + } + + @Test + public void executeRepairFlowStreamsFollowerDataBackToLeaderTest() throws Exception { + TConsensusGroupId groupId = new TConsensusGroupId(TConsensusGroupType.DataRegion, 4); + RowRefIndex rowRefIndex = + new RowRefIndex.Builder() + .addDevice("root.sg.d1", Collections.singletonList("s1")) + .setTimeBucketStart(0L) + .build(); + long missingCompositeKey = CompositeKeyCodec.encode(0, 0, 100L, 0L, 1L); + DiffEntry followerOwnedPoint = + new DiffEntry(missingCompositeKey, 168L, DiffEntry.DiffType.FOLLOWER_HAS); + DataPointLocator stableLocator = new DataPointLocator("root.sg.d1", "s1", 200L); + DataPointLocator followerLocator = new DataPointLocator("root.sg.d1", "s1", 100L); + + MerkleFileContent largeFile = + new MerkleFileContent( + 2L, + 2L, + Collections.singletonList( + new MerkleEntry("root.sg.d1", "s1", 0L, 3_600_000L, 1000, 2L)), + "large.tsfile"); + + Map tsFileSizes = Collections.singletonMap("large.tsfile", 128L * 1024 * 1024); + Map totalPointCounts = Collections.singletonMap("large.tsfile", 1000); + + SimulatedReplicaState replicaState = new SimulatedReplicaState(); + replicaState.addLeaderPoint("large.tsfile", stableLocator, 20L, "stable-value"); + replicaState.addFollowerPoint("large.tsfile", stableLocator, 20L, "stable-value"); + replicaState.addFollowerPoint("large.tsfile", followerLocator, 25L, "follower-owned-value"); + + TestPartitionRepairContext partitionContext = + new TestPartitionRepairContext( + 0L, + false, + Collections.singletonList(largeFile), + Collections.singletonList(largeFile), + rowRefIndex, + Collections.singletonList(followerOwnedPoint), + true, + 1L, + Collections.emptyList(), + tsFileSizes, + totalPointCounts, + Collections.emptyList(), + Collections.emptyList(), + (repairPlans, rootHashMatched) -> { + RepairPlan largePlan = repairPlans.get("large.tsfile"); + return !rootHashMatched + && largePlan != null + && largePlan.getStrategy() == RepairStrategy.POINT_STREAMING + && replicaState.isConsistentWithLeader(); + }); + + TestExecutionContext executionContext = + new TestExecutionContext(Collections.singletonMap(0L, partitionContext), replicaState); + ExposedRepairRegionProcedure procedure = + new ExposedRepairRegionProcedure(groupId, executionContext); + + executeProcedureToCompletion(procedure, 32); + Assert.assertTrue(executionContext.transferredTsFiles.isEmpty()); + Assert.assertEquals(1, executionContext.generatedRecords.size()); + Assert.assertEquals(1, executionContext.appliedInsertRecords.size()); + Assert.assertTrue(executionContext.appliedDeleteRecords.isEmpty()); + Assert.assertEquals( + Arrays.asList("append:INSERT", "commit", "delete"), executionContext.repairJournalEvents); + Assert.assertTrue(replicaState.isConsistentWithLeader()); + Assert.assertTrue(replicaState.hasLeaderPoint(followerLocator)); + Assert.assertFalse(executionContext.rolledBack); + } + + private static class TestExecutionContext + implements RepairRegionProcedure.RepairExecutionContext { + + private final Map partitionContexts; + private final SimulatedReplicaState replicaState; + private final List transferredTsFiles = new ArrayList<>(); + private final List generatedRecords = new ArrayList<>(); + private final List appliedInsertRecords = new ArrayList<>(); + private final List appliedDeleteRecords = new ArrayList<>(); + private final List repairJournalEvents = new ArrayList<>(); + private final List committedPartitions = new ArrayList<>(); + private long advancedWatermark = -1L; + private boolean closed; + private boolean rolledBack; + + private TestExecutionContext( + Map partitionContexts, + SimulatedReplicaState replicaState) { + this.partitionContexts = partitionContexts; + this.replicaState = replicaState; + } + + @Override + public boolean isReplicationComplete() { + return true; + } + + @Override + public long computeSafeWatermark() { + return 1000L; + } + + @Override + public List collectPendingPartitions( + long globalRepairedWatermark, long safeWatermark, RepairProgressTable repairProgressTable) { + return new ArrayList<>(partitionContexts.keySet()); + } + + @Override + public RepairRegionProcedure.PartitionRepairContext getPartitionContext(long partitionId) { + return partitionContexts.get(partitionId); + } + + @Override + public RepairRecord buildRepairRecord( + RepairRegionProcedure.PartitionRepairContext partitionContext, + DiffEntry diffEntry, + RowRefIndex rowRefIndex, + RepairConflictResolver conflictResolver) { + DataPointLocator locator = rowRefIndex.resolve(diffEntry.getCompositeKey()); + SimulatedPoint leaderPoint = replicaState.getLeaderPoint(locator); + SimulatedPoint followerPoint = replicaState.getFollowerPoint(locator); + RepairRecord record = null; + if (diffEntry.getType() == DiffEntry.DiffType.LEADER_HAS) { + if (leaderPoint == null) { + return null; + } + RepairAction action = + conflictResolver.resolveLeaderHas(locator, leaderPoint.getProgressIndex()); + if (action == RepairAction.SEND_TO_FOLLOWER) { + record = + RepairRecord.insertToFollower( + locator, + leaderPoint.getProgressIndex(), + leaderPoint.getValue(), + locator.getTimestamp()); + } else if (action == RepairAction.DELETE_ON_LEADER) { + record = + RepairRecord.deleteOnLeader( + locator, leaderPoint.getProgressIndex(), locator.getTimestamp()); + } + } else if (diffEntry.getType() == DiffEntry.DiffType.FOLLOWER_HAS) { + if (followerPoint == null) { + return null; + } + RepairAction action = + conflictResolver.resolveFollowerHas(locator, followerPoint.getProgressIndex()); + if (action == RepairAction.DELETE_ON_FOLLOWER) { + record = + RepairRecord.deleteOnFollower( + locator, followerPoint.getProgressIndex(), locator.getTimestamp()); + } else if (action == RepairAction.SEND_TO_LEADER) { + record = + RepairRecord.insertToLeader( + locator, + followerPoint.getProgressIndex(), + followerPoint.getValue(), + locator.getTimestamp()); + } + } + if (record != null) { + generatedRecords.add(record); + } + return record; + } + + @Override + public void transferTsFile(String tsFilePath) { + transferredTsFiles.add(tsFilePath); + replicaState.transferTsFile(tsFilePath); + } + + @Override + public RepairSession createRepairSession(long partitionId) { + return new RepairSession( + partitionId, + (sessionId, ignoredPartitionId, inserts, deletes) -> { + appliedInsertRecords.addAll(inserts); + appliedDeleteRecords.addAll(deletes); + replicaState.applyInserts(inserts); + replicaState.applyDeletes(deletes); + }, + new RepairSession.RepairSessionJournal() { + @Override + public void append(String sessionId, RepairRecord record) { + repairJournalEvents.add("append:" + record.getType()); + } + + @Override + public void markCommitted(String sessionId) { + repairJournalEvents.add("commit"); + } + + @Override + public void delete(String sessionId) { + repairJournalEvents.add("delete"); + } + }); + } + + @Override + public void onPartitionCommitted( + long partitionId, long repairedTo, RepairProgressTable repairProgressTable) { + committedPartitions.add(partitionId); + } + + @Override + public void onWatermarkAdvanced(long globalWatermark, RepairProgressTable repairProgressTable) { + this.advancedWatermark = globalWatermark; + } + + @Override + public void rollbackPartition( + long partitionId, RepairSession repairSession, RepairProgressTable repairProgressTable) { + rolledBack = true; + } + + @Override + public void close() { + closed = true; + } + } + + private static class ExposedRepairRegionProcedure extends RepairRegionProcedure { + + private ExposedRepairRegionProcedure( + TConsensusGroupId consensusGroupId, RepairExecutionContext executionContext) { + super(consensusGroupId, executionContext); + } + + private Procedure[] executeOnce() throws InterruptedException { + return doExecute(null); + } + } + + private static class TestPartitionRepairContext + implements RepairRegionProcedure.PartitionRepairContext { + + private final long partitionId; + private final boolean rootHashMatched; + private final List leaderMerkleFiles; + private final List mismatchedLeaderMerkleFiles; + private final RowRefIndex rowRefIndex; + private final List decodedDiffs; + private final boolean diffDecodeSuccessful; + private final long estimatedDiffCount; + private final List fallbackTsFiles; + private final Map tsFileSizes; + private final Map totalPointCounts; + private final List leaderDeletions; + private final List followerDeletions; + private final RepairVerifier verifier; + + private TestPartitionRepairContext( + long partitionId, + boolean rootHashMatched, + List leaderMerkleFiles, + List mismatchedLeaderMerkleFiles, + RowRefIndex rowRefIndex, + List decodedDiffs, + boolean diffDecodeSuccessful, + long estimatedDiffCount, + List fallbackTsFiles, + Map tsFileSizes, + Map totalPointCounts, + List leaderDeletions, + List followerDeletions, + RepairVerifier verifier) { + this.partitionId = partitionId; + this.rootHashMatched = rootHashMatched; + this.leaderMerkleFiles = leaderMerkleFiles; + this.mismatchedLeaderMerkleFiles = mismatchedLeaderMerkleFiles; + this.rowRefIndex = rowRefIndex; + this.decodedDiffs = decodedDiffs; + this.diffDecodeSuccessful = diffDecodeSuccessful; + this.estimatedDiffCount = estimatedDiffCount; + this.fallbackTsFiles = fallbackTsFiles; + this.tsFileSizes = tsFileSizes; + this.totalPointCounts = totalPointCounts; + this.leaderDeletions = leaderDeletions; + this.followerDeletions = followerDeletions; + this.verifier = verifier; + } + + @Override + public long getPartitionId() { + return partitionId; + } + + @Override + public boolean isRootHashMatched() { + return rootHashMatched; + } + + @Override + public List getLeaderMerkleFiles() { + return leaderMerkleFiles; + } + + @Override + public List getMismatchedLeaderMerkleFiles() { + return mismatchedLeaderMerkleFiles; + } + + @Override + public RowRefIndex getRowRefIndex() { + return rowRefIndex; + } + + @Override + public List decodeDiffs() { + return decodedDiffs; + } + + @Override + public boolean isDiffDecodeSuccessful() { + return diffDecodeSuccessful; + } + + @Override + public long estimateDiffCount() { + return estimatedDiffCount; + } + + @Override + public List getFallbackTsFiles() { + return fallbackTsFiles; + } + + @Override + public long getTsFileSize(String tsFilePath) { + return tsFileSizes.getOrDefault(tsFilePath, 0L); + } + + @Override + public int getTotalPointCount(String tsFilePath) { + return totalPointCounts.getOrDefault(tsFilePath, 0); + } + + @Override + public List getLeaderDeletions() { + return leaderDeletions; + } + + @Override + public List getFollowerDeletions() { + return followerDeletions; + } + + @Override + public boolean verify(Map repairPlans, boolean rootHashMatched) { + return verifier.verify(repairPlans, rootHashMatched); + } + } + + private static void executeProcedureToCompletion( + ExposedRepairRegionProcedure procedure, int maxSteps) throws Exception { + int steps = 0; + Procedure[] next; + do { + next = procedure.executeOnce(); + steps++; + } while (next != null && steps < maxSteps); + Assert.assertTrue("procedure should finish within " + maxSteps + " steps", steps < maxSteps); + } + + @FunctionalInterface + private interface RepairVerifier { + boolean verify(Map repairPlans, boolean rootHashMatched); + } + + private static class SimulatedReplicaState { + + private final Map> leaderPointsByTsFile = new LinkedHashMap<>(); + private final Map leaderPoints = new LinkedHashMap<>(); + private final Map followerPoints = new LinkedHashMap<>(); + private final Set leaderTsFiles = new LinkedHashSet<>(); + private final Set followerTsFiles = new LinkedHashSet<>(); + + private void addLeaderPoint( + String tsFilePath, DataPointLocator locator, long progressIndex, Object value) { + SimulatedPoint point = new SimulatedPoint(tsFilePath, locator, progressIndex, value); + leaderTsFiles.add(tsFilePath); + leaderPoints.put(locator, point); + leaderPointsByTsFile.computeIfAbsent(tsFilePath, ignored -> new ArrayList<>()).add(point); + } + + private void addFollowerTsFile(String tsFilePath) { + followerTsFiles.add(tsFilePath); + } + + private void addFollowerPoint( + String tsFilePath, DataPointLocator locator, long progressIndex, Object value) { + followerTsFiles.add(tsFilePath); + followerPoints.put(locator, new SimulatedPoint(tsFilePath, locator, progressIndex, value)); + } + + private SimulatedPoint getLeaderPoint(DataPointLocator locator) { + return leaderPoints.get(locator); + } + + private SimulatedPoint getFollowerPoint(DataPointLocator locator) { + return followerPoints.get(locator); + } + + private boolean hasFollowerPoint(DataPointLocator locator) { + return followerPoints.containsKey(locator); + } + + private boolean hasLeaderPoint(DataPointLocator locator) { + return leaderPoints.containsKey(locator); + } + + private boolean followerHasTsFile(String tsFilePath) { + return followerTsFiles.contains(tsFilePath); + } + + private void transferTsFile(String tsFilePath) { + followerTsFiles.add(tsFilePath); + for (SimulatedPoint leaderPoint : + leaderPointsByTsFile.getOrDefault(tsFilePath, Collections.emptyList())) { + followerPoints.put(leaderPoint.getLocator(), leaderPoint); + } + } + + private void applyInserts(List inserts) { + for (RepairRecord record : inserts) { + if (record.getTargetReplica() == RepairRecord.TargetReplica.LEADER) { + SimulatedPoint followerPoint = followerPoints.get(record.getLocator()); + String tsFilePath = followerPoint == null ? "stream.tsfile" : followerPoint.getTsFilePath(); + leaderTsFiles.add(tsFilePath); + leaderPoints.put( + record.getLocator(), + new SimulatedPoint( + tsFilePath, + record.getLocator(), + record.getProgressIndex(), + record.getValue())); + } else { + SimulatedPoint leaderPoint = leaderPoints.get(record.getLocator()); + String tsFilePath = leaderPoint == null ? "stream.tsfile" : leaderPoint.getTsFilePath(); + followerTsFiles.add(tsFilePath); + followerPoints.put( + record.getLocator(), + new SimulatedPoint( + tsFilePath, + record.getLocator(), + record.getProgressIndex(), + record.getValue())); + } + } + } + + private void applyDeletes(List deletes) { + for (RepairRecord record : deletes) { + if (record.getTargetReplica() == RepairRecord.TargetReplica.LEADER) { + leaderPoints.remove(record.getLocator()); + } else { + followerPoints.remove(record.getLocator()); + } + } + } + + private boolean isConsistentWithLeader() { + return leaderTsFiles.equals(followerTsFiles) && Objects.equals(leaderPoints, followerPoints); + } + } + + private static class SimulatedPoint { + + private final String tsFilePath; + private final DataPointLocator locator; + private final long progressIndex; + private final Object value; + + private SimulatedPoint( + String tsFilePath, DataPointLocator locator, long progressIndex, Object value) { + this.tsFilePath = tsFilePath; + this.locator = locator; + this.progressIndex = progressIndex; + this.value = value; + } + + private String getTsFilePath() { + return tsFilePath; + } + + private DataPointLocator getLocator() { + return locator; + } + + private long getProgressIndex() { + return progressIndex; + } + + private Object getValue() { + return value; + } + + @Override + public boolean equals(Object o) { + if (this == o) { + return true; + } + if (!(o instanceof SimulatedPoint)) { + return false; + } + SimulatedPoint that = (SimulatedPoint) o; + return progressIndex == that.progressIndex + && Objects.equals(tsFilePath, that.tsFilePath) + && Objects.equals(locator, that.locator) + && Objects.equals(value, that.value); + } + + @Override + public int hashCode() { + return Objects.hash(tsFilePath, locator, progressIndex, value); + } + } +} diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/protocol/client/ConfigNodeClient.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/protocol/client/ConfigNodeClient.java index e2c04caedfb20..6ac2da44c7ef3 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/protocol/client/ConfigNodeClient.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/protocol/client/ConfigNodeClient.java @@ -181,6 +181,7 @@ import org.apache.iotdb.confignode.rpc.thrift.TSystemConfigurationResp; import org.apache.iotdb.confignode.rpc.thrift.TTestOperation; import org.apache.iotdb.confignode.rpc.thrift.TThrottleQuotaResp; +import org.apache.iotdb.confignode.rpc.thrift.TTriggerRegionConsistencyRepairReq; import org.apache.iotdb.confignode.rpc.thrift.TUnsetSchemaTemplateReq; import org.apache.iotdb.confignode.rpc.thrift.TUnsubscribeReq; import org.apache.iotdb.db.conf.IoTDBConfig; @@ -827,6 +828,12 @@ public TSStatus stopRepairData() throws TException { () -> client.stopRepairData(), status -> !updateConfigNodeLeader(status)); } + @Override + public TSStatus triggerRegionConsistencyRepair(TTriggerRegionConsistencyRepairReq req) + throws TException { + throw new UnsupportedOperationException(UNSUPPORTED_INVOCATION); + } + @Override public TSStatus submitLoadConfigurationTask() throws TException { return executeRemoteCallWithRetry( diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/protocol/thrift/impl/DataNodeInternalRPCServiceImpl.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/protocol/thrift/impl/DataNodeInternalRPCServiceImpl.java index 42929be741819..2e87d24144d2f 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/protocol/thrift/impl/DataNodeInternalRPCServiceImpl.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/protocol/thrift/impl/DataNodeInternalRPCServiceImpl.java @@ -193,6 +193,7 @@ import org.apache.iotdb.db.service.externalservice.ExternalServiceManagementService; import org.apache.iotdb.db.service.metrics.FileMetrics; import org.apache.iotdb.db.storageengine.StorageEngine; +import org.apache.iotdb.db.storageengine.dataregion.consistency.DataRegionConsistencyRepairService; import org.apache.iotdb.db.storageengine.dataregion.compaction.repair.RepairTaskStatus; import org.apache.iotdb.db.storageengine.dataregion.compaction.schedule.CompactionScheduleTaskManager; import org.apache.iotdb.db.storageengine.dataregion.compaction.schedule.CompactionTaskManager; @@ -238,6 +239,8 @@ import org.apache.iotdb.mpp.rpc.thrift.TCreatePipePluginInstanceReq; import org.apache.iotdb.mpp.rpc.thrift.TCreateSchemaRegionReq; import org.apache.iotdb.mpp.rpc.thrift.TCreateTriggerInstanceReq; +import org.apache.iotdb.mpp.rpc.thrift.TDataRegionConsistencySnapshotReq; +import org.apache.iotdb.mpp.rpc.thrift.TDataRegionConsistencySnapshotResp; import org.apache.iotdb.mpp.rpc.thrift.TDataNodeHeartbeatReq; import org.apache.iotdb.mpp.rpc.thrift.TDataNodeHeartbeatResp; import org.apache.iotdb.mpp.rpc.thrift.TDeactivateTemplateReq; @@ -286,6 +289,7 @@ import org.apache.iotdb.mpp.rpc.thrift.TPushTopicMetaReq; import org.apache.iotdb.mpp.rpc.thrift.TPushTopicMetaResp; import org.apache.iotdb.mpp.rpc.thrift.TPushTopicMetaRespExceptionMessage; +import org.apache.iotdb.mpp.rpc.thrift.TRepairTransferTsFileReq; import org.apache.iotdb.mpp.rpc.thrift.TRegionLeaderChangeReq; import org.apache.iotdb.mpp.rpc.thrift.TRegionLeaderChangeResp; import org.apache.iotdb.mpp.rpc.thrift.TRegionMigrateResult; @@ -384,6 +388,8 @@ public class DataNodeInternalRPCServiceImpl implements IDataNodeRPCService.Iface private final SchemaEngine schemaEngine = SchemaEngine.getInstance(); private final StorageEngine storageEngine = StorageEngine.getInstance(); + private final DataRegionConsistencyRepairService consistencyRepairService = + new DataRegionConsistencyRepairService(); private final TableDeviceSchemaCache tableDeviceSchemaCache = TableDeviceSchemaCache.getInstance(); @@ -590,6 +596,17 @@ public TLoadResp sendLoadCommand(TLoadCommandReq req) { timePartitionProgressIndexMap)); } + @Override + public TDataRegionConsistencySnapshotResp getDataRegionConsistencySnapshot( + TDataRegionConsistencySnapshotReq req) { + return consistencyRepairService.getSnapshot(req); + } + + @Override + public TSStatus repairTransferTsFile(TRepairTransferTsFileReq req) { + return consistencyRepairService.repairTransferTsFile(req); + } + @Override public TSStatus updateAttribute(final TAttributeUpdateReq req) { TableDeviceSchemaFetcher.getInstance().getAttributeGuard().handleAttributeUpdate(req); diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/consistency/DataRegionConsistencyRepairService.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/consistency/DataRegionConsistencyRepairService.java new file mode 100644 index 0000000000000..2a9700c3cbd18 --- /dev/null +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/consistency/DataRegionConsistencyRepairService.java @@ -0,0 +1,352 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.iotdb.db.storageengine.dataregion.consistency; + +import org.apache.iotdb.common.rpc.thrift.TConsensusGroupId; +import org.apache.iotdb.common.rpc.thrift.TDataNodeLocation; +import org.apache.iotdb.common.rpc.thrift.TEndPoint; +import org.apache.iotdb.common.rpc.thrift.TSStatus; +import org.apache.iotdb.common.rpc.thrift.TTimePartitionSlot; +import org.apache.iotdb.commons.client.ClientPoolFactory; +import org.apache.iotdb.commons.client.IClientManager; +import org.apache.iotdb.commons.client.sync.SyncDataNodeInternalServiceClient; +import org.apache.iotdb.commons.consensus.ConsensusGroupId; +import org.apache.iotdb.commons.consensus.DataRegionId; +import org.apache.iotdb.commons.consensus.index.ProgressIndex; +import org.apache.iotdb.db.conf.IoTDBDescriptor; +import org.apache.iotdb.db.queryengine.plan.planner.plan.node.PlanNodeId; +import org.apache.iotdb.db.queryengine.plan.planner.plan.node.load.LoadTsFilePieceNode; +import org.apache.iotdb.db.queryengine.plan.scheduler.load.LoadTsFileScheduler; +import org.apache.iotdb.db.storageengine.StorageEngine; +import org.apache.iotdb.db.storageengine.dataregion.DataRegion; +import org.apache.iotdb.db.storageengine.dataregion.tsfile.TsFileResource; +import org.apache.iotdb.db.storageengine.load.splitter.TsFileSplitter; +import org.apache.iotdb.mpp.rpc.thrift.TConsistencyMerkleFile; +import org.apache.iotdb.mpp.rpc.thrift.TDataRegionConsistencySnapshotReq; +import org.apache.iotdb.mpp.rpc.thrift.TDataRegionConsistencySnapshotResp; +import org.apache.iotdb.mpp.rpc.thrift.TLoadCommandReq; +import org.apache.iotdb.mpp.rpc.thrift.TLoadResp; +import org.apache.iotdb.mpp.rpc.thrift.TRepairTransferTsFileReq; +import org.apache.iotdb.mpp.rpc.thrift.TTimePartitionConsistencyView; +import org.apache.iotdb.mpp.rpc.thrift.TTsFilePieceReq; +import org.apache.iotdb.rpc.RpcUtils; +import org.apache.iotdb.rpc.TSStatusCode; + +import org.apache.tsfile.utils.PublicBAOS; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.io.DataOutputStream; +import java.io.File; +import java.io.IOException; +import java.nio.ByteBuffer; +import java.util.ArrayList; +import java.util.Comparator; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import java.util.UUID; + +/** DataNode-side snapshot and direct TsFile repair primitives for replica consistency repair. */ +public class DataRegionConsistencyRepairService { + + private static final Logger LOGGER = + LoggerFactory.getLogger(DataRegionConsistencyRepairService.class); + + private static final long MAX_PIECE_NODE_SIZE = + IoTDBDescriptor.getInstance().getConfig().getThriftMaxFrameSize() >> 2; + + private final StorageEngine storageEngine = StorageEngine.getInstance(); + + private final IClientManager + clientManager = + new IClientManager.Factory() + .createClientManager( + new ClientPoolFactory.SyncDataNodeInternalServiceClientPoolFactory()); + + public TDataRegionConsistencySnapshotResp getSnapshot(TDataRegionConsistencySnapshotReq req) { + DataRegion dataRegion = getDataRegion(req.getConsensusGroupId()); + if (dataRegion == null) { + return new TDataRegionConsistencySnapshotResp( + RpcUtils.getStatus( + TSStatusCode.DATAREGION_PROCESS_ERROR, + "DataRegion " + req.getConsensusGroupId() + " is not found on this DataNode")); + } + + List partitionViews = new ArrayList<>(); + List timePartitions = new ArrayList<>(dataRegion.getTimePartitions()); + timePartitions.sort(Long::compareTo); + + try { + for (Long timePartition : timePartitions) { + List merkleFiles = collectMerkleFiles(dataRegion, timePartition); + if (!merkleFiles.isEmpty()) { + partitionViews.add(new TTimePartitionConsistencyView(timePartition, merkleFiles)); + } + } + return new TDataRegionConsistencySnapshotResp(RpcUtils.SUCCESS_STATUS) + .setTimePartitionViews(partitionViews); + } catch (Exception e) { + LOGGER.warn( + "Failed to build consistency snapshot for region {}", + req.getConsensusGroupId(), + e); + return new TDataRegionConsistencySnapshotResp( + RpcUtils.getStatus(TSStatusCode.EXECUTE_STATEMENT_ERROR, e.getMessage())); + } + } + + public TSStatus repairTransferTsFile(TRepairTransferTsFileReq req) { + DataRegion dataRegion = getDataRegion(req.getConsensusGroupId()); + if (dataRegion == null) { + return RpcUtils.getStatus( + TSStatusCode.DATAREGION_PROCESS_ERROR, + "DataRegion " + req.getConsensusGroupId() + " is not found on this DataNode"); + } + if (req.getTargetDataNodes() == null || req.getTargetDataNodes().isEmpty()) { + return RpcUtils.getStatus( + TSStatusCode.ILLEGAL_PARAMETER, "Repair transfer requires at least one target DataNode"); + } + + TsFileResource tsFileResource = findTsFileResource(dataRegion, req.getSourceTsFilePath()); + if (tsFileResource == null) { + return RpcUtils.getStatus( + TSStatusCode.LOAD_FILE_ERROR, + "Cannot find sealed TsFile " + req.getSourceTsFilePath() + " on leader"); + } + if (!tsFileResource.isClosed() + || tsFileResource.isDeleted() + || !tsFileResource.getTsFile().exists()) { + return RpcUtils.getStatus( + TSStatusCode.LOAD_FILE_ERROR, + "TsFile " + req.getSourceTsFilePath() + " is not available for repair transfer"); + } + if (tsFileResource.isSpanMultiTimePartitions()) { + return RpcUtils.getStatus( + TSStatusCode.UNSUPPORTED_OPERATION, + "Replica consistency repair does not support multi-time-partition TsFiles yet"); + } + + for (TDataNodeLocation targetDataNode : req.getTargetDataNodes()) { + TSStatus status = transferOneTarget(req.getConsensusGroupId(), tsFileResource, targetDataNode); + if (status.getCode() != TSStatusCode.SUCCESS_STATUS.getStatusCode()) { + return status; + } + } + return RpcUtils.SUCCESS_STATUS; + } + + private List collectMerkleFiles(DataRegion dataRegion, long timePartition) + throws IOException { + List merkleFiles = new ArrayList<>(); + for (boolean sequence : new boolean[] {true, false}) { + for (TsFileResource tsFileResource : + dataRegion.getTsFileManager().getTsFileListSnapshot(timePartition, sequence)) { + if (!tsFileResource.isClosed() + || tsFileResource.isDeleted() + || !tsFileResource.getTsFile().exists()) { + continue; + } + + String tsFilePath = tsFileResource.getTsFilePath(); + List entries = + MerkleHashComputer.computeEntries(tsFilePath); + merkleFiles.add( + new TConsistencyMerkleFile( + tsFilePath, + tsFileResource.getTsFileSize(), + org.apache.iotdb.commons.consensus.iotv2.consistency.merkle.MerkleFileWriter + .computeFileXorHash(entries), + org.apache.iotdb.commons.consensus.iotv2.consistency.merkle.MerkleFileWriter + .computeFileAddHash(entries))); + } + } + + merkleFiles.sort(Comparator.comparing(TConsistencyMerkleFile::getSourceTsFilePath)); + return merkleFiles; + } + + private TSStatus transferOneTarget( + TConsensusGroupId consensusGroupId, + TsFileResource tsFileResource, + TDataNodeLocation targetDataNode) { + String uuid = UUID.randomUUID().toString(); + TransferTracker tracker = new TransferTracker(); + + try { + sendAllPieces( + tsFileResource.getTsFile(), + uuid, + consensusGroupId, + targetDataNode.getInternalEndPoint(), + tracker); + TSStatus secondPhaseStatus = + sendLoadCommand( + targetDataNode.getInternalEndPoint(), + buildLoadCommandReq(uuid, tsFileResource, LoadTsFileScheduler.LoadCommand.EXECUTE)); + if (secondPhaseStatus.getCode() != TSStatusCode.SUCCESS_STATUS.getStatusCode()) { + return secondPhaseStatus; + } + return RpcUtils.SUCCESS_STATUS; + } catch (Exception e) { + LOGGER.warn( + "Failed to transfer TsFile {} to follower {}", + tsFileResource.getTsFilePath(), + targetDataNode.getDataNodeId(), + e); + if (tracker.hasSentPieces) { + TSStatus rollbackStatus = + sendLoadCommand( + targetDataNode.getInternalEndPoint(), + buildLoadCommandReq(uuid, tsFileResource, LoadTsFileScheduler.LoadCommand.ROLLBACK)); + if (rollbackStatus.getCode() != TSStatusCode.SUCCESS_STATUS.getStatusCode()) { + rollbackStatus.setMessage( + rollbackStatus.getMessage() + + ", original transfer failure: " + + e.getMessage()); + return rollbackStatus; + } + } + return RpcUtils.getStatus( + TSStatusCode.LOAD_FILE_ERROR, + "Failed to transfer TsFile " + + tsFileResource.getTsFilePath() + + " to follower " + + targetDataNode.getDataNodeId() + + ": " + + e.getMessage()); + } + } + + private void sendAllPieces( + File tsFile, + String uuid, + TConsensusGroupId consensusGroupId, + TEndPoint targetEndPoint, + TransferTracker tracker) + throws Exception { + final LoadTsFilePieceNode[] pieceHolder = { + new LoadTsFilePieceNode(new PlanNodeId("repair-tsfile-piece"), tsFile) + }; + + new TsFileSplitter( + tsFile, + tsFileData -> { + pieceHolder[0].addTsFileData(tsFileData); + if (pieceHolder[0].getDataSize() >= MAX_PIECE_NODE_SIZE) { + dispatchPieceNode(targetEndPoint, uuid, consensusGroupId, pieceHolder[0]); + tracker.hasSentPieces = true; + pieceHolder[0] = + new LoadTsFilePieceNode(new PlanNodeId("repair-tsfile-piece"), tsFile); + } + return true; + }) + .splitTsFileByDataPartition(); + + if (pieceHolder[0].getDataSize() > 0) { + dispatchPieceNode(targetEndPoint, uuid, consensusGroupId, pieceHolder[0]); + tracker.hasSentPieces = true; + } + } + + private void dispatchPieceNode( + TEndPoint targetEndPoint, + String uuid, + TConsensusGroupId consensusGroupId, + LoadTsFilePieceNode pieceNode) + throws Exception { + TTsFilePieceReq request = + new TTsFilePieceReq(pieceNode.serializeToByteBuffer(), uuid, consensusGroupId); + try (SyncDataNodeInternalServiceClient client = clientManager.borrowClient(targetEndPoint)) { + TLoadResp response = client.sendTsFilePieceNode(request); + if (!response.isAccepted()) { + throw new IllegalStateException( + response.isSetStatus() ? response.getStatus().getMessage() : response.getMessage()); + } + } + } + + private TSStatus sendLoadCommand( + TEndPoint targetEndPoint, TLoadCommandReq request) { + try (SyncDataNodeInternalServiceClient client = clientManager.borrowClient(targetEndPoint)) { + TLoadResp response = client.sendLoadCommand(request); + if (response.isAccepted()) { + return RpcUtils.SUCCESS_STATUS; + } + if (response.isSetStatus()) { + return response.getStatus(); + } + return RpcUtils.getStatus(TSStatusCode.LOAD_FILE_ERROR, response.getMessage()); + } catch (Exception e) { + return RpcUtils.getStatus(TSStatusCode.LOAD_FILE_ERROR, e.getMessage()); + } + } + + private TLoadCommandReq buildLoadCommandReq( + String uuid, TsFileResource tsFileResource, LoadTsFileScheduler.LoadCommand loadCommand) + throws IOException { + TLoadCommandReq request = new TLoadCommandReq(loadCommand.ordinal(), uuid); + Map timePartition2ProgressIndex = new HashMap<>(); + timePartition2ProgressIndex.put( + new TTimePartitionSlot(tsFileResource.getTimePartition()), + serializeProgressIndex(tsFileResource.getMaxProgressIndex())); + request.setTimePartition2ProgressIndex(timePartition2ProgressIndex); + request.setIsGeneratedByPipe(false); + return request; + } + + private ByteBuffer serializeProgressIndex(ProgressIndex progressIndex) throws IOException { + try (PublicBAOS byteArrayOutputStream = new PublicBAOS(); + DataOutputStream dataOutputStream = new DataOutputStream(byteArrayOutputStream)) { + progressIndex.serialize(dataOutputStream); + return ByteBuffer.wrap(byteArrayOutputStream.getBuf(), 0, byteArrayOutputStream.size()); + } + } + + private TsFileResource findTsFileResource(DataRegion dataRegion, String sourceTsFilePath) { + List timePartitions = new ArrayList<>(dataRegion.getTimePartitions()); + timePartitions.sort(Long::compareTo); + + for (Long timePartition : timePartitions) { + for (boolean sequence : new boolean[] {true, false}) { + for (TsFileResource tsFileResource : + dataRegion.getTsFileManager().getTsFileListSnapshot(timePartition, sequence)) { + if (sourceTsFilePath.equals(tsFileResource.getTsFilePath())) { + return tsFileResource; + } + } + } + } + return null; + } + + private DataRegion getDataRegion(TConsensusGroupId consensusGroupId) { + ConsensusGroupId groupId = + ConsensusGroupId.Factory.createFromTConsensusGroupId(consensusGroupId); + if (!(groupId instanceof DataRegionId)) { + return null; + } + return storageEngine.getDataRegion((DataRegionId) groupId); + } + + private static final class TransferTracker { + private boolean hasSentPieces; + } +} diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/consistency/MerkleCompactionHook.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/consistency/MerkleCompactionHook.java new file mode 100644 index 0000000000000..1a00532fd1ed9 --- /dev/null +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/consistency/MerkleCompactionHook.java @@ -0,0 +1,162 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.iotdb.db.storageengine.dataregion.consistency; + +import org.apache.iotdb.commons.consensus.iotv2.consistency.ConsistencyMerkleTree; +import org.apache.iotdb.commons.consensus.iotv2.consistency.DualDigest; +import org.apache.iotdb.commons.consensus.iotv2.consistency.merkle.MerkleEntry; +import org.apache.iotdb.commons.consensus.iotv2.consistency.merkle.MerkleFileCache; +import org.apache.iotdb.commons.consensus.iotv2.consistency.merkle.MerkleFileContent; +import org.apache.iotdb.commons.consensus.iotv2.consistency.merkle.MerkleFileWriter; +import org.apache.iotdb.db.storageengine.dataregion.tsfile.TsFileResource; + +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.io.File; +import java.io.IOException; +import java.util.ArrayList; +import java.util.List; + +/** + * Hook into TsFileManager.replace() for compaction-aware Merkle tree updates. When source TsFiles + * are merged into target TsFiles during compaction, this hook: + * + *
    + *
  1. XOR-out each source file's root hash from the partition digest + *
  2. Scan target TsFiles and generate .merkle files + *
  3. XOR-in each target file's root hash + *
  4. Delete source .merkle files + *
  5. Invalidate cache entries for source files + *
+ */ +public class MerkleCompactionHook { + + private static final Logger LOGGER = LoggerFactory.getLogger(MerkleCompactionHook.class); + + private final ConsistencyMerkleTree merkleTree; + private final MerkleFileCache merkleFileCache; + + public MerkleCompactionHook(ConsistencyMerkleTree merkleTree, MerkleFileCache merkleFileCache) { + this.merkleTree = merkleTree; + this.merkleFileCache = merkleFileCache; + } + + /** + * Called after compaction replaces source TsFiles with target TsFiles. + * + * @param seqSourceFiles source sequence TsFiles being removed + * @param unseqSourceFiles source unsequence TsFiles being removed + * @param targetFiles newly created target TsFiles + * @param timePartition the time partition being compacted + */ + public void onCompaction( + List seqSourceFiles, + List unseqSourceFiles, + List targetFiles, + long timePartition) { + try { + List sourceDigests = new ArrayList<>(); + boolean digestUpdatePossible = true; + + // Collect and XOR-out source file hashes + for (TsFileResource source : seqSourceFiles) { + DualDigest digest = getFileDigest(source); + if (digest == null) { + digestUpdatePossible = false; + } else { + sourceDigests.add(digest); + } + } + for (TsFileResource source : unseqSourceFiles) { + DualDigest digest = getFileDigest(source); + if (digest == null) { + digestUpdatePossible = false; + } else { + sourceDigests.add(digest); + } + } + + // Compute target file hashes and generate .merkle files + List targetDigests = new ArrayList<>(); + for (TsFileResource target : targetFiles) { + if (target.isDeleted()) { + continue; + } + String tsFilePath = target.getTsFilePath(); + List entries = MerkleHashComputer.computeEntries(tsFilePath); + if (entries.isEmpty()) { + continue; + } + + long fileXorHash = MerkleFileWriter.computeFileXorHash(entries); + long fileAddHash = MerkleFileWriter.computeFileAddHash(entries); + MerkleFileWriter.write(tsFilePath + ".merkle", entries, fileXorHash, fileAddHash); + targetDigests.add(new DualDigest(fileXorHash, fileAddHash)); + } + + if (digestUpdatePossible) { + merkleTree.onCompaction(sourceDigests, targetDigests, timePartition); + } else { + merkleTree.markPartitionDirty(timePartition); + } + + // Cleanup: delete source .merkle files and invalidate cache + cleanupSourceMerkleFiles(seqSourceFiles); + cleanupSourceMerkleFiles(unseqSourceFiles); + + LOGGER.debug( + "Compaction hook: updated partition {} Merkle tree, removed {} source digests", + timePartition, + sourceDigests.size()); + + } catch (IOException e) { + LOGGER.warn( + "Failed to update Merkle tree during compaction for partition {}: {}", + timePartition, + e.getMessage(), + e); + merkleTree.markPartitionDirty(timePartition); + } + } + + private DualDigest getFileDigest(TsFileResource source) { + String tsFilePath = source.getTsFilePath(); + try { + MerkleFileContent content = merkleFileCache.get(tsFilePath); + return content.getFileDigest(); + } catch (IOException e) { + // .merkle file might not exist (e.g., created before consistency module was enabled) + LOGGER.debug("No .merkle file for source TsFile {}: {}", tsFilePath, e.getMessage()); + return null; + } + } + + private void cleanupSourceMerkleFiles(List sourceFiles) { + for (TsFileResource source : sourceFiles) { + String tsFilePath = source.getTsFilePath(); + merkleFileCache.invalidate(tsFilePath); + File merkleFile = new File(tsFilePath + ".merkle"); + if (merkleFile.exists() && !merkleFile.delete()) { + LOGGER.warn("Failed to delete .merkle file: {}", merkleFile.getAbsolutePath()); + } + } + } +} diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/consistency/MerkleDeletionHook.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/consistency/MerkleDeletionHook.java new file mode 100644 index 0000000000000..16e49abbb865d --- /dev/null +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/consistency/MerkleDeletionHook.java @@ -0,0 +1,97 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.iotdb.db.storageengine.dataregion.consistency; + +import org.apache.iotdb.commons.consensus.iotv2.consistency.ConsistencyMerkleTree; +import org.apache.iotdb.db.storageengine.dataregion.tsfile.TsFileResource; + +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.util.List; +import java.util.Set; +import java.util.TreeSet; + +/** + * Hook into DataRegion's deletion path to mark affected time partitions as dirty in the + * ConsistencyMerkleTree. Rather than attempting to XOR-out exact deleted hashes (which requires + * expensive disk reads and has crash-consistency risks), we use lazy invalidation: mark the + * partition dirty and trigger a full rescan during the next consistency check cycle. + */ +public class MerkleDeletionHook { + + private static final Logger LOGGER = LoggerFactory.getLogger(MerkleDeletionHook.class); + + private final ConsistencyMerkleTree merkleTree; + + public MerkleDeletionHook(ConsistencyMerkleTree merkleTree) { + this.merkleTree = merkleTree; + } + + /** + * Called when a deletion (mod entry) affects one or more TsFiles. Marks the time partitions of + * all affected TsFiles as dirty. + * + * @param affectedTsFiles list of TsFileResource objects that overlap with the deletion + */ + public void onDeletion(List affectedTsFiles) { + Set affectedPartitions = new TreeSet<>(); + for (TsFileResource resource : affectedTsFiles) { + long partitionId = resource.getTimePartition(); + affectedPartitions.add(partitionId); + } + + for (long partitionId : affectedPartitions) { + merkleTree.markPartitionDirty(partitionId); + } + + LOGGER.debug( + "Deletion affected {} partitions, marked dirty: {}", + affectedPartitions.size(), + affectedPartitions); + } + + /** + * Called when a deletion affects a specific time range. Marks all tracked partitions that overlap + * with the range as dirty. + * + * @param startTime inclusive start timestamp of deletion + * @param endTime inclusive end timestamp of deletion + * @param timePartitionInterval the time partition interval in ms + */ + public void onDeletion(long startTime, long endTime, long timePartitionInterval) { + if (timePartitionInterval <= 0) { + return; + } + long startPartition = startTime / timePartitionInterval; + long endPartition = endTime / timePartitionInterval; + + for (long partitionId = startPartition; partitionId <= endPartition; partitionId++) { + merkleTree.markPartitionDirty(partitionId); + } + + LOGGER.debug( + "Deletion [{}, {}] marked partitions [{}, {}] as dirty", + startTime, + endTime, + startPartition, + endPartition); + } +} diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/consistency/MerkleFlushListener.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/consistency/MerkleFlushListener.java new file mode 100644 index 0000000000000..c35fd0779db28 --- /dev/null +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/consistency/MerkleFlushListener.java @@ -0,0 +1,95 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.iotdb.db.storageengine.dataregion.consistency; + +import org.apache.iotdb.commons.consensus.iotv2.consistency.ConsistencyMerkleTree; +import org.apache.iotdb.commons.consensus.iotv2.consistency.DualDigest; +import org.apache.iotdb.commons.consensus.iotv2.consistency.merkle.MerkleEntry; +import org.apache.iotdb.commons.consensus.iotv2.consistency.merkle.MerkleFileWriter; +import org.apache.iotdb.db.storageengine.dataregion.flush.FlushListener; +import org.apache.iotdb.db.storageengine.dataregion.memtable.IMemTable; +import org.apache.iotdb.db.storageengine.dataregion.tsfile.TsFileResource; + +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.io.IOException; +import java.util.List; + +/** + * FlushListener that computes hash digests for a newly flushed TsFile and generates the + * corresponding .merkle sidecar file. Also updates the in-memory ConsistencyMerkleTree with the + * file's root hash. + */ +public class MerkleFlushListener implements FlushListener { + + private static final Logger LOGGER = LoggerFactory.getLogger(MerkleFlushListener.class); + + private final TsFileResource tsFileResource; + private final ConsistencyMerkleTree merkleTree; + + public MerkleFlushListener(TsFileResource tsFileResource, ConsistencyMerkleTree merkleTree) { + this.tsFileResource = tsFileResource; + this.merkleTree = merkleTree; + } + + @Override + public void onMemTableFlushStarted(IMemTable memTable) { + // No action needed at flush start + } + + @Override + public void onMemTableFlushed(IMemTable memTable) { + try { + String tsFilePath = tsFileResource.getTsFilePath(); + long partitionId = tsFileResource.getTimePartition(); + + // Scan the flushed TsFile and compute per-(device, measurement, timeBucket) hashes + List entries = MerkleHashComputer.computeEntries(tsFilePath); + + if (entries.isEmpty()) { + return; + } + + long fileXorHash = MerkleFileWriter.computeFileXorHash(entries); + long fileAddHash = MerkleFileWriter.computeFileAddHash(entries); + + // Write .merkle sidecar file + MerkleFileWriter.write(tsFilePath + ".merkle", entries, fileXorHash, fileAddHash); + + // Update in-memory Merkle tree + merkleTree.onTsFileFlushed(partitionId, new DualDigest(fileXorHash, fileAddHash)); + + LOGGER.debug( + "Generated .merkle file for {} with {} entries, xorHash=0x{}, addHash=0x{}, partitionId={}", + tsFilePath, + entries.size(), + Long.toHexString(fileXorHash), + Long.toHexString(fileAddHash), + partitionId); + } catch (IOException e) { + LOGGER.warn( + "Failed to generate .merkle file for {}: {}", + tsFileResource.getTsFilePath(), + e.getMessage(), + e); + } + } +} diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/consistency/MerkleHashComputer.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/consistency/MerkleHashComputer.java new file mode 100644 index 0000000000000..2f24dccb9b6fe --- /dev/null +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/consistency/MerkleHashComputer.java @@ -0,0 +1,232 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.iotdb.db.storageengine.dataregion.consistency; + +import org.apache.iotdb.commons.consensus.iotv2.consistency.merkle.MerkleEntry; +import org.apache.iotdb.db.queryengine.execution.operator.source.relational.aggregation.grouped.hash.XxHash64; +import org.apache.iotdb.db.utils.EncryptDBUtils; + +import org.apache.tsfile.enums.TSDataType; +import org.apache.tsfile.file.metadata.ChunkMetadata; +import org.apache.tsfile.file.metadata.IDeviceID; +import org.apache.tsfile.read.TsFileDeviceIterator; +import org.apache.tsfile.read.TsFileSequenceReader; +import org.apache.tsfile.read.common.BatchData; +import org.apache.tsfile.read.common.Chunk; +import org.apache.tsfile.read.reader.chunk.ChunkReader; +import org.apache.tsfile.utils.Pair; + +import java.io.IOException; +import java.util.ArrayList; +import java.util.Iterator; +import java.util.List; +import java.util.Map; +import java.util.TreeMap; + +/** + * Scans a TsFile and computes per-(device, measurement, timeBucket) hash entries for the .merkle + * sidecar file. Each entry's hash is the XxHash64 of all sorted (timestamp, value) pairs in that + * bucket. + */ +public class MerkleHashComputer { + + static final long DEFAULT_TIME_BUCKET_INTERVAL_MS = 3_600_000L; // 1 hour + + private MerkleHashComputer() {} + + /** + * Scan a TsFile and compute all MerkleEntry hashes. + * + * @param tsFilePath absolute path to the TsFile + * @return sorted list of MerkleEntries (sorted by device -> measurement -> timeBucketStart) + */ + public static List computeEntries(String tsFilePath) throws IOException { + return computeEntries(tsFilePath, DEFAULT_TIME_BUCKET_INTERVAL_MS); + } + + public static List computeEntries(String tsFilePath, long timeBucketIntervalMs) + throws IOException { + // Accumulator: device -> measurement -> bucketStart -> BucketAccumulator + TreeMap>> accumulator = + new TreeMap<>(); + + try (TsFileSequenceReader reader = + new TsFileSequenceReader( + tsFilePath, EncryptDBUtils.getFirstEncryptParamFromTSFilePath(tsFilePath))) { + + TsFileDeviceIterator deviceIterator = reader.getAllDevicesIteratorWithIsAligned(); + while (deviceIterator.hasNext()) { + Pair devicePair = deviceIterator.next(); + IDeviceID deviceId = devicePair.left; + String deviceIdStr = deviceId.toString(); + + TreeMap> deviceAccum = + accumulator.computeIfAbsent(deviceIdStr, k -> new TreeMap<>()); + + Iterator>> measurementChunkIter = + reader.getMeasurementChunkMetadataListMapIterator(deviceId); + + while (measurementChunkIter.hasNext()) { + Map> measurementChunks = measurementChunkIter.next(); + for (Map.Entry> entry : measurementChunks.entrySet()) { + String measurement = entry.getKey(); + // Skip empty or time-column measurements + if (measurement.isEmpty()) { + continue; + } + TreeMap measAccum = + deviceAccum.computeIfAbsent(measurement, k -> new TreeMap<>()); + + for (ChunkMetadata chunkMeta : entry.getValue()) { + TSDataType dataType = chunkMeta.getDataType(); + Chunk chunk = reader.readMemChunk(chunkMeta); + ChunkReader chunkReader = new ChunkReader(chunk); + + while (chunkReader.hasNextSatisfiedPage()) { + BatchData batchData = chunkReader.nextPageData(); + while (batchData.hasCurrent()) { + long timestamp = batchData.currentTime(); + long bucketStart = (timestamp / timeBucketIntervalMs) * timeBucketIntervalMs; + + BucketAccumulator bucket = + measAccum.computeIfAbsent( + bucketStart, + bs -> new BucketAccumulator(bs, bs + timeBucketIntervalMs, dataType)); + bucket.addPoint(timestamp, batchData.currentValue()); + batchData.next(); + } + } + } + } + } + } + } + + return flattenToEntries(accumulator); + } + + private static List flattenToEntries( + TreeMap>> accumulator) { + List entries = new ArrayList<>(); + for (Map.Entry>> deviceEntry : + accumulator.entrySet()) { + String deviceId = deviceEntry.getKey(); + for (Map.Entry> measEntry : + deviceEntry.getValue().entrySet()) { + String measurement = measEntry.getKey(); + for (BucketAccumulator bucket : measEntry.getValue().values()) { + entries.add( + new MerkleEntry( + deviceId, + measurement, + bucket.bucketStart, + bucket.bucketEnd, + bucket.pointCount, + bucket.computeHash())); + } + } + } + return entries; + } + + /** Accumulates (timestamp, value) pairs in a time bucket and computes their combined hash. */ + static class BucketAccumulator { + final long bucketStart; + final long bucketEnd; + final TSDataType dataType; + int pointCount; + final TreeMap points; + + BucketAccumulator(long bucketStart, long bucketEnd, TSDataType dataType) { + this.bucketStart = bucketStart; + this.bucketEnd = bucketEnd; + this.dataType = dataType; + this.pointCount = 0; + this.points = new TreeMap<>(); + } + + void addPoint(long timestamp, Object value) { + points.put(timestamp, value); + pointCount++; + } + + long computeHash() { + XxHash64 hasher = new XxHash64(); + byte[] buf = new byte[8]; + for (Map.Entry entry : points.entrySet()) { + longToBytes(entry.getKey(), buf); + hasher.update(buf); + byte[] valueBytes = valueToBytes(entry.getValue(), dataType); + hasher.update(valueBytes); + } + return hasher.hash(); + } + } + + static void longToBytes(long v, byte[] buf) { + buf[0] = (byte) (v >>> 56); + buf[1] = (byte) (v >>> 48); + buf[2] = (byte) (v >>> 40); + buf[3] = (byte) (v >>> 32); + buf[4] = (byte) (v >>> 24); + buf[5] = (byte) (v >>> 16); + buf[6] = (byte) (v >>> 8); + buf[7] = (byte) v; + } + + static byte[] valueToBytes(Object value, TSDataType dataType) { + if (value == null) { + return new byte[0]; + } + switch (dataType) { + case BOOLEAN: + return new byte[] {(byte) ((Boolean) value ? 1 : 0)}; + case INT32: + case DATE: + return intToBytes((Integer) value); + case INT64: + case TIMESTAMP: + return longToBytesNew((Long) value); + case FLOAT: + return intToBytes(Float.floatToIntBits((Float) value)); + case DOUBLE: + return longToBytesNew(Double.doubleToLongBits((Double) value)); + case TEXT: + case STRING: + case BLOB: + if (value instanceof byte[]) { + return (byte[]) value; + } + return value.toString().getBytes(); + default: + return value.toString().getBytes(); + } + } + + private static byte[] intToBytes(int v) { + return new byte[] {(byte) (v >>> 24), (byte) (v >>> 16), (byte) (v >>> 8), (byte) v}; + } + + private static byte[] longToBytesNew(long v) { + byte[] buf = new byte[8]; + longToBytes(v, buf); + return buf; + } +} diff --git a/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/consensus/iotv2/consistency/ConsistencyCheckScheduler.java b/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/consensus/iotv2/consistency/ConsistencyCheckScheduler.java new file mode 100644 index 0000000000000..20aca89ff29ca --- /dev/null +++ b/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/consensus/iotv2/consistency/ConsistencyCheckScheduler.java @@ -0,0 +1,217 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.iotdb.commons.consensus.iotv2.consistency; + +import org.apache.iotdb.commons.concurrent.threadpool.ScheduledExecutorUtil; + +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.util.Map; +import java.util.concurrent.ConcurrentHashMap; +import java.util.concurrent.ScheduledExecutorService; +import java.util.concurrent.ScheduledFuture; +import java.util.concurrent.TimeUnit; +import java.util.concurrent.atomic.AtomicBoolean; + +/** + * Background scheduler for periodic consistency checks across all consensus groups managed by this + * DataNode. Integrates with IoTV2GlobalComponentContainer's background task service. + * + *

Features: + * + *

    + *
  • Configurable check interval (default: 1 hour) + *
  • Rate limiting to prevent resource saturation + *
  • Per-region tracking to avoid overlapping checks + *
  • SyncLag pre-check gate (only checks regions with completed replication) + *
+ */ +public class ConsistencyCheckScheduler { + + private static final Logger LOGGER = LoggerFactory.getLogger(ConsistencyCheckScheduler.class); + + private static final long DEFAULT_CHECK_INTERVAL_MS = 3_600_000L; // 1 hour + private static final long DEFAULT_INITIAL_DELAY_MS = 300_000L; // 5 minutes + + /** Callback interface for triggering consistency checks. */ + public interface ConsistencyCheckTrigger { + /** + * Trigger a consistency check for a specific consensus group. + * + * @param consensusGroupId the group to check + * @return true if the check was successfully initiated + */ + boolean triggerCheck(String consensusGroupId); + + /** + * Check if replication is complete for a consensus group. + * + * @param consensusGroupId the group to check + * @return true if syncLag <= 0 + */ + boolean isReplicationComplete(String consensusGroupId); + } + + private final ScheduledExecutorService executorService; + private final ConsistencyCheckTrigger trigger; + private final long checkIntervalMs; + private final long initialDelayMs; + private final AtomicBoolean running; + private ScheduledFuture scheduledFuture; + private final ConcurrentHashMap lastCheckTimes; + private final ConcurrentHashMap activeChecks; + + public ConsistencyCheckScheduler( + ScheduledExecutorService executorService, ConsistencyCheckTrigger trigger) { + this(executorService, trigger, DEFAULT_CHECK_INTERVAL_MS, DEFAULT_INITIAL_DELAY_MS); + } + + public ConsistencyCheckScheduler( + ScheduledExecutorService executorService, + ConsistencyCheckTrigger trigger, + long checkIntervalMs, + long initialDelayMs) { + this.executorService = executorService; + this.trigger = trigger; + this.checkIntervalMs = checkIntervalMs; + this.initialDelayMs = initialDelayMs; + this.running = new AtomicBoolean(false); + this.lastCheckTimes = new ConcurrentHashMap<>(); + this.activeChecks = new ConcurrentHashMap<>(); + } + + /** Start the periodic scheduling. */ + public void start() { + if (!running.compareAndSet(false, true)) { + LOGGER.warn("ConsistencyCheckScheduler already running"); + return; + } + scheduledFuture = + ScheduledExecutorUtil.safelyScheduleWithFixedDelay( + executorService, + this::runChecks, + initialDelayMs, + checkIntervalMs, + TimeUnit.MILLISECONDS); + LOGGER.info( + "ConsistencyCheckScheduler started: initialDelay={}ms, interval={}ms", + initialDelayMs, + checkIntervalMs); + } + + /** Stop the scheduler gracefully. */ + public void stop() { + if (!running.compareAndSet(true, false)) { + return; + } + if (scheduledFuture != null) { + scheduledFuture.cancel(false); + } + LOGGER.info("ConsistencyCheckScheduler stopped"); + } + + /** Register a consensus group for periodic checking. */ + public void registerGroup(String consensusGroupId) { + lastCheckTimes.putIfAbsent(consensusGroupId, 0L); + activeChecks.putIfAbsent(consensusGroupId, false); + LOGGER.debug("Registered consensus group {} for consistency checks", consensusGroupId); + } + + /** Unregister a consensus group. */ + public void unregisterGroup(String consensusGroupId) { + lastCheckTimes.remove(consensusGroupId); + activeChecks.remove(consensusGroupId); + } + + /** Trigger an immediate check for a specific group (e.g., from CLI). */ + public boolean triggerManualCheck(String consensusGroupId) { + if (activeChecks.getOrDefault(consensusGroupId, false)) { + LOGGER.info("Check already active for group {}, skipping", consensusGroupId); + return false; + } + return checkGroup(consensusGroupId); + } + + private void runChecks() { + if (!running.get()) { + return; + } + + long now = System.currentTimeMillis(); + for (Map.Entry entry : lastCheckTimes.entrySet()) { + String groupId = entry.getKey(); + long lastCheck = entry.getValue(); + + // Rate limiting: skip if checked recently + if (now - lastCheck < checkIntervalMs) { + continue; + } + + // Skip if a check is already active + if (activeChecks.getOrDefault(groupId, false)) { + continue; + } + + checkGroup(groupId); + } + } + + private boolean checkGroup(String consensusGroupId) { + try { + // SyncLag pre-check + if (!trigger.isReplicationComplete(consensusGroupId)) { + LOGGER.debug( + "Skipping consistency check for group {}: replication in progress", consensusGroupId); + return false; + } + + activeChecks.put(consensusGroupId, true); + lastCheckTimes.put(consensusGroupId, System.currentTimeMillis()); + + boolean triggered = trigger.triggerCheck(consensusGroupId); + if (triggered) { + LOGGER.info("Triggered consistency check for group {}", consensusGroupId); + } + return triggered; + } catch (Exception e) { + LOGGER.error( + "Error triggering consistency check for group {}: {}", + consensusGroupId, + e.getMessage(), + e); + return false; + } finally { + activeChecks.put(consensusGroupId, false); + } + } + + public boolean isRunning() { + return running.get(); + } + + public long getCheckIntervalMs() { + return checkIntervalMs; + } + + public int getRegisteredGroupCount() { + return lastCheckTimes.size(); + } +} diff --git a/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/consensus/iotv2/consistency/ConsistencyMerkleTree.java b/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/consensus/iotv2/consistency/ConsistencyMerkleTree.java new file mode 100644 index 0000000000000..cd5d3ef3b3c5f --- /dev/null +++ b/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/consensus/iotv2/consistency/ConsistencyMerkleTree.java @@ -0,0 +1,238 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.iotdb.commons.consensus.iotv2.consistency; + +import java.util.ArrayList; +import java.util.Collections; +import java.util.List; +import java.util.Map; +import java.util.TreeMap; +import java.util.concurrent.locks.ReadWriteLock; +import java.util.concurrent.locks.ReentrantReadWriteLock; + +/** + * In-memory incremental Merkle tree that holds Level 0 (region digest) and Level 1 (per-partition + * digest) as resident data. Level 2+ (device/measurement/timeBucket) are loaded on-demand from + * .merkle files. Supports dual-digest aggregation and lazy invalidation for deletions. + */ +public class ConsistencyMerkleTree { + + private volatile DualDigest regionDigest; + private final TreeMap partitions; + private final ReadWriteLock treeLock; + + public ConsistencyMerkleTree() { + this.regionDigest = DualDigest.ZERO; + this.partitions = new TreeMap<>(); + this.treeLock = new ReentrantReadWriteLock(); + } + + /** + * Get or create the node for a given partition. Lazily initializes partition nodes on first + * encounter. + */ + public TimePartitionMerkleNode getOrCreatePartitionNode(long partitionId) { + treeLock.readLock().lock(); + try { + TimePartitionMerkleNode node = partitions.get(partitionId); + if (node != null) { + return node; + } + } finally { + treeLock.readLock().unlock(); + } + + treeLock.writeLock().lock(); + try { + return partitions.computeIfAbsent(partitionId, TimePartitionMerkleNode::new); + } finally { + treeLock.writeLock().unlock(); + } + } + + public TimePartitionMerkleNode getPartitionNode(long partitionId) { + treeLock.readLock().lock(); + try { + return partitions.get(partitionId); + } finally { + treeLock.readLock().unlock(); + } + } + + /** + * Called after a TsFile flush to incorporate the new file's hash into the appropriate partition's + * dual-digest. + */ + public void onTsFileFlushed(long partitionId, long fileRootHash) { + onTsFileFlushed(partitionId, DualDigest.fromSingleHash(fileRootHash)); + } + + public void onTsFileFlushed(long partitionId, DualDigest fileDigest) { + TimePartitionMerkleNode node = getOrCreatePartitionNode(partitionId); + node.addDigest(fileDigest); + recomputeRegionDigest(); + } + + /** + * Called during compaction when source TsFiles are removed and a target TsFile is created. + * + * @param partitionId the partition being compacted + * @param sourceFileRootHashes root hashes of the source TsFiles being removed + * @param targetFileRootHash root hash of the newly created target TsFile + */ + public void onCompaction( + long partitionId, List sourceFileRootHashes, long targetFileRootHash) { + onCompaction( + partitionId, + sourceFileRootHashes.stream().map(DualDigest::fromSingleHash).collect(java.util.stream.Collectors.toList()), + Collections.singletonList(DualDigest.fromSingleHash(targetFileRootHash))); + } + + public void onCompaction( + long partitionId, List sourceDigests, List targetDigests) { + TimePartitionMerkleNode node = getOrCreatePartitionNode(partitionId); + for (DualDigest sourceDigest : sourceDigests) { + node.removeDigest(sourceDigest); + } + for (DualDigest targetDigest : targetDigests) { + node.addDigest(targetDigest); + } + recomputeRegionDigest(); + } + + /** + * Overloaded compaction hook accepting pre-collected source hashes and a combined target hash. + * + * @param sourceHashes root hashes of source TsFiles being removed + * @param targetCombinedHash XOR of all target TsFile root hashes + * @param partitionId the partition being compacted + */ + public void onCompaction(List sourceHashes, long targetCombinedHash, long partitionId) { + onCompaction( + partitionId, + sourceHashes.stream().map(DualDigest::fromSingleHash).collect(java.util.stream.Collectors.toList()), + Collections.singletonList(DualDigest.fromSingleHash(targetCombinedHash))); + } + + public void onCompaction( + List sourceDigests, List targetDigests, long partitionId) { + TimePartitionMerkleNode node = getOrCreatePartitionNode(partitionId); + for (DualDigest sourceDigest : sourceDigests) { + node.removeDigest(sourceDigest); + } + for (DualDigest targetDigest : targetDigests) { + node.addDigest(targetDigest); + } + recomputeRegionDigest(); + } + + /** + * Mark a partition as dirty when deletions affect it. The next consistency check cycle will + * trigger a full rescan and rebuild for this partition. + */ + public void markPartitionDirty(long partitionId) { + TimePartitionMerkleNode node = getOrCreatePartitionNode(partitionId); + node.markDirty(); + } + + /** + * Prune a verified cold partition from the in-memory tree. Called after successful verification + * and commit to reduce memory footprint. + */ + public void prunePartition(long partitionId) { + treeLock.writeLock().lock(); + try { + partitions.remove(partitionId); + recomputeRegionDigestLocked(); + } finally { + treeLock.writeLock().unlock(); + } + } + + /** Get the current region-level dual-digest (Level 0). */ + public DualDigest getRegionDigest() { + return regionDigest; + } + + /** Return all partition IDs within the given time range [startInclusive, endExclusive). */ + public List getPartitionIds(long startInclusive, long endExclusive) { + treeLock.readLock().lock(); + try { + return new ArrayList<>(partitions.subMap(startInclusive, endExclusive).keySet()); + } finally { + treeLock.readLock().unlock(); + } + } + + /** Return all partition IDs in the tree. */ + public List getAllPartitionIds() { + treeLock.readLock().lock(); + try { + return new ArrayList<>(partitions.keySet()); + } finally { + treeLock.readLock().unlock(); + } + } + + /** Return an unmodifiable view of dirty partitions. */ + public List getDirtyPartitionIds() { + treeLock.readLock().lock(); + try { + List dirtyIds = new ArrayList<>(); + for (Map.Entry entry : partitions.entrySet()) { + if (entry.getValue().isDirty()) { + dirtyIds.add(entry.getKey()); + } + } + return Collections.unmodifiableList(dirtyIds); + } finally { + treeLock.readLock().unlock(); + } + } + + /** Recompute Level-0 region digest from all partition digests. */ + private void recomputeRegionDigest() { + treeLock.readLock().lock(); + try { + recomputeRegionDigestLocked(); + } finally { + treeLock.readLock().unlock(); + } + } + + /** Must be called while holding at least a read lock on treeLock. */ + private void recomputeRegionDigestLocked() { + DualDigest digest = DualDigest.ZERO; + for (TimePartitionMerkleNode node : partitions.values()) { + digest = digest.merge(node.getPartitionDigest()); + } + this.regionDigest = digest; + } + + /** Return the number of tracked partitions. */ + public int getPartitionCount() { + treeLock.readLock().lock(); + try { + return partitions.size(); + } finally { + treeLock.readLock().unlock(); + } + } +} diff --git a/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/consensus/iotv2/consistency/ConsistencyViewBuilder.java b/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/consensus/iotv2/consistency/ConsistencyViewBuilder.java new file mode 100644 index 0000000000000..9076fc5406176 --- /dev/null +++ b/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/consensus/iotv2/consistency/ConsistencyViewBuilder.java @@ -0,0 +1,133 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.iotdb.commons.consensus.iotv2.consistency; + +import org.apache.iotdb.commons.consensus.iotv2.consistency.merkle.MerkleFileCache; +import org.apache.iotdb.commons.consensus.iotv2.consistency.merkle.MerkleFileContent; + +import java.io.IOException; +import java.util.ArrayList; +import java.util.List; + +/** + * Builds a consistency view for a partition using the snapshot-then-release pattern: acquires a + * short read lock to snapshot the TsFile list, then releases the lock before performing heavy + * .merkle file I/O. Includes staleness detection via version counter. + */ +public class ConsistencyViewBuilder { + + /** + * Interface to decouple from TsFileManager; allows the builder to be used without direct + * dependency on the storage engine module. + */ + public interface TsFileListProvider { + + /** Acquire a read lock on the TsFile resource list. */ + void readLock(); + + /** Release the read lock. */ + void readUnlock(); + + /** + * Get all TsFile paths for the given partition. + * + * @param partitionId the time partition to query + * @return list of TsFile paths (both sequence and unsequence) + */ + List getTsFileSnapshots(long partitionId); + } + + /** Lightweight snapshot of a TsFile reference for off-lock processing. */ + public static class TsFileSnapshot { + private final String tsFilePath; + private final boolean deleted; + + public TsFileSnapshot(String tsFilePath, boolean deleted) { + this.tsFilePath = tsFilePath; + this.deleted = deleted; + } + + public String getTsFilePath() { + return tsFilePath; + } + + public boolean isDeleted() { + return deleted; + } + } + + private final MerkleFileCache merkleFileCache; + + public ConsistencyViewBuilder(MerkleFileCache merkleFileCache) { + this.merkleFileCache = merkleFileCache; + } + + /** + * Build a consistency view for a specific partition. + * + * @param provider the TsFile list provider (wraps TsFileManager) + * @param partitionNode the partition's Merkle node for staleness checks + * @return list of loaded MerkleFileContent for all active TsFiles in the partition + * @throws StaleSnapshotException if the partition was modified while building the view + * @throws IOException on .merkle file read failure + */ + public List buildView( + TsFileListProvider provider, TimePartitionMerkleNode partitionNode) + throws IOException, StaleSnapshotException { + long expectedVersion = partitionNode.getVersion(); + + // Step 1: Short read lock -- snapshot the TsFile list (microseconds) + List snapshot; + provider.readLock(); + try { + snapshot = new ArrayList<>(provider.getTsFileSnapshots(partitionNode.getPartitionId())); + } finally { + provider.readUnlock(); + } + + // Step 2: Off-lock -- heavy I/O to load .merkle files (milliseconds-seconds) + List contents = new ArrayList<>(); + for (TsFileSnapshot tsFile : snapshot) { + if (tsFile.isDeleted()) { + continue; + } + try { + MerkleFileContent content = merkleFileCache.get(tsFile.getTsFilePath()); + contents.add(content); + } catch (IOException e) { + // .merkle file may have been cleaned up by compaction; skip this file + if (!tsFile.isDeleted()) { + throw e; + } + } + } + + // Step 3: Staleness check -- verify snapshot is still valid + long currentVersion = partitionNode.getVersion(); + if (currentVersion != expectedVersion) { + throw new StaleSnapshotException( + String.format( + "Partition %d modified during view build (expected version=%d, current=%d), retry", + partitionNode.getPartitionId(), expectedVersion, currentVersion)); + } + + return contents; + } +} diff --git a/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/consensus/iotv2/consistency/DualDigest.java b/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/consensus/iotv2/consistency/DualDigest.java new file mode 100644 index 0000000000000..6a5b1c34e15d0 --- /dev/null +++ b/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/consensus/iotv2/consistency/DualDigest.java @@ -0,0 +1,105 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.iotdb.commons.consensus.iotv2.consistency; + +import java.io.DataOutputStream; +import java.io.IOException; +import java.nio.ByteBuffer; +import java.util.Objects; + +/** + * Immutable dual-digest tuple combining XOR and additive hashing to eliminate the cancellation + * vulnerability of single-XOR aggregation. For two datasets to produce the same dual-digest while + * being different, they must satisfy BOTH XOR(S1)==XOR(S2) AND SUM(S1)==SUM(S2) mod 2^64 + * simultaneously, giving a false-negative probability of ~2^(-128). + */ +public final class DualDigest { + + public static final DualDigest ZERO = new DualDigest(0L, 0L); + + private final long xorHash; + private final long additiveHash; + + public DualDigest(long xorHash, long additiveHash) { + this.xorHash = xorHash; + this.additiveHash = additiveHash; + } + + public static DualDigest fromSingleHash(long hash) { + return new DualDigest(hash, hash); + } + + public DualDigest xorIn(long childHash) { + return new DualDigest(this.xorHash ^ childHash, this.additiveHash + childHash); + } + + public DualDigest xorOut(long childHash) { + return new DualDigest(this.xorHash ^ childHash, this.additiveHash - childHash); + } + + public DualDigest merge(DualDigest other) { + return new DualDigest(this.xorHash ^ other.xorHash, this.additiveHash + other.additiveHash); + } + + public DualDigest subtract(DualDigest other) { + return new DualDigest(this.xorHash ^ other.xorHash, this.additiveHash - other.additiveHash); + } + + public boolean matches(DualDigest other) { + return this.xorHash == other.xorHash && this.additiveHash == other.additiveHash; + } + + public long getXorHash() { + return xorHash; + } + + public long getAdditiveHash() { + return additiveHash; + } + + public void serialize(DataOutputStream stream) throws IOException { + stream.writeLong(xorHash); + stream.writeLong(additiveHash); + } + + public static DualDigest deserialize(ByteBuffer buffer) { + long xor = buffer.getLong(); + long add = buffer.getLong(); + return new DualDigest(xor, add); + } + + @Override + public boolean equals(Object o) { + if (this == o) return true; + if (!(o instanceof DualDigest)) return false; + DualDigest that = (DualDigest) o; + return xorHash == that.xorHash && additiveHash == that.additiveHash; + } + + @Override + public int hashCode() { + return Objects.hash(xorHash, additiveHash); + } + + @Override + public String toString() { + return String.format("DualDigest{xor=0x%016X, add=0x%016X}", xorHash, additiveHash); + } +} diff --git a/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/consensus/iotv2/consistency/RepairProgressTable.java b/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/consensus/iotv2/consistency/RepairProgressTable.java new file mode 100644 index 0000000000000..2fcaf948fc1e5 --- /dev/null +++ b/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/consensus/iotv2/consistency/RepairProgressTable.java @@ -0,0 +1,229 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.iotdb.commons.consensus.iotv2.consistency; + +import java.util.Collection; +import java.util.Map; +import java.util.concurrent.ConcurrentHashMap; + +/** + * In-memory representation of the partition-level repair progress tracking for a single consensus + * group. This serves as the runtime cache backed by the __system.repair_progress system table. + * + *

Schema: + * + *

    + *
  • Region-level row: (consensus_group_id, global_repaired_watermark, last_repair_time, + * repair_status) + *
  • Partition-level rows: (consensus_group_id, partition_id, repaired_to, status, + * last_failure_reason) + *
+ */ +public class RepairProgressTable { + + /** Status of repair for a single partition. */ + public enum PartitionRepairStatus { + PENDING, + VERIFIED, + FAILED, + DIRTY + } + + /** Status of the overall repair cycle for a region. */ + public enum RegionRepairStatus { + IDLE, + RUNNING, + FAILED + } + + /** Progress entry for a single partition. */ + public static class PartitionProgress { + private final long partitionId; + private volatile long repairedTo; + private volatile PartitionRepairStatus status; + private volatile String lastFailureReason; + + public PartitionProgress(long partitionId) { + this.partitionId = partitionId; + this.repairedTo = 0L; + this.status = PartitionRepairStatus.PENDING; + this.lastFailureReason = null; + } + + public long getPartitionId() { + return partitionId; + } + + public long getRepairedTo() { + return repairedTo; + } + + public void setRepairedTo(long repairedTo) { + this.repairedTo = repairedTo; + } + + public PartitionRepairStatus getStatus() { + return status; + } + + public void setStatus(PartitionRepairStatus status) { + this.status = status; + } + + public String getLastFailureReason() { + return lastFailureReason; + } + + public void markVerified(long newRepairedTo) { + this.repairedTo = newRepairedTo; + this.status = PartitionRepairStatus.VERIFIED; + this.lastFailureReason = null; + } + + public void markFailed(String reason) { + this.status = PartitionRepairStatus.FAILED; + this.lastFailureReason = reason; + } + + public void markDirty() { + this.status = PartitionRepairStatus.DIRTY; + } + } + + private final String consensusGroupId; + private volatile long globalRepairedWatermark; + private volatile long lastRepairTime; + private volatile RegionRepairStatus regionStatus; + private final ConcurrentHashMap partitionProgress; + + public RepairProgressTable(String consensusGroupId) { + this.consensusGroupId = consensusGroupId; + this.globalRepairedWatermark = 0L; + this.lastRepairTime = 0L; + this.regionStatus = RegionRepairStatus.IDLE; + this.partitionProgress = new ConcurrentHashMap<>(); + } + + /** Get or create progress entry for a partition. */ + public PartitionProgress getOrCreatePartition(long partitionId) { + return partitionProgress.computeIfAbsent(partitionId, PartitionProgress::new); + } + + public PartitionProgress getPartition(long partitionId) { + return partitionProgress.get(partitionId); + } + + /** + * Commit a partition as verified with a new repaired_to watermark. + * + * @param partitionId the partition to commit + * @param repairedTo the new watermark for this partition + */ + public void commitPartition(long partitionId, long repairedTo) { + PartitionProgress progress = getOrCreatePartition(partitionId); + progress.markVerified(repairedTo); + } + + /** + * Mark a partition as failed. + * + * @param partitionId the partition that failed + * @param reason the failure reason + */ + public void failPartition(long partitionId, String reason) { + PartitionProgress progress = getOrCreatePartition(partitionId); + progress.markFailed(reason); + } + + /** + * Mark a partition as dirty (modified after verification). + * + * @param partitionId the partition to mark dirty + */ + public void dirtyPartition(long partitionId) { + PartitionProgress progress = getOrCreatePartition(partitionId); + progress.markDirty(); + } + + /** + * Advance the global watermark. Rule: global_repaired_watermark = MIN(p.repaired_to for ALL + * partitions p in effective range). Only advances when every partition has been successfully + * verified. + * + * @return the new global watermark + */ + public long advanceGlobalWatermark() { + long minRepairedTo = Long.MAX_VALUE; + boolean hasPartitions = false; + + for (PartitionProgress progress : partitionProgress.values()) { + if (progress.getStatus() == PartitionRepairStatus.VERIFIED) { + hasPartitions = true; + minRepairedTo = Math.min(minRepairedTo, progress.getRepairedTo()); + } else if (progress.getStatus() == PartitionRepairStatus.FAILED + || progress.getStatus() == PartitionRepairStatus.DIRTY) { + // Failed/dirty partitions block watermark advancement + hasPartitions = true; + minRepairedTo = Math.min(minRepairedTo, progress.getRepairedTo()); + } + } + + if (hasPartitions && minRepairedTo != Long.MAX_VALUE) { + this.globalRepairedWatermark = minRepairedTo; + } + this.lastRepairTime = System.currentTimeMillis(); + return this.globalRepairedWatermark; + } + + /** Get all partitions that need repair (PENDING, FAILED, or DIRTY). */ + public Collection getPartitionsNeedingRepair() { + return partitionProgress.values().stream() + .filter( + p -> + p.getStatus() == PartitionRepairStatus.PENDING + || p.getStatus() == PartitionRepairStatus.FAILED + || p.getStatus() == PartitionRepairStatus.DIRTY) + .collect(java.util.stream.Collectors.toList()); + } + + public String getConsensusGroupId() { + return consensusGroupId; + } + + public long getGlobalRepairedWatermark() { + return globalRepairedWatermark; + } + + public long getLastRepairTime() { + return lastRepairTime; + } + + public RegionRepairStatus getRegionStatus() { + return regionStatus; + } + + public void setRegionStatus(RegionRepairStatus status) { + this.regionStatus = status; + } + + public Map getAllPartitionProgress() { + return java.util.Collections.unmodifiableMap(partitionProgress); + } +} diff --git a/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/consensus/iotv2/consistency/StaleSnapshotException.java b/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/consensus/iotv2/consistency/StaleSnapshotException.java new file mode 100644 index 0000000000000..3a1c14f549206 --- /dev/null +++ b/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/consensus/iotv2/consistency/StaleSnapshotException.java @@ -0,0 +1,31 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.iotdb.commons.consensus.iotv2.consistency; + +/** + * Thrown when a partition is modified (by flush, compaction, or deletion) during consistency view + * construction, indicating the snapshot is stale and the operation should be retried. + */ +public class StaleSnapshotException extends Exception { + + public StaleSnapshotException(String message) { + super(message); + } +} diff --git a/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/consensus/iotv2/consistency/TimePartitionMerkleNode.java b/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/consensus/iotv2/consistency/TimePartitionMerkleNode.java new file mode 100644 index 0000000000000..10f672c856a8b --- /dev/null +++ b/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/consensus/iotv2/consistency/TimePartitionMerkleNode.java @@ -0,0 +1,129 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.iotdb.commons.consensus.iotv2.consistency; + +import java.util.concurrent.atomic.AtomicLong; +import java.util.concurrent.locks.ReadWriteLock; +import java.util.concurrent.locks.ReentrantReadWriteLock; + +/** + * Represents a Level-1 node in the consistency Merkle tree, holding the dual-digest for a single + * time partition. Uses fine-grained locking to support concurrent flush/compaction/check threads. + */ +public class TimePartitionMerkleNode { + + private final long partitionId; + private volatile DualDigest partitionDigest; + private volatile boolean dirty; + private volatile long lastVerifiedTime; + private final AtomicLong version; + private final ReadWriteLock lock; + + public TimePartitionMerkleNode(long partitionId) { + this.partitionId = partitionId; + this.partitionDigest = DualDigest.ZERO; + this.dirty = false; + this.lastVerifiedTime = 0L; + this.version = new AtomicLong(0); + this.lock = new ReentrantReadWriteLock(); + } + + /** + * XOR a file's root hash into this partition's dual-digest. Called when a new TsFile is flushed + * or a compaction target is created. + */ + public void xorIn(long fileRootHash) { + addDigest(DualDigest.fromSingleHash(fileRootHash)); + } + + public void addDigest(DualDigest digest) { + lock.writeLock().lock(); + try { + partitionDigest = partitionDigest.merge(digest); + version.incrementAndGet(); + } finally { + lock.writeLock().unlock(); + } + } + + /** + * XOR a file's root hash out of this partition's dual-digest. Called when a source TsFile is + * removed during compaction. + */ + public void xorOut(long fileRootHash) { + removeDigest(DualDigest.fromSingleHash(fileRootHash)); + } + + public void removeDigest(DualDigest digest) { + lock.writeLock().lock(); + try { + partitionDigest = partitionDigest.subtract(digest); + version.incrementAndGet(); + } finally { + lock.writeLock().unlock(); + } + } + + public DualDigest getPartitionDigest() { + lock.readLock().lock(); + try { + return partitionDigest; + } finally { + lock.readLock().unlock(); + } + } + + /** Reset the digest from scratch, typically after rebuilding a dirty partition. */ + public void resetDigest(DualDigest newDigest) { + lock.writeLock().lock(); + try { + this.partitionDigest = newDigest; + this.dirty = false; + this.version.incrementAndGet(); + } finally { + lock.writeLock().unlock(); + } + } + + public void markDirty() { + this.dirty = true; + this.version.incrementAndGet(); + } + + public boolean isDirty() { + return dirty; + } + + public long getPartitionId() { + return partitionId; + } + + public long getVersion() { + return version.get(); + } + + public long getLastVerifiedTime() { + return lastVerifiedTime; + } + + public void setLastVerifiedTime(long lastVerifiedTime) { + this.lastVerifiedTime = lastVerifiedTime; + } +} diff --git a/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/consensus/iotv2/consistency/ibf/CompositeKeyCodec.java b/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/consensus/iotv2/consistency/ibf/CompositeKeyCodec.java new file mode 100644 index 0000000000000..65aee1fd88c8d --- /dev/null +++ b/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/consensus/iotv2/consistency/ibf/CompositeKeyCodec.java @@ -0,0 +1,88 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.iotdb.commons.consensus.iotv2.consistency.ibf; + +/** + * Codec for the 8-byte composite key used in IBF cells. Layout: + * + *
+ *   [deviceIndex: 2 bytes][measurementIndex: 2 bytes][timestamp_bucket_offset: 4 bytes]
+ * 
+ * + * The deviceIndex and measurementIndex are ordinal indices (sorted order) agreed upon by Leader and + * Follower during the NEGOTIATE_KEY_MAPPING phase. The timestamp_bucket_offset encodes (timestamp - + * timeBucketStart) in units of the minimum timestamp resolution. + */ +public final class CompositeKeyCodec { + + public static final long DEFAULT_TIMESTAMP_RESOLUTION = 1L; + public static final int MAX_DEVICE_INDEX = 0xFFFF; + public static final int MAX_MEASUREMENT_INDEX = 0xFFFF; + + private CompositeKeyCodec() {} + + /** + * Encode a data point's location into a composite key. + * + * @param deviceIndex ordinal index of the device within the partition + * @param measurementIndex ordinal index of the measurement within the device + * @param timestamp the data point's timestamp + * @param bucketStart the start of the time bucket + * @param timestampResolution minimum timestamp resolution + * @return 8-byte composite key + */ + public static long encode( + int deviceIndex, + int measurementIndex, + long timestamp, + long bucketStart, + long timestampResolution) { + int tsOffset = (int) ((timestamp - bucketStart) / timestampResolution); + return ((long) (deviceIndex & 0xFFFF) << 48) + | ((long) (measurementIndex & 0xFFFF) << 32) + | (tsOffset & 0xFFFFFFFFL); + } + + public static int extractDeviceIndex(long compositeKey) { + return (int) ((compositeKey >>> 48) & 0xFFFF); + } + + public static int extractMeasurementIndex(long compositeKey) { + return (int) ((compositeKey >>> 32) & 0xFFFF); + } + + public static int extractTimestampOffset(long compositeKey) { + return (int) (compositeKey & 0xFFFFFFFFL); + } + + /** + * Recover the original timestamp from the composite key. + * + * @param compositeKey the encoded key + * @param bucketStart the time bucket start + * @param timestampResolution the timestamp resolution + * @return the original timestamp + */ + public static long recoverTimestamp( + long compositeKey, long bucketStart, long timestampResolution) { + int tsOffset = extractTimestampOffset(compositeKey); + return bucketStart + (long) tsOffset * timestampResolution; + } +} diff --git a/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/consensus/iotv2/consistency/ibf/DataPointLocator.java b/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/consensus/iotv2/consistency/ibf/DataPointLocator.java new file mode 100644 index 0000000000000..f865ca5e7b33c --- /dev/null +++ b/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/consensus/iotv2/consistency/ibf/DataPointLocator.java @@ -0,0 +1,71 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.iotdb.commons.consensus.iotv2.consistency.ibf; + +import java.util.Objects; + +/** + * Locates a specific data point by its logical coordinates: (deviceId, measurement, timestamp). + * Resolved from a composite key via the RowRefIndex. + */ +public class DataPointLocator { + + private final String deviceId; + private final String measurement; + private final long timestamp; + + public DataPointLocator(String deviceId, String measurement, long timestamp) { + this.deviceId = deviceId; + this.measurement = measurement; + this.timestamp = timestamp; + } + + public String getDeviceId() { + return deviceId; + } + + public String getMeasurement() { + return measurement; + } + + public long getTimestamp() { + return timestamp; + } + + @Override + public boolean equals(Object o) { + if (this == o) return true; + if (!(o instanceof DataPointLocator)) return false; + DataPointLocator that = (DataPointLocator) o; + return timestamp == that.timestamp + && Objects.equals(deviceId, that.deviceId) + && Objects.equals(measurement, that.measurement); + } + + @Override + public int hashCode() { + return Objects.hash(deviceId, measurement, timestamp); + } + + @Override + public String toString() { + return String.format("(%s, %s, %d)", deviceId, measurement, timestamp); + } +} diff --git a/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/consensus/iotv2/consistency/ibf/DiffEntry.java b/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/consensus/iotv2/consistency/ibf/DiffEntry.java new file mode 100644 index 0000000000000..f5d6edb164fb3 --- /dev/null +++ b/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/consensus/iotv2/consistency/ibf/DiffEntry.java @@ -0,0 +1,59 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.iotdb.commons.consensus.iotv2.consistency.ibf; + +/** Represents a single decoded diff from IBF subtraction and decoding. */ +public class DiffEntry { + + public enum DiffType { + /** Key exists on the Leader (positive side) but not on the Follower. */ + LEADER_HAS, + /** Key exists on the Follower (negative side) but not on the Leader. */ + FOLLOWER_HAS + } + + private final long compositeKey; + private final long valueHash; + private final DiffType type; + + public DiffEntry(long compositeKey, long valueHash, DiffType type) { + this.compositeKey = compositeKey; + this.valueHash = valueHash; + this.type = type; + } + + public long getCompositeKey() { + return compositeKey; + } + + public long getValueHash() { + return valueHash; + } + + public DiffType getType() { + return type; + } + + @Override + public String toString() { + return String.format( + "DiffEntry{key=0x%016X, valHash=0x%016X, type=%s}", compositeKey, valueHash, type); + } +} diff --git a/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/consensus/iotv2/consistency/ibf/IBFCell.java b/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/consensus/iotv2/consistency/ibf/IBFCell.java new file mode 100644 index 0000000000000..1fb84c5d3a4ac --- /dev/null +++ b/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/consensus/iotv2/consistency/ibf/IBFCell.java @@ -0,0 +1,98 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.iotdb.commons.consensus.iotv2.consistency.ibf; + +import java.io.DataOutputStream; +import java.io.IOException; +import java.nio.ByteBuffer; + +/** A single cell in an Invertible Bloom Filter. */ +public class IBFCell { + + int count; + long keySum; + long valueChecksum; + + public IBFCell() { + this.count = 0; + this.keySum = 0L; + this.valueChecksum = 0L; + } + + public IBFCell(int count, long keySum, long valueChecksum) { + this.count = count; + this.keySum = keySum; + this.valueChecksum = valueChecksum; + } + + public boolean isPure() { + return count == 1 || count == -1; + } + + public boolean isEmpty() { + return count == 0 && keySum == 0L && valueChecksum == 0L; + } + + public void add(long key, long valueHash) { + count += 1; + keySum ^= key; + valueChecksum ^= valueHash; + } + + public void remove(long key, long valueHash) { + count -= 1; + keySum ^= key; + valueChecksum ^= valueHash; + } + + public void subtract(IBFCell other) { + count -= other.count; + keySum ^= other.keySum; + valueChecksum ^= other.valueChecksum; + } + + public int getCount() { + return count; + } + + public long getKeySum() { + return keySum; + } + + public long getValueChecksum() { + return valueChecksum; + } + + public void serialize(DataOutputStream out) throws IOException { + out.writeInt(count); + out.writeLong(keySum); + out.writeLong(valueChecksum); + } + + public static IBFCell deserialize(ByteBuffer buffer) { + int count = buffer.getInt(); + long keySum = buffer.getLong(); + long valueChecksum = buffer.getLong(); + return new IBFCell(count, keySum, valueChecksum); + } + + /** Serialized size in bytes: 4 (count) + 8 (keySum) + 8 (valueChecksum) = 20 */ + public static final int SERIALIZED_SIZE = 20; +} diff --git a/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/consensus/iotv2/consistency/ibf/IBFDecodeResult.java b/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/consensus/iotv2/consistency/ibf/IBFDecodeResult.java new file mode 100644 index 0000000000000..5d0dbab75acad --- /dev/null +++ b/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/consensus/iotv2/consistency/ibf/IBFDecodeResult.java @@ -0,0 +1,58 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.iotdb.commons.consensus.iotv2.consistency.ibf; + +import java.util.Collections; +import java.util.List; + +/** The result of decoding an IBF difference. */ +public class IBFDecodeResult { + + private final boolean success; + private final List decodedEntries; + private final int partialCount; + + private IBFDecodeResult(boolean success, List decodedEntries, int partialCount) { + this.success = success; + this.decodedEntries = decodedEntries; + this.partialCount = partialCount; + } + + public static IBFDecodeResult success(List entries) { + return new IBFDecodeResult(true, entries, entries.size()); + } + + public static IBFDecodeResult failure(List partialEntries, int partialCount) { + return new IBFDecodeResult(false, partialEntries, partialCount); + } + + public boolean isSuccess() { + return success; + } + + public List getDecodedEntries() { + return Collections.unmodifiableList(decodedEntries); + } + + /** Number of entries decoded (may be partial if decode failed). */ + public int getPartialCount() { + return partialCount; + } +} diff --git a/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/consensus/iotv2/consistency/ibf/InvertibleBloomFilter.java b/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/consensus/iotv2/consistency/ibf/InvertibleBloomFilter.java new file mode 100644 index 0000000000000..1aba086bed094 --- /dev/null +++ b/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/consensus/iotv2/consistency/ibf/InvertibleBloomFilter.java @@ -0,0 +1,210 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.iotdb.commons.consensus.iotv2.consistency.ibf; + +import java.io.DataOutputStream; +import java.io.IOException; +import java.nio.ByteBuffer; +import java.util.ArrayDeque; +import java.util.ArrayList; +import java.util.Deque; +import java.util.List; + +/** + * Invertible Bloom Filter (IBF) for set difference computation with O(d) communication complexity + * where d is the number of differences. Uses k=3 hash functions to map (compositeKey, valueHash) + * pairs into cells. + */ +public class InvertibleBloomFilter { + + private static final int DEFAULT_HASH_COUNT = 3; + private static final long HASH_SEED_1 = 0x9E3779B97F4A7C15L; + private static final long HASH_SEED_2 = 0x517CC1B727220A95L; + private static final long HASH_SEED_3 = 0x6C62272E07BB0142L; + private static final long[] SEEDS = {HASH_SEED_1, HASH_SEED_2, HASH_SEED_3}; + + private final IBFCell[] cells; + private final int cellCount; + private final int hashCount; + + public InvertibleBloomFilter(int cellCount) { + this(cellCount, DEFAULT_HASH_COUNT); + } + + public InvertibleBloomFilter(int cellCount, int hashCount) { + this.cellCount = Math.max(cellCount, 1); + this.hashCount = hashCount; + this.cells = new IBFCell[this.cellCount]; + for (int i = 0; i < this.cellCount; i++) { + cells[i] = new IBFCell(); + } + } + + private InvertibleBloomFilter(IBFCell[] cells, int hashCount) { + this.cells = cells; + this.cellCount = cells.length; + this.hashCount = hashCount; + } + + public void insert(long compositeKey, long valueHash) { + for (int i = 0; i < hashCount; i++) { + int idx = hashToIndex(compositeKey, i); + cells[idx].add(compositeKey, valueHash); + } + } + + public void remove(long compositeKey, long valueHash) { + for (int i = 0; i < hashCount; i++) { + int idx = hashToIndex(compositeKey, i); + cells[idx].remove(compositeKey, valueHash); + } + } + + /** + * Subtract another IBF from this one (element-wise). The result IBF encodes the symmetric + * difference: keys in this but not other (count=+1) and keys in other but not this (count=-1). + */ + public InvertibleBloomFilter subtract(InvertibleBloomFilter other) { + if (this.cellCount != other.cellCount) { + throw new IllegalArgumentException( + "IBF cell counts must match: " + this.cellCount + " vs " + other.cellCount); + } + IBFCell[] result = new IBFCell[cellCount]; + for (int i = 0; i < cellCount; i++) { + result[i] = + new IBFCell(this.cells[i].count, this.cells[i].keySum, this.cells[i].valueChecksum); + result[i].subtract(other.cells[i]); + } + return new InvertibleBloomFilter(result, this.hashCount); + } + + /** + * Decode the IBF to recover all diff entries. The IBF should be the result of a subtraction + * (IBF_Leader - IBF_Follower). + * + * @return decode result with success flag and decoded entries + */ + public IBFDecodeResult decode() { + List entries = new ArrayList<>(); + Deque pureIndices = new ArrayDeque<>(); + + // Initial scan for pure cells + for (int i = 0; i < cellCount; i++) { + if (cells[i].isPure()) { + pureIndices.add(i); + } + } + + while (!pureIndices.isEmpty()) { + int idx = pureIndices.poll(); + IBFCell cell = cells[idx]; + + if (!cell.isPure()) { + continue; + } + + long key = cell.keySum; + long valueHash = cell.valueChecksum; + DiffEntry.DiffType type = + cell.count == 1 ? DiffEntry.DiffType.LEADER_HAS : DiffEntry.DiffType.FOLLOWER_HAS; + entries.add(new DiffEntry(key, valueHash, type)); + + // Peel this entry from all cells it hashes to + for (int i = 0; i < hashCount; i++) { + int cellIdx = hashToIndex(key, i); + if (cell.count == 1) { + cells[cellIdx].remove(key, valueHash); + } else { + cells[cellIdx].add(key, valueHash); + } + if (cells[cellIdx].isPure()) { + pureIndices.add(cellIdx); + } + } + } + + // Check if all cells are empty (complete decode) + boolean complete = true; + for (IBFCell cell : cells) { + if (!cell.isEmpty()) { + complete = false; + break; + } + } + + if (complete) { + return IBFDecodeResult.success(entries); + } else { + return IBFDecodeResult.failure(entries, entries.size()); + } + } + + /** + * Hash a composite key to a cell index for the i-th hash function, using multiplicative hashing + * with distinct seeds to minimize collisions. + */ + private int hashToIndex(long key, int hashFunctionIndex) { + long seed = SEEDS[hashFunctionIndex % SEEDS.length]; + if (hashFunctionIndex >= SEEDS.length) { + seed ^= hashFunctionIndex * 0xDEADBEEFL; + } + long hash = key * seed; + hash ^= hash >>> 33; + hash *= 0xFF51AFD7ED558CCDL; + hash ^= hash >>> 33; + return (int) ((hash & 0x7FFFFFFFFFFFFFFFL) % cellCount); + } + + public int getCellCount() { + return cellCount; + } + + public IBFCell[] getCells() { + return cells; + } + + /** Compute the optimal IBF size for an estimated diff count d with >99% decode probability. */ + public static int optimalCellCount(long estimatedDiffCount) { + return (int) Math.max(Math.ceil(2.0 * estimatedDiffCount), 3); + } + + public void serialize(DataOutputStream out) throws IOException { + out.writeInt(cellCount); + out.writeInt(hashCount); + for (IBFCell cell : cells) { + cell.serialize(out); + } + } + + public static InvertibleBloomFilter deserialize(ByteBuffer buffer) { + int cellCount = buffer.getInt(); + int hashCount = buffer.getInt(); + IBFCell[] cells = new IBFCell[cellCount]; + for (int i = 0; i < cellCount; i++) { + cells[i] = IBFCell.deserialize(buffer); + } + return new InvertibleBloomFilter(cells, hashCount); + } + + /** Total serialized byte size: 4 (cellCount) + 4 (hashCount) + cellCount * 20 bytes */ + public int serializedSize() { + return 8 + cellCount * IBFCell.SERIALIZED_SIZE; + } +} diff --git a/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/consensus/iotv2/consistency/ibf/RowRefIndex.java b/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/consensus/iotv2/consistency/ibf/RowRefIndex.java new file mode 100644 index 0000000000000..0d5fd3c90aea9 --- /dev/null +++ b/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/consensus/iotv2/consistency/ibf/RowRefIndex.java @@ -0,0 +1,160 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.iotdb.commons.consensus.iotv2.consistency.ibf; + +import java.util.ArrayList; +import java.util.Collections; +import java.util.List; +import java.util.TreeMap; + +/** + * Temporary in-memory index built during IBF construction that maps composite keys back to the + * original (deviceId, measurement, timestamp) coordinates. This is the critical bridge between IBF + * decoding (which recovers composite keys) and repair execution (which needs actual data point + * identifiers). + * + *

The index is built from the negotiated device/measurement mapping and discarded after repair. + */ +public class RowRefIndex { + + private final List deviceIdByIndex; + private final List> measurementsByDevice; + private final long timeBucketStart; + private final long timestampResolution; + + private RowRefIndex( + List deviceIdByIndex, + List> measurementsByDevice, + long timeBucketStart, + long timestampResolution) { + this.deviceIdByIndex = deviceIdByIndex; + this.measurementsByDevice = measurementsByDevice; + this.timeBucketStart = timeBucketStart; + this.timestampResolution = timestampResolution; + } + + /** + * Resolve a decoded composite key back to original data point coordinates. + * + * @param compositeKey the 8-byte composite key from IBF decoding + * @return the resolved data point locator + * @throws IndexOutOfBoundsException if indices are invalid + */ + public DataPointLocator resolve(long compositeKey) { + int deviceIdx = CompositeKeyCodec.extractDeviceIndex(compositeKey); + int measIdx = CompositeKeyCodec.extractMeasurementIndex(compositeKey); + long timestamp = + CompositeKeyCodec.recoverTimestamp(compositeKey, timeBucketStart, timestampResolution); + + return new DataPointLocator( + deviceIdByIndex.get(deviceIdx), + measurementsByDevice.get(deviceIdx).get(measIdx), + timestamp); + } + + public List getDeviceIdByIndex() { + return Collections.unmodifiableList(deviceIdByIndex); + } + + public List> getMeasurementsByDevice() { + return Collections.unmodifiableList(measurementsByDevice); + } + + public long getTimeBucketStart() { + return timeBucketStart; + } + + public long getTimestampResolution() { + return timestampResolution; + } + + /** Estimate heap memory consumed by this index. */ + public int estimatedMemoryBytes() { + int bytes = 64; // object overhead + for (String deviceId : deviceIdByIndex) { + bytes += 40 + deviceId.length() * 2; + } + for (List measurements : measurementsByDevice) { + bytes += 40; + for (String m : measurements) { + bytes += 40 + m.length() * 2; + } + } + return bytes; + } + + /** Builder for constructing RowRefIndex from device/measurement lists. */ + public static class Builder { + private final TreeMap> deviceMeasurements = new TreeMap<>(); + private long timeBucketStart = 0; + private long timestampResolution = CompositeKeyCodec.DEFAULT_TIMESTAMP_RESOLUTION; + + public Builder addDevice(String deviceId, List measurements) { + List sorted = new ArrayList<>(measurements); + Collections.sort(sorted); + deviceMeasurements.put(deviceId, sorted); + return this; + } + + public Builder setTimeBucketStart(long timeBucketStart) { + this.timeBucketStart = timeBucketStart; + return this; + } + + public Builder setTimestampResolution(long timestampResolution) { + this.timestampResolution = timestampResolution; + return this; + } + + /** + * Build the RowRefIndex and return a mapping from (deviceId, measurement) to their ordinal + * indices. + */ + public RowRefIndex build() { + List deviceList = new ArrayList<>(deviceMeasurements.keySet()); + List> measurementList = new ArrayList<>(); + for (String device : deviceList) { + measurementList.add(deviceMeasurements.get(device)); + } + return new RowRefIndex(deviceList, measurementList, timeBucketStart, timestampResolution); + } + + /** Get the device index for a given deviceId. Returns -1 if not found. */ + public int getDeviceIndex(String deviceId) { + int idx = 0; + for (String d : deviceMeasurements.keySet()) { + if (d.equals(deviceId)) { + return idx; + } + idx++; + } + return -1; + } + + /** Get the measurement index for a given device and measurement. Returns -1 if not found. */ + public int getMeasurementIndex(String deviceId, String measurement) { + List measurements = deviceMeasurements.get(deviceId); + if (measurements == null) { + return -1; + } + return Collections.binarySearch(measurements, measurement); + } + } +} diff --git a/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/consensus/iotv2/consistency/ibf/StrataEstimator.java b/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/consensus/iotv2/consistency/ibf/StrataEstimator.java new file mode 100644 index 0000000000000..82805b1ac8b88 --- /dev/null +++ b/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/consensus/iotv2/consistency/ibf/StrataEstimator.java @@ -0,0 +1,108 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.iotdb.commons.consensus.iotv2.consistency.ibf; + +import java.io.DataOutputStream; +import java.io.IOException; +import java.nio.ByteBuffer; + +/** + * Strata Estimator for estimating the number of differences between two sets before sizing the main + * IBF. Uses a stack of small IBFs at different strata levels, where each element is assigned to a + * stratum based on the number of trailing zeros in its key. Communication cost: ~50 KB. + * + *

Reference: Eppstein et al., "What's the Difference? Efficient Set Reconciliation without Prior + * Context" + */ +public class StrataEstimator { + + private static final int STRATA_COUNT = 32; + private static final int CELLS_PER_STRATUM = 80; + private static final int HASH_COUNT = 3; + + private final InvertibleBloomFilter[] strata; + + public StrataEstimator() { + strata = new InvertibleBloomFilter[STRATA_COUNT]; + for (int i = 0; i < STRATA_COUNT; i++) { + strata[i] = new InvertibleBloomFilter(CELLS_PER_STRATUM, HASH_COUNT); + } + } + + private StrataEstimator(InvertibleBloomFilter[] strata) { + this.strata = strata; + } + + /** Insert a (key, valueHash) pair into the appropriate stratum. */ + public void insert(long key, long valueHash) { + int level = Long.numberOfTrailingZeros(key); + if (level >= STRATA_COUNT) { + level = STRATA_COUNT - 1; + } + strata[level].insert(key, valueHash); + } + + /** + * Estimate the number of differences between this estimator and another. + * + * @param other the other side's strata estimator + * @return estimated diff count + */ + public long estimateDifference(StrataEstimator other) { + long estimate = 0; + + for (int i = STRATA_COUNT - 1; i >= 0; i--) { + InvertibleBloomFilter diff = this.strata[i].subtract(other.strata[i]); + IBFDecodeResult result = diff.decode(); + if (!result.isSuccess()) { + // If decode fails at this level, scale up the partial count + return (estimate + result.getPartialCount()) * (1L << (i + 1)); + } + estimate += result.getPartialCount(); + } + + return estimate; + } + + public void serialize(DataOutputStream out) throws IOException { + out.writeInt(STRATA_COUNT); + for (InvertibleBloomFilter ibf : strata) { + ibf.serialize(out); + } + } + + public static StrataEstimator deserialize(ByteBuffer buffer) { + int count = buffer.getInt(); + InvertibleBloomFilter[] strata = new InvertibleBloomFilter[count]; + for (int i = 0; i < count; i++) { + strata[i] = InvertibleBloomFilter.deserialize(buffer); + } + return new StrataEstimator(strata); + } + + /** Approximate serialized size: 4 + STRATA_COUNT * (8 + CELLS_PER_STRATUM * 20) ≈ 50 KB. */ + public int serializedSize() { + int size = 4; + for (InvertibleBloomFilter ibf : strata) { + size += ibf.serializedSize(); + } + return size; + } +} diff --git a/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/consensus/iotv2/consistency/merkle/MerkleEntry.java b/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/consensus/iotv2/consistency/merkle/MerkleEntry.java new file mode 100644 index 0000000000000..3c7b13725c03f --- /dev/null +++ b/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/consensus/iotv2/consistency/merkle/MerkleEntry.java @@ -0,0 +1,73 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.iotdb.commons.consensus.iotv2.consistency.merkle; + +/** + * A single entry in a .merkle file representing the hash of a (device, measurement, timeBucket) + * slice. + */ +public class MerkleEntry { + + private final String deviceId; + private final String measurement; + private final long timeBucketStart; + private final long timeBucketEnd; + private final int pointCount; + private final long entryHash; + + public MerkleEntry( + String deviceId, + String measurement, + long timeBucketStart, + long timeBucketEnd, + int pointCount, + long entryHash) { + this.deviceId = deviceId; + this.measurement = measurement; + this.timeBucketStart = timeBucketStart; + this.timeBucketEnd = timeBucketEnd; + this.pointCount = pointCount; + this.entryHash = entryHash; + } + + public String getDeviceId() { + return deviceId; + } + + public String getMeasurement() { + return measurement; + } + + public long getTimeBucketStart() { + return timeBucketStart; + } + + public long getTimeBucketEnd() { + return timeBucketEnd; + } + + public int getPointCount() { + return pointCount; + } + + public long getEntryHash() { + return entryHash; + } +} diff --git a/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/consensus/iotv2/consistency/merkle/MerkleFileCache.java b/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/consensus/iotv2/consistency/merkle/MerkleFileCache.java new file mode 100644 index 0000000000000..8c737e1b4fe5c --- /dev/null +++ b/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/consensus/iotv2/consistency/merkle/MerkleFileCache.java @@ -0,0 +1,124 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.iotdb.commons.consensus.iotv2.consistency.merkle; + +import java.io.IOException; +import java.util.LinkedHashMap; +import java.util.Map; +import java.util.concurrent.locks.ReentrantLock; + +/** + * LRU cache for on-demand loading of .merkle file contents. Level 2+ Merkle data is loaded from + * disk into this cache and evicted when memory pressure exceeds the configured maximum. + */ +public class MerkleFileCache { + + private final long maxMemoryBytes; + private volatile long currentMemoryBytes; + private final LinkedHashMap cache; + private final ReentrantLock lock; + + public MerkleFileCache(long maxMemoryBytes) { + this.maxMemoryBytes = maxMemoryBytes; + this.currentMemoryBytes = 0; + this.lock = new ReentrantLock(); + this.cache = + new LinkedHashMap(64, 0.75f, true) { + @Override + protected boolean removeEldestEntry(Map.Entry eldest) { + if (currentMemoryBytes > maxMemoryBytes) { + currentMemoryBytes -= eldest.getValue().estimatedMemoryBytes(); + return true; + } + return false; + } + }; + } + + /** + * Get the .merkle content for a TsFile. Loads from disk if not cached. + * + * @param tsFilePath path to the TsFile (the .merkle file is at tsFilePath + ".merkle") + * @return cached or freshly loaded content + */ + public MerkleFileContent get(String tsFilePath) throws IOException { + lock.lock(); + try { + MerkleFileContent content = cache.get(tsFilePath); + if (content != null) { + return content; + } + } finally { + lock.unlock(); + } + + MerkleFileContent loaded = MerkleFileReader.read(tsFilePath + ".merkle", tsFilePath); + + lock.lock(); + try { + MerkleFileContent existing = cache.get(tsFilePath); + if (existing != null) { + return existing; + } + cache.put(tsFilePath, loaded); + currentMemoryBytes += loaded.estimatedMemoryBytes(); + return loaded; + } finally { + lock.unlock(); + } + } + + /** Invalidate a specific entry, e.g., when its TsFile is compacted away. */ + public void invalidate(String tsFilePath) { + lock.lock(); + try { + MerkleFileContent removed = cache.remove(tsFilePath); + if (removed != null) { + currentMemoryBytes -= removed.estimatedMemoryBytes(); + } + } finally { + lock.unlock(); + } + } + + /** Clear all cached entries. */ + public void clear() { + lock.lock(); + try { + cache.clear(); + currentMemoryBytes = 0; + } finally { + lock.unlock(); + } + } + + public long getCurrentMemoryBytes() { + return currentMemoryBytes; + } + + public int size() { + lock.lock(); + try { + return cache.size(); + } finally { + lock.unlock(); + } + } +} diff --git a/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/consensus/iotv2/consistency/merkle/MerkleFileContent.java b/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/consensus/iotv2/consistency/merkle/MerkleFileContent.java new file mode 100644 index 0000000000000..98f31a28888c8 --- /dev/null +++ b/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/consensus/iotv2/consistency/merkle/MerkleFileContent.java @@ -0,0 +1,82 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.iotdb.commons.consensus.iotv2.consistency.merkle; + +import org.apache.iotdb.commons.consensus.iotv2.consistency.DualDigest; + +import java.util.Collections; +import java.util.List; + +/** + * In-memory representation of a parsed .merkle file. Holds the file-level dual-digest and all + * per-(device, measurement, timeBucket) entries. + */ +public class MerkleFileContent { + + private final long fileXorHash; + private final long fileAddHash; + private final List entries; + private final String sourceTsFilePath; + + public MerkleFileContent( + long fileXorHash, long fileAddHash, List entries, String sourceTsFilePath) { + this.fileXorHash = fileXorHash; + this.fileAddHash = fileAddHash; + this.entries = Collections.unmodifiableList(entries); + this.sourceTsFilePath = sourceTsFilePath; + } + + public long getFileXorHash() { + return fileXorHash; + } + + public long getFileAddHash() { + return fileAddHash; + } + + public DualDigest getFileDigest() { + return new DualDigest(fileXorHash, fileAddHash); + } + + public List getEntries() { + return entries; + } + + public String getSourceTsFilePath() { + return sourceTsFilePath; + } + + public int getTotalPointCount() { + int total = 0; + for (MerkleEntry entry : entries) { + total += entry.getPointCount(); + } + return total; + } + + /** Estimate heap memory consumed by this content for cache weighing. */ + public int estimatedMemoryBytes() { + int base = 64; + for (MerkleEntry entry : entries) { + base += 80 + entry.getDeviceId().length() * 2 + entry.getMeasurement().length() * 2; + } + return base; + } +} diff --git a/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/consensus/iotv2/consistency/merkle/MerkleFileReader.java b/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/consensus/iotv2/consistency/merkle/MerkleFileReader.java new file mode 100644 index 0000000000000..579e709396659 --- /dev/null +++ b/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/consensus/iotv2/consistency/merkle/MerkleFileReader.java @@ -0,0 +1,148 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.iotdb.commons.consensus.iotv2.consistency.merkle; + +import java.io.BufferedInputStream; +import java.io.DataInputStream; +import java.io.FileInputStream; +import java.io.IOException; +import java.io.InputStream; +import java.nio.charset.StandardCharsets; +import java.util.ArrayList; +import java.util.List; +import java.util.zip.CRC32; + +/** Reads and validates a .merkle sidecar file. Supports version 2 format with dual-digest. */ +public class MerkleFileReader { + + private static final byte[] MAGIC = {'M', 'R', 'K', 'L'}; + + private MerkleFileReader() {} + + /** + * Read a .merkle file and return its in-memory content. + * + * @param merklePath full path to the .merkle file + * @param sourceTsFilePath the associated TsFile path (for attribution) + * @return parsed content, or throws on corruption + */ + public static MerkleFileContent read(String merklePath, String sourceTsFilePath) + throws IOException { + CRC32 crc32 = new CRC32(); + try (InputStream fis = new FileInputStream(merklePath); + BufferedInputStream bis = new BufferedInputStream(fis); + CrcInputStream crcStream = new CrcInputStream(bis, crc32); + DataInputStream in = new DataInputStream(crcStream)) { + + // Header: magic (4), version (2), flags (2), fileXorHash (8), fileAddHash (8), entryCount + // (4) + byte[] magic = new byte[4]; + in.readFully(magic); + if (magic[0] != MAGIC[0] + || magic[1] != MAGIC[1] + || magic[2] != MAGIC[2] + || magic[3] != MAGIC[3]) { + throw new IOException("Invalid .merkle file magic: " + merklePath); + } + + short version = in.readShort(); + if (version < 2) { + throw new IOException("Unsupported .merkle version: " + version + " at " + merklePath); + } + in.readShort(); // flags, reserved + + long fileXorHash = in.readLong(); + long fileAddHash = in.readLong(); + int entryCount = in.readInt(); + + List entries = new ArrayList<>(entryCount); + for (int i = 0; i < entryCount; i++) { + short deviceIdLen = in.readShort(); + byte[] deviceIdBytes = new byte[deviceIdLen]; + in.readFully(deviceIdBytes); + String deviceId = new String(deviceIdBytes, StandardCharsets.UTF_8); + + short measurementLen = in.readShort(); + byte[] measurementBytes = new byte[measurementLen]; + in.readFully(measurementBytes); + String measurement = new String(measurementBytes, StandardCharsets.UTF_8); + + long timeBucketStart = in.readLong(); + long timeBucketEnd = in.readLong(); + int pointCount = in.readInt(); + long entryHash = in.readLong(); + + entries.add( + new MerkleEntry( + deviceId, measurement, timeBucketStart, timeBucketEnd, pointCount, entryHash)); + } + + // Footer: CRC32 validation -- capture CRC before reading the footer + int computedCrc = (int) crc32.getValue(); + // Read the stored CRC via DataInputStream (this will update crc32, but we already saved it) + int storedCrc = in.readInt(); + if (computedCrc != storedCrc) { + throw new IOException( + String.format( + "CRC mismatch in .merkle file %s: expected 0x%08X, got 0x%08X", + merklePath, storedCrc, computedCrc)); + } + + return new MerkleFileContent(fileXorHash, fileAddHash, entries, sourceTsFilePath); + } + } + + /** + * An InputStream wrapper that updates a CRC32 on every byte read. DataInputStream is then + * constructed on top of this, ensuring all reads flow through the CRC update. + */ + private static class CrcInputStream extends InputStream { + private final InputStream delegate; + private final CRC32 crc; + + CrcInputStream(InputStream delegate, CRC32 crc) { + this.delegate = delegate; + this.crc = crc; + } + + @Override + public int read() throws IOException { + int b = delegate.read(); + if (b >= 0) { + crc.update(b); + } + return b; + } + + @Override + public int read(byte[] buf, int off, int len) throws IOException { + int n = delegate.read(buf, off, len); + if (n > 0) { + crc.update(buf, off, n); + } + return n; + } + + @Override + public void close() throws IOException { + delegate.close(); + } + } +} diff --git a/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/consensus/iotv2/consistency/merkle/MerkleFileWriter.java b/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/consensus/iotv2/consistency/merkle/MerkleFileWriter.java new file mode 100644 index 0000000000000..74328ccb331a7 --- /dev/null +++ b/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/consensus/iotv2/consistency/merkle/MerkleFileWriter.java @@ -0,0 +1,136 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.iotdb.commons.consensus.iotv2.consistency.merkle; + +import java.io.BufferedOutputStream; +import java.io.DataOutputStream; +import java.io.FileOutputStream; +import java.io.IOException; +import java.io.OutputStream; +import java.nio.charset.StandardCharsets; +import java.util.List; +import java.util.zip.CRC32; + +/** + * Writes a .merkle sidecar file alongside a TsFile. Format version 2 supports dual-digest + * (fileXorHash + fileAddHash) in the header. + */ +public class MerkleFileWriter { + + private static final byte[] MAGIC = {'M', 'R', 'K', 'L'}; + private static final short VERSION = 2; + private static final short FLAGS = 0; + + private MerkleFileWriter() {} + + /** + * Write a .merkle file at the specified path. + * + * @param merklePath full path for the .merkle file (typically tsFilePath + ".merkle") + * @param entries all (device, measurement, timeBucket) hash entries sorted by device -> meas -> + * timeBucketStart + * @param fileXorHash XOR of all entry hashes + * @param fileAddHash SUM of all entry hashes (mod 2^64) + */ + public static void write( + String merklePath, List entries, long fileXorHash, long fileAddHash) + throws IOException { + CRC32 crc32 = new CRC32(); + try (OutputStream fos = new FileOutputStream(merklePath); + BufferedOutputStream bos = new BufferedOutputStream(fos); + CrcDataOutputStream out = new CrcDataOutputStream(bos, crc32)) { + + // Header (28 bytes) + out.write(MAGIC); + out.writeShort(VERSION); + out.writeShort(FLAGS); + out.writeLong(fileXorHash); + out.writeLong(fileAddHash); + out.writeInt(entries.size()); + + // Entries + for (MerkleEntry entry : entries) { + byte[] deviceBytes = entry.getDeviceId().getBytes(StandardCharsets.UTF_8); + out.writeShort(deviceBytes.length); + out.write(deviceBytes); + + byte[] measurementBytes = entry.getMeasurement().getBytes(StandardCharsets.UTF_8); + out.writeShort(measurementBytes.length); + out.write(measurementBytes); + + out.writeLong(entry.getTimeBucketStart()); + out.writeLong(entry.getTimeBucketEnd()); + out.writeInt(entry.getPointCount()); + out.writeLong(entry.getEntryHash()); + } + + // Footer: CRC32 checksum + out.flush(); + int crc = (int) crc32.getValue(); + // Write CRC without updating the CRC itself + bos.write((crc >>> 24) & 0xFF); + bos.write((crc >>> 16) & 0xFF); + bos.write((crc >>> 8) & 0xFF); + bos.write(crc & 0xFF); + } + } + + /** Computes the file-level dual-digest from a list of entries. */ + public static long computeFileXorHash(List entries) { + long xor = 0; + for (MerkleEntry entry : entries) { + xor ^= entry.getEntryHash(); + } + return xor; + } + + public static long computeFileAddHash(List entries) { + long add = 0; + for (MerkleEntry entry : entries) { + add += entry.getEntryHash(); + } + return add; + } + + /** + * A DataOutputStream wrapper that updates a CRC32 on every byte written, so we can compute the + * checksum in a single pass. + */ + private static class CrcDataOutputStream extends DataOutputStream { + private final CRC32 crc; + + CrcDataOutputStream(OutputStream out, CRC32 crc) { + super(out); + this.crc = crc; + } + + @Override + public synchronized void write(int b) throws IOException { + super.write(b); + crc.update(b); + } + + @Override + public synchronized void write(byte[] buf, int off, int len) throws IOException { + super.write(buf, off, len); + crc.update(buf, off, len); + } + } +} diff --git a/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/consensus/iotv2/consistency/repair/AdaptiveRepairExecutor.java b/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/consensus/iotv2/consistency/repair/AdaptiveRepairExecutor.java new file mode 100644 index 0000000000000..e3d72cb0d92f5 --- /dev/null +++ b/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/consensus/iotv2/consistency/repair/AdaptiveRepairExecutor.java @@ -0,0 +1,204 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.iotdb.commons.consensus.iotv2.consistency.repair; + +import org.apache.iotdb.commons.consensus.iotv2.consistency.ibf.DiffEntry; +import org.apache.iotdb.commons.consensus.iotv2.consistency.ibf.RowRefIndex; +import org.apache.iotdb.commons.consensus.iotv2.consistency.merkle.MerkleFileContent; + +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.util.ArrayList; +import java.util.HashMap; +import java.util.List; +import java.util.Map; + +/** + * Orchestrates the adaptive repair execution phase, combining direct TsFile transfers (for dense + * diffs or small TsFiles) with point-level streaming (for sparse diffs in large TsFiles). + * + *

The executor follows these steps: + * + *

    + *
  1. Attribute decoded diffs to source TsFiles via DiffAttribution + *
  2. For each TsFile, use RepairCostModel to select the optimal strategy + *
  3. Execute TsFile transfers (zero packaging cost, idempotent loading) + *
  4. Execute point streaming via RepairSession with atomic promote + *
+ */ +public class AdaptiveRepairExecutor { + + private static final Logger LOGGER = LoggerFactory.getLogger(AdaptiveRepairExecutor.class); + + private final RepairCostModel costModel; + private final DiffAttribution attribution; + + /** Callback interface for executing actual repair operations. */ + public interface RepairOperationCallback { + /** Transfer an existing TsFile from Leader to Follower. */ + void transferTsFile(String tsFilePath) throws Exception; + + /** Get the size of a TsFile. */ + long getTsFileSize(String tsFilePath); + + /** Get the total point count from a TsFile's .merkle content. */ + int getTotalPointCount(String tsFilePath); + + /** Build a concrete repair record for point streaming. */ + default RepairRecord buildRepairRecord(DiffEntry diffEntry, RowRefIndex rowRefIndex) { + return null; + } + } + + public AdaptiveRepairExecutor() { + this(new RepairCostModel(), new DiffAttribution()); + } + + public AdaptiveRepairExecutor(RepairCostModel costModel, DiffAttribution attribution) { + this.costModel = costModel; + this.attribution = attribution; + } + + /** + * Build repair plans for all attributed TsFiles. + * + * @param decodedDiffs the decoded diff entries + * @param rowRefIndex the composite key resolver + * @param leaderMerkleFiles Leader's .merkle files for the checked range + * @param callback for querying TsFile metadata + * @return map of TsFile path to RepairPlan + */ + public Map buildRepairPlans( + List decodedDiffs, + RowRefIndex rowRefIndex, + List leaderMerkleFiles, + RepairOperationCallback callback) { + + // Step 1: Attribute diffs to source TsFiles + Map> attributedDiffs = + attribution.attributeToSourceTsFiles(decodedDiffs, rowRefIndex, leaderMerkleFiles); + + // Step 2: Select strategy per TsFile + Map plans = new HashMap<>(); + for (Map.Entry> entry : attributedDiffs.entrySet()) { + String tsFilePath = entry.getKey(); + List diffs = entry.getValue(); + + long tsFileSize = callback.getTsFileSize(tsFilePath); + int totalPoints = callback.getTotalPointCount(tsFilePath); + + RepairStrategy strategy = costModel.selectStrategy(tsFileSize, totalPoints, diffs.size()); + + if (strategy == RepairStrategy.DIRECT_TSFILE_TRANSFER) { + plans.put(tsFilePath, RepairPlan.directTransfer(tsFilePath, tsFileSize)); + } else { + plans.put(tsFilePath, RepairPlan.pointStreaming(tsFilePath, tsFileSize, diffs)); + } + } + + return plans; + } + + /** + * Execute all repair plans. + * + * @param plans the repair plans (per TsFile) + * @param rowRefIndex for resolving composite keys during point streaming + * @param session the RepairSession for staging point-level repairs + * @param callback for executing actual transfer operations + * @return true if all operations succeeded + */ + public boolean executeRepairPlans( + Map plans, + RowRefIndex rowRefIndex, + RepairSession session, + RepairOperationCallback callback) { + boolean allSucceeded = true; + + // Phase 1: Direct TsFile transfers (zero packaging cost) + List transferPlans = new ArrayList<>(); + List streamingPlans = new ArrayList<>(); + + for (RepairPlan plan : plans.values()) { + if (plan.getStrategy() == RepairStrategy.DIRECT_TSFILE_TRANSFER) { + transferPlans.add(plan); + } else { + streamingPlans.add(plan); + } + } + + LOGGER.info( + "Executing repair: {} TsFile transfers, {} point streaming plans", + transferPlans.size(), + streamingPlans.size()); + + for (RepairPlan plan : transferPlans) { + try { + callback.transferTsFile(plan.getTsFilePath()); + LOGGER.debug("Transferred TsFile: {}", plan.getTsFilePath()); + } catch (Exception e) { + LOGGER.error("Failed to transfer TsFile {}: {}", plan.getTsFilePath(), e.getMessage(), e); + allSucceeded = false; + } + } + + // Phase 2: Point-level streaming with atomic promote + if (!streamingPlans.isEmpty()) { + for (RepairPlan plan : streamingPlans) { + for (DiffEntry diff : plan.getDiffs()) { + RepairRecord record = callback.buildRepairRecord(diff, rowRefIndex); + if (record != null) { + session.stage(record); + LOGGER.debug("Staged diff entry for streaming: {}", diff); + } + } + } + + if (session.getStagedCount() > 0) { + boolean promoted = session.promoteAtomically(); + if (!promoted) { + LOGGER.error("Failed to atomically promote repair session"); + allSucceeded = false; + } + } + } + + return allSucceeded; + } + + /** + * Build repair plans for full-range fallback (when IBF decode fails). Transfers all overlapping + * Leader TsFiles for the affected range. + * + * @param overlappingTsFilePaths paths of Leader TsFiles overlapping the affected range + * @param callback for querying TsFile sizes + * @return list of direct transfer plans + */ + public List buildFullRangeFallbackPlans( + List overlappingTsFilePaths, RepairOperationCallback callback) { + List plans = new ArrayList<>(); + for (String tsFilePath : overlappingTsFilePaths) { + long tsFileSize = callback.getTsFileSize(tsFilePath); + plans.add(RepairPlan.directTransfer(tsFilePath, tsFileSize)); + } + return plans; + } +} diff --git a/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/consensus/iotv2/consistency/repair/DiffAttribution.java b/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/consensus/iotv2/consistency/repair/DiffAttribution.java new file mode 100644 index 0000000000000..8ea72d31c8ee4 --- /dev/null +++ b/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/consensus/iotv2/consistency/repair/DiffAttribution.java @@ -0,0 +1,111 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.iotdb.commons.consensus.iotv2.consistency.repair; + +import org.apache.iotdb.commons.consensus.iotv2.consistency.ibf.DataPointLocator; +import org.apache.iotdb.commons.consensus.iotv2.consistency.ibf.DiffEntry; +import org.apache.iotdb.commons.consensus.iotv2.consistency.ibf.RowRefIndex; +import org.apache.iotdb.commons.consensus.iotv2.consistency.merkle.MerkleEntry; +import org.apache.iotdb.commons.consensus.iotv2.consistency.merkle.MerkleFileContent; + +import java.util.ArrayList; +import java.util.HashMap; +import java.util.List; +import java.util.Map; + +/** + * Maps decoded IBF diff entries back to their source TsFiles on the Leader. Each diff point is + * attributed to the TsFile whose .merkle file contains a matching (device, measurement, timeBucket) + * entry. This attribution enables per-TsFile repair strategy selection. + */ +public class DiffAttribution { + + /** + * Attribute decoded diffs to their source TsFiles on the Leader. + * + * @param decodedDiffs the decoded diff entries from IBF + * @param rowRefIndex the index for resolving composite keys to data point locators + * @param leaderMerkleFiles .merkle file contents loaded from the Leader + * @return map from TsFile path to list of attributed diff entries + */ + public Map> attributeToSourceTsFiles( + List decodedDiffs, + RowRefIndex rowRefIndex, + List leaderMerkleFiles) { + Map>> bucketIndex = new HashMap<>(); + for (MerkleFileContent content : leaderMerkleFiles) { + for (MerkleEntry entry : content.getEntries()) { + bucketIndex + .computeIfAbsent(entry.getDeviceId(), ignored -> new HashMap<>()) + .computeIfAbsent(entry.getMeasurement(), ignored -> new ArrayList<>()) + .add( + new BucketOwner( + entry.getTimeBucketStart(), + entry.getTimeBucketEnd(), + content.getSourceTsFilePath())); + } + } + + Map> attribution = new HashMap<>(); + for (DiffEntry diff : decodedDiffs) { + DataPointLocator loc = rowRefIndex.resolve(diff.getCompositeKey()); + + String sourceTsFile = findSourceTsFile(loc, bucketIndex); + if (sourceTsFile != null) { + attribution.computeIfAbsent(sourceTsFile, k -> new ArrayList<>()).add(diff); + } + } + + return attribution; + } + + private String findSourceTsFile( + DataPointLocator loc, Map>> bucketIndex) { + List owners = + bucketIndex + .getOrDefault(loc.getDeviceId(), java.util.Collections.emptyMap()) + .get(loc.getMeasurement()); + if (owners == null) { + return null; + } + for (BucketOwner owner : owners) { + if (owner.contains(loc.getTimestamp())) { + return owner.sourceTsFilePath; + } + } + return null; + } + + private static final class BucketOwner { + private final long bucketStart; + private final long bucketEnd; + private final String sourceTsFilePath; + + private BucketOwner(long bucketStart, long bucketEnd, String sourceTsFilePath) { + this.bucketStart = bucketStart; + this.bucketEnd = bucketEnd; + this.sourceTsFilePath = sourceTsFilePath; + } + + private boolean contains(long timestamp) { + return timestamp >= bucketStart && timestamp < bucketEnd; + } + } +} diff --git a/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/consensus/iotv2/consistency/repair/ModEntrySummary.java b/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/consensus/iotv2/consistency/repair/ModEntrySummary.java new file mode 100644 index 0000000000000..95bfbee1ed312 --- /dev/null +++ b/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/consensus/iotv2/consistency/repair/ModEntrySummary.java @@ -0,0 +1,97 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.iotdb.commons.consensus.iotv2.consistency.repair; + +/** + * Summary of a mod file entry (deletion) exchanged during the NEGOTIATE_KEY_MAPPING phase. Contains + * enough information to determine if a deletion covers a specific data point, along with the + * ProgressIndex for causal ordering. + */ +public class ModEntrySummary { + + private final String devicePattern; + private final String measurementPattern; + private final long timeRangeStart; + private final long timeRangeEnd; + private final long progressIndex; + + public ModEntrySummary( + String devicePattern, + String measurementPattern, + long timeRangeStart, + long timeRangeEnd, + long progressIndex) { + this.devicePattern = devicePattern; + this.measurementPattern = measurementPattern; + this.timeRangeStart = timeRangeStart; + this.timeRangeEnd = timeRangeEnd; + this.progressIndex = progressIndex; + } + + /** + * Check if this deletion covers a specific data point. + * + * @param deviceId the device ID of the data point + * @param measurement the measurement name + * @param timestamp the timestamp of the data point + * @return true if this deletion covers the specified data point + */ + public boolean covers(String deviceId, String measurement, long timestamp) { + if (timestamp < timeRangeStart || timestamp > timeRangeEnd) { + return false; + } + return matchesPattern(devicePattern, deviceId) + && matchesPattern(measurementPattern, measurement); + } + + private boolean matchesPattern(String pattern, String value) { + if (pattern == null || pattern.equals("*") || pattern.equals("**")) { + return true; + } + return pattern.equals(value); + } + + public String getDevicePattern() { + return devicePattern; + } + + public String getMeasurementPattern() { + return measurementPattern; + } + + public long getTimeRangeStart() { + return timeRangeStart; + } + + public long getTimeRangeEnd() { + return timeRangeEnd; + } + + public long getProgressIndex() { + return progressIndex; + } + + @Override + public String toString() { + return String.format( + "ModEntrySummary{device=%s, meas=%s, range=[%d,%d], pi=%d}", + devicePattern, measurementPattern, timeRangeStart, timeRangeEnd, progressIndex); + } +} diff --git a/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/consensus/iotv2/consistency/repair/RepairAction.java b/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/consensus/iotv2/consistency/repair/RepairAction.java new file mode 100644 index 0000000000000..b997733f9010e --- /dev/null +++ b/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/consensus/iotv2/consistency/repair/RepairAction.java @@ -0,0 +1,38 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.iotdb.commons.consensus.iotv2.consistency.repair; + +/** Actions that the RepairConflictResolver can determine for a single diff entry. */ +public enum RepairAction { + /** Send the Leader's data to the Follower (insert/update). */ + SEND_TO_FOLLOWER, + /** Send the Follower's data to the Leader (insert/update). */ + SEND_TO_LEADER, + /** Delete the data on the Follower. */ + DELETE_ON_FOLLOWER, + /** Delete the data on the Leader. */ + DELETE_ON_LEADER, + /** Keep the Follower's version (it's newer or more authoritative). */ + KEEP_FOLLOWER, + /** Skip this entry (no action needed, e.g., Follower's deletion is newer). */ + SKIP, + /** Skip and raise an alert (anomalous state, e.g., data exists on Follower with no deletion). */ + SKIP_AND_ALERT +} diff --git a/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/consensus/iotv2/consistency/repair/RepairConflictResolver.java b/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/consensus/iotv2/consistency/repair/RepairConflictResolver.java new file mode 100644 index 0000000000000..a99bc8ebe4aba --- /dev/null +++ b/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/consensus/iotv2/consistency/repair/RepairConflictResolver.java @@ -0,0 +1,145 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.iotdb.commons.consensus.iotv2.consistency.repair; + +import org.apache.iotdb.commons.consensus.iotv2.consistency.ibf.DataPointLocator; + +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.util.Collections; +import java.util.List; +import java.util.Optional; + +/** + * ProgressIndex-based conflict resolver for all repair decisions. Replaces the unsafe "Leader wins" + * heuristic with causal ordering via ProgressIndex comparison. This is critical for preventing + * deletion resurrection and handling leadership transfers correctly. + * + *

Diff Interpretation Matrix: + * + * + * + * + * + * + * + * + * + *
Diff TypeLeader StateFollower StateResolution
LEADER_HASPoint exists, PI=XMissingIf no Follower deletion covers it: SEND_TO_FOLLOWER. If Follower deletion PI_del > X: + * DELETE_ON_LEADER
FOLLOWER_HASMissingPoint exists, PI=YIf Leader deletion PI_del > Y: DELETE_ON_FOLLOWER. Otherwise: SEND_TO_LEADER
VALUE_DIFFPI=X, V=VlPI=Y, V=VfHigher ProgressIndex wins and repairs the stale replica
+ */ +public class RepairConflictResolver { + + private static final Logger LOGGER = LoggerFactory.getLogger(RepairConflictResolver.class); + + private final List leaderDeletions; + private final List followerDeletions; + + public RepairConflictResolver( + List leaderDeletions, List followerDeletions) { + this.leaderDeletions = leaderDeletions != null ? leaderDeletions : Collections.emptyList(); + this.followerDeletions = + followerDeletions != null ? followerDeletions : Collections.emptyList(); + } + + /** + * Resolve a LEADER_HAS diff: Leader has the point but Follower doesn't. + * + * @param loc the data point location + * @param pointProgressIndex the ProgressIndex of the TsFile containing this point on the Leader + * @return the repair action to take + */ + public RepairAction resolveLeaderHas(DataPointLocator loc, long pointProgressIndex) { + Optional followerDel = findCoveringDeletion(followerDeletions, loc); + if (followerDel.isPresent() && followerDel.get().getProgressIndex() > pointProgressIndex) { + // Follower's deletion is more recent than the Leader's write -- repair the stale Leader. + LOGGER.debug( + "Deleting stale Leader point for {}: Follower deletion PI={} > point PI={}", + loc, + followerDel.get().getProgressIndex(), + pointProgressIndex); + return RepairAction.DELETE_ON_LEADER; + } + return RepairAction.SEND_TO_FOLLOWER; + } + + /** + * Resolve a FOLLOWER_HAS diff: Follower has the point but Leader doesn't. + * + * @param loc the data point location + * @param pointProgressIndex the ProgressIndex of the data on the Follower + * @return the repair action to take + */ + public RepairAction resolveFollowerHas(DataPointLocator loc, long pointProgressIndex) { + Optional leaderDel = findCoveringDeletion(leaderDeletions, loc); + if (leaderDel.isPresent() && leaderDel.get().getProgressIndex() > pointProgressIndex) { + // Leader's deletion is more recent than the Follower's write -- delete on Follower + return RepairAction.DELETE_ON_FOLLOWER; + } + + // The point can legitimately originate from a previous leadership epoch on the Follower. + return RepairAction.SEND_TO_LEADER; + } + + /** + * Resolve a VALUE_DIFF: both sides have the point but with different values. + * + * @param loc the data point location + * @param leaderProgressIndex the Leader's ProgressIndex for this point + * @param followerProgressIndex the Follower's ProgressIndex for this point + * @return the repair action to take + */ + public RepairAction resolveValueDiff( + DataPointLocator loc, long leaderProgressIndex, long followerProgressIndex) { + if (leaderProgressIndex > followerProgressIndex) { + return RepairAction.SEND_TO_FOLLOWER; + } else if (followerProgressIndex > leaderProgressIndex) { + return RepairAction.SEND_TO_LEADER; + } else { + // Concurrent writes (same ProgressIndex) -- Leader wins as tiebreaker + LOGGER.debug( + "Concurrent writes for {} with equal PI={}, using Leader as tiebreaker", + loc, + leaderProgressIndex); + return RepairAction.SEND_TO_FOLLOWER; + } + } + + /** + * Find a deletion entry that covers the given data point location. + * + * @param deletions the list of deletion summaries to search + * @param loc the data point to check + * @return the covering deletion with the highest ProgressIndex, if any + */ + static Optional findCoveringDeletion( + List deletions, DataPointLocator loc) { + ModEntrySummary bestMatch = null; + for (ModEntrySummary del : deletions) { + if (del.covers(loc.getDeviceId(), loc.getMeasurement(), loc.getTimestamp())) { + if (bestMatch == null || del.getProgressIndex() > bestMatch.getProgressIndex()) { + bestMatch = del; + } + } + } + return Optional.ofNullable(bestMatch); + } +} diff --git a/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/consensus/iotv2/consistency/repair/RepairCostModel.java b/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/consensus/iotv2/consistency/repair/RepairCostModel.java new file mode 100644 index 0000000000000..08bebc0103f0c --- /dev/null +++ b/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/consensus/iotv2/consistency/repair/RepairCostModel.java @@ -0,0 +1,119 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.iotdb.commons.consensus.iotv2.consistency.repair; + +/** + * Cost-based strategy selector for repair execution. For each TsFile with attributed diffs, + * computes whether direct TsFile transfer or point-level streaming is more efficient. + * + *

Decision matrix: + * + *

+ *                      TsFile Small (<16MB)    TsFile Medium          TsFile Large (>256MB)
+ * Diff Sparse  (<1%)   Transfer (cheap)        Stream points          Stream points
+ * Diff Medium (1-5%)   Transfer (cheap)        Cost-model decides     Stream points
+ * Diff Dense   (>5%)   Transfer (cheap)        Transfer TsFile        Transfer TsFile
+ * 
+ */ +public class RepairCostModel { + + private static final double DEFAULT_COST_PER_POINT_QUERY = 0.1; + private static final double DEFAULT_COST_PER_POINT_STREAM = 0.05; + private static final double DEFAULT_COST_PER_BYTE_TRANSFER = 0.001; + private static final double DEFAULT_DIFF_DENSITY_THRESHOLD = 0.05; + private static final long DEFAULT_SMALL_TSFILE_THRESHOLD = 16L * 1024 * 1024; + + private final double costPerPointQuery; + private final double costPerPointStream; + private final double costPerByteTransfer; + private final double diffDensityThreshold; + private final long smallTsFileThreshold; + + public RepairCostModel() { + this( + DEFAULT_COST_PER_POINT_QUERY, + DEFAULT_COST_PER_POINT_STREAM, + DEFAULT_COST_PER_BYTE_TRANSFER, + DEFAULT_DIFF_DENSITY_THRESHOLD, + DEFAULT_SMALL_TSFILE_THRESHOLD); + } + + public RepairCostModel( + double costPerPointQuery, + double costPerPointStream, + double costPerByteTransfer, + double diffDensityThreshold, + long smallTsFileThreshold) { + this.costPerPointQuery = costPerPointQuery; + this.costPerPointStream = costPerPointStream; + this.costPerByteTransfer = costPerByteTransfer; + this.diffDensityThreshold = diffDensityThreshold; + this.smallTsFileThreshold = smallTsFileThreshold; + } + + /** + * Select the optimal repair strategy for a TsFile with known diff characteristics. + * + * @param tsFileSize size of the TsFile in bytes + * @param totalPointCount estimated total point count in the TsFile + * @param diffPointCount number of diff points attributed to this TsFile + * @return the selected repair strategy + */ + public RepairStrategy selectStrategy(long tsFileSize, int totalPointCount, int diffPointCount) { + // Small TsFile: always transfer (cheaper than IBF + stream overhead) + if (tsFileSize < smallTsFileThreshold) { + return RepairStrategy.DIRECT_TSFILE_TRANSFER; + } + + // High diff density: transfer whole TsFile + double diffRatio = (double) diffPointCount / Math.max(totalPointCount, 1); + if (diffRatio > diffDensityThreshold) { + return RepairStrategy.DIRECT_TSFILE_TRANSFER; + } + + // Compare costs + double streamingCost = diffPointCount * (costPerPointQuery + costPerPointStream); + double transferCost = tsFileSize * costPerByteTransfer; + + if (transferCost < streamingCost) { + return RepairStrategy.DIRECT_TSFILE_TRANSFER; + } + + return RepairStrategy.POINT_STREAMING; + } + + /** + * Determine if a TsFile is small enough to skip IBF entirely (Merkle-level short circuit). + * + * @param tsFileSize size of the TsFile in bytes + * @return true if the TsFile should be directly transferred without IBF + */ + public boolean shouldBypassIBF(long tsFileSize) { + return tsFileSize < smallTsFileThreshold; + } + + public long getSmallTsFileThreshold() { + return smallTsFileThreshold; + } + + public double getDiffDensityThreshold() { + return diffDensityThreshold; + } +} diff --git a/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/consensus/iotv2/consistency/repair/RepairPlan.java b/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/consensus/iotv2/consistency/repair/RepairPlan.java new file mode 100644 index 0000000000000..9337d64b3508f --- /dev/null +++ b/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/consensus/iotv2/consistency/repair/RepairPlan.java @@ -0,0 +1,80 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.iotdb.commons.consensus.iotv2.consistency.repair; + +import org.apache.iotdb.commons.consensus.iotv2.consistency.ibf.DiffEntry; + +import java.util.ArrayList; +import java.util.List; + +/** + * Repair plan for a single TsFile, containing the selected strategy and the attributed diff + * entries. + */ +public class RepairPlan { + + private final String tsFilePath; + private final long tsFileSize; + private final RepairStrategy strategy; + private final List diffs; + + public RepairPlan( + String tsFilePath, long tsFileSize, RepairStrategy strategy, List diffs) { + this.tsFilePath = tsFilePath; + this.tsFileSize = tsFileSize; + this.strategy = strategy; + this.diffs = new ArrayList<>(diffs); + } + + /** Create a plan for direct TsFile transfer (no individual diff entries needed). */ + public static RepairPlan directTransfer(String tsFilePath, long tsFileSize) { + return new RepairPlan( + tsFilePath, tsFileSize, RepairStrategy.DIRECT_TSFILE_TRANSFER, new ArrayList<>()); + } + + /** Create a plan for point streaming with specific diff entries. */ + public static RepairPlan pointStreaming( + String tsFilePath, long tsFileSize, List diffs) { + return new RepairPlan(tsFilePath, tsFileSize, RepairStrategy.POINT_STREAMING, diffs); + } + + public String getTsFilePath() { + return tsFilePath; + } + + public long getTsFileSize() { + return tsFileSize; + } + + public RepairStrategy getStrategy() { + return strategy; + } + + public List getDiffs() { + return diffs; + } + + @Override + public String toString() { + return String.format( + "RepairPlan{file=%s, size=%d, strategy=%s, diffs=%d}", + tsFilePath, tsFileSize, strategy, diffs.size()); + } +} diff --git a/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/consensus/iotv2/consistency/repair/RepairRecord.java b/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/consensus/iotv2/consistency/repair/RepairRecord.java new file mode 100644 index 0000000000000..9cac62bfce38d --- /dev/null +++ b/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/consensus/iotv2/consistency/repair/RepairRecord.java @@ -0,0 +1,132 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.iotdb.commons.consensus.iotv2.consistency.repair; + +import org.apache.iotdb.commons.consensus.iotv2.consistency.ibf.DataPointLocator; + +/** + * A single repair record representing either an insert or delete operation to be applied during + * point-level streaming repair. Each record carries the ProgressIndex from its source for conflict + * resolution. + */ +public class RepairRecord { + + public enum RecordType { + INSERT, + DELETE + } + + public enum TargetReplica { + LEADER, + FOLLOWER + } + + private final RecordType type; + private final TargetReplica targetReplica; + private final DataPointLocator locator; + private final long progressIndex; + private final Object value; + private final long timestamp; + + private RepairRecord( + RecordType type, + TargetReplica targetReplica, + DataPointLocator locator, + long progressIndex, + Object value, + long timestamp) { + this.type = type; + this.targetReplica = targetReplica; + this.locator = locator; + this.progressIndex = progressIndex; + this.value = value; + this.timestamp = timestamp; + } + + /** Create an insert repair record. */ + public static RepairRecord insert( + DataPointLocator locator, long progressIndex, Object value, long timestamp) { + return insertToFollower(locator, progressIndex, value, timestamp); + } + + /** Create a delete repair record. */ + public static RepairRecord delete(DataPointLocator locator, long progressIndex, long timestamp) { + return deleteOnFollower(locator, progressIndex, timestamp); + } + + /** Create an insert repair record that targets the leader replica. */ + public static RepairRecord insertToLeader( + DataPointLocator locator, long progressIndex, Object value, long timestamp) { + return new RepairRecord( + RecordType.INSERT, TargetReplica.LEADER, locator, progressIndex, value, timestamp); + } + + /** Create an insert repair record that targets the follower replica. */ + public static RepairRecord insertToFollower( + DataPointLocator locator, long progressIndex, Object value, long timestamp) { + return new RepairRecord( + RecordType.INSERT, TargetReplica.FOLLOWER, locator, progressIndex, value, timestamp); + } + + /** Create a delete repair record that targets the leader replica. */ + public static RepairRecord deleteOnLeader( + DataPointLocator locator, long progressIndex, long timestamp) { + return new RepairRecord( + RecordType.DELETE, TargetReplica.LEADER, locator, progressIndex, null, timestamp); + } + + /** Create a delete repair record that targets the follower replica. */ + public static RepairRecord deleteOnFollower( + DataPointLocator locator, long progressIndex, long timestamp) { + return new RepairRecord( + RecordType.DELETE, TargetReplica.FOLLOWER, locator, progressIndex, null, timestamp); + } + + public RecordType getType() { + return type; + } + + public TargetReplica getTargetReplica() { + return targetReplica; + } + + public DataPointLocator getLocator() { + return locator; + } + + public long getProgressIndex() { + return progressIndex; + } + + public Object getValue() { + return value; + } + + public long getTimestamp() { + return timestamp; + } + + @Override + public String toString() { + return String.format( + "RepairRecord{type=%s, target=%s, loc=%s, pi=%d}", + type, targetReplica, locator, progressIndex); + } +} diff --git a/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/consensus/iotv2/consistency/repair/RepairSession.java b/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/consensus/iotv2/consistency/repair/RepairSession.java new file mode 100644 index 0000000000000..9b8fec71fc670 --- /dev/null +++ b/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/consensus/iotv2/consistency/repair/RepairSession.java @@ -0,0 +1,195 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.iotdb.commons.consensus.iotv2.consistency.repair; + +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.util.ArrayList; +import java.util.Collections; +import java.util.List; +import java.util.UUID; + +/** + * WAL-backed staging area for streaming repair records with atomic promote semantics. All staged + * records are committed atomically (all-or-nothing) via promoteAtomically(). Uncommitted sessions + * are discarded on restart for crash safety. + * + *

Each RepairSession has a unique sessionId for idempotent replay detection. + */ +public class RepairSession { + + private static final Logger LOGGER = LoggerFactory.getLogger(RepairSession.class); + + public enum SessionState { + STAGING, + COMMITTED, + ABORTED + } + + public interface RepairSessionApplier { + void apply(String sessionId, long partitionId, List inserts, List deletes) + throws Exception; + } + + public interface RepairSessionJournal { + void append(String sessionId, RepairRecord record) throws Exception; + + void markCommitted(String sessionId) throws Exception; + + void delete(String sessionId); + } + + private static final RepairSessionApplier NO_OP_APPLIER = (sessionId, partitionId, inserts, deletes) -> {}; + + private static final RepairSessionJournal NO_OP_JOURNAL = + new RepairSessionJournal() { + @Override + public void append(String sessionId, RepairRecord record) {} + + @Override + public void markCommitted(String sessionId) {} + + @Override + public void delete(String sessionId) {} + }; + + private final String sessionId; + private final long partitionId; + private final List stagedRecords; + private final RepairSessionApplier applier; + private final RepairSessionJournal journal; + private volatile SessionState state; + + public RepairSession(long partitionId) { + this(partitionId, NO_OP_APPLIER, NO_OP_JOURNAL); + } + + public RepairSession(long partitionId, RepairSessionApplier applier) { + this(partitionId, applier, NO_OP_JOURNAL); + } + + public RepairSession( + long partitionId, RepairSessionApplier applier, RepairSessionJournal journal) { + this.sessionId = UUID.randomUUID().toString(); + this.partitionId = partitionId; + this.stagedRecords = new ArrayList<>(); + this.applier = applier == null ? NO_OP_APPLIER : applier; + this.journal = journal == null ? NO_OP_JOURNAL : journal; + this.state = SessionState.STAGING; + } + + /** + * Stage a repair record for atomic commit. The record is appended to the WAL before being added + * to the in-memory list. + * + * @param record the repair record to stage + * @throws IllegalStateException if the session is not in STAGING state + */ + public void stage(RepairRecord record) { + if (state != SessionState.STAGING) { + throw new IllegalStateException( + "Cannot stage records in session " + sessionId + " with state " + state); + } + try { + journal.append(sessionId, record); + } catch (Exception e) { + throw new IllegalStateException( + "Failed to append repair record into journal for session " + sessionId, e); + } + stagedRecords.add(record); + } + + /** + * @return true if promotion succeeded + */ + public boolean promoteAtomically() { + if (state != SessionState.STAGING) { + LOGGER.warn("Cannot promote session {} in state {}", sessionId, state); + return false; + } + + try { + // Separate inserts and deletes + List inserts = new ArrayList<>(); + List deletes = new ArrayList<>(); + for (RepairRecord record : stagedRecords) { + if (record.getType() == RepairRecord.RecordType.INSERT) { + inserts.add(record); + } else { + deletes.add(record); + } + } + applier.apply( + sessionId, + partitionId, + Collections.unmodifiableList(inserts), + Collections.unmodifiableList(deletes)); + journal.markCommitted(sessionId); + + LOGGER.info( + "RepairSession {} promoted: {} inserts, {} deletes for partition {}", + sessionId, + inserts.size(), + deletes.size(), + partitionId); + + state = SessionState.COMMITTED; + return true; + } catch (Exception e) { + LOGGER.error("Failed to promote RepairSession {}: {}", sessionId, e.getMessage(), e); + state = SessionState.ABORTED; + return false; + } + } + + /** Abort and clean up the session. Staged records are discarded. */ + public void abort() { + state = SessionState.ABORTED; + stagedRecords.clear(); + journal.delete(sessionId); + } + + /** Clean up WAL resources after successful promotion. */ + public void cleanup() { + journal.delete(sessionId); + stagedRecords.clear(); + } + + public String getSessionId() { + return sessionId; + } + + public long getPartitionId() { + return partitionId; + } + + public SessionState getState() { + return state; + } + + public List getStagedRecords() { + return Collections.unmodifiableList(stagedRecords); + } + + public int getStagedCount() { + return stagedRecords.size(); + } +} diff --git a/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/consensus/iotv2/consistency/repair/RepairStrategy.java b/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/consensus/iotv2/consistency/repair/RepairStrategy.java new file mode 100644 index 0000000000000..26b6e1d0ecb9a --- /dev/null +++ b/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/consensus/iotv2/consistency/repair/RepairStrategy.java @@ -0,0 +1,28 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.iotdb.commons.consensus.iotv2.consistency.repair; + +/** Per-TsFile repair strategy selected by the cost model. */ +public enum RepairStrategy { + /** Send entire existing TsFile from Leader to Follower. Zero packaging cost. */ + DIRECT_TSFILE_TRANSFER, + /** Stream individual diff points via RepairSession with atomic promote. */ + POINT_STREAMING +} diff --git a/iotdb-core/node-commons/src/test/java/org/apache/iotdb/commons/consensus/iotv2/consistency/ConsistencyMerkleTreeTest.java b/iotdb-core/node-commons/src/test/java/org/apache/iotdb/commons/consensus/iotv2/consistency/ConsistencyMerkleTreeTest.java new file mode 100644 index 0000000000000..016dcd2692a3c --- /dev/null +++ b/iotdb-core/node-commons/src/test/java/org/apache/iotdb/commons/consensus/iotv2/consistency/ConsistencyMerkleTreeTest.java @@ -0,0 +1,59 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.iotdb.commons.consensus.iotv2.consistency; + +import org.junit.Assert; +import org.junit.Test; + +import java.util.Arrays; +import java.util.Collections; + +public class ConsistencyMerkleTreeTest { + + @Test + public void shouldPreserveDualDigestOnFlush() { + ConsistencyMerkleTree merkleTree = new ConsistencyMerkleTree(); + DualDigest first = new DualDigest(0x01L, 0x10L); + DualDigest second = new DualDigest(0x02L, 0x20L); + + merkleTree.onTsFileFlushed(1L, first); + merkleTree.onTsFileFlushed(1L, second); + + Assert.assertEquals(new DualDigest(0x03L, 0x30L), merkleTree.getPartitionNode(1L).getPartitionDigest()); + Assert.assertEquals(new DualDigest(0x03L, 0x30L), merkleTree.getRegionDigest()); + } + + @Test + public void shouldPreserveDualDigestOnCompaction() { + ConsistencyMerkleTree merkleTree = new ConsistencyMerkleTree(); + DualDigest sourceOne = new DualDigest(0x01L, 0x10L); + DualDigest sourceTwo = new DualDigest(0x02L, 0x20L); + DualDigest targetOne = new DualDigest(0x04L, 0x40L); + DualDigest targetTwo = new DualDigest(0x08L, 0x80L); + + merkleTree.onTsFileFlushed(7L, sourceOne); + merkleTree.onTsFileFlushed(7L, sourceTwo); + merkleTree.onCompaction(Collections.singletonList(sourceOne), Arrays.asList(targetOne, targetTwo), 7L); + + Assert.assertEquals( + new DualDigest(0x0EL, 0xE0L), merkleTree.getPartitionNode(7L).getPartitionDigest()); + Assert.assertEquals(new DualDigest(0x0EL, 0xE0L), merkleTree.getRegionDigest()); + } +} diff --git a/iotdb-core/node-commons/src/test/java/org/apache/iotdb/commons/consensus/iotv2/consistency/repair/DiffAttributionTest.java b/iotdb-core/node-commons/src/test/java/org/apache/iotdb/commons/consensus/iotv2/consistency/repair/DiffAttributionTest.java new file mode 100644 index 0000000000000..ac738a14dce12 --- /dev/null +++ b/iotdb-core/node-commons/src/test/java/org/apache/iotdb/commons/consensus/iotv2/consistency/repair/DiffAttributionTest.java @@ -0,0 +1,70 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.iotdb.commons.consensus.iotv2.consistency.repair; + +import org.apache.iotdb.commons.consensus.iotv2.consistency.ibf.CompositeKeyCodec; +import org.apache.iotdb.commons.consensus.iotv2.consistency.ibf.DiffEntry; +import org.apache.iotdb.commons.consensus.iotv2.consistency.ibf.RowRefIndex; +import org.apache.iotdb.commons.consensus.iotv2.consistency.merkle.MerkleEntry; +import org.apache.iotdb.commons.consensus.iotv2.consistency.merkle.MerkleFileContent; + +import org.junit.Assert; +import org.junit.Test; + +import java.util.Arrays; +import java.util.Collections; +import java.util.List; +import java.util.Map; + +public class DiffAttributionTest { + + @Test + public void shouldUseActualBucketBoundariesWhenAttributingDiffs() { + DiffAttribution diffAttribution = new DiffAttribution(); + RowRefIndex rowRefIndex = + new RowRefIndex.Builder() + .addDevice("root.sg.d1", Collections.singletonList("s1")) + .setTimeBucketStart(0L) + .build(); + long compositeKey = CompositeKeyCodec.encode(0, 0, 250L, 0L, 1L); + DiffEntry diffEntry = new DiffEntry(compositeKey, 1L, DiffEntry.DiffType.LEADER_HAS); + List merkleFiles = + Arrays.asList( + new MerkleFileContent( + 1L, + 10L, + Collections.singletonList( + new MerkleEntry("root.sg.d1", "s1", 100L, 200L, 10, 101L)), + "first.tsfile"), + new MerkleFileContent( + 2L, + 20L, + Collections.singletonList( + new MerkleEntry("root.sg.d1", "s1", 200L, 300L, 10, 202L)), + "second.tsfile")); + + Map> attributed = + diffAttribution.attributeToSourceTsFiles( + Collections.singletonList(diffEntry), rowRefIndex, merkleFiles); + + Assert.assertFalse(attributed.containsKey("first.tsfile")); + Assert.assertEquals(Collections.singletonList(diffEntry), attributed.get("second.tsfile")); + } +} diff --git a/iotdb-core/node-commons/src/test/java/org/apache/iotdb/commons/consensus/iotv2/consistency/repair/RepairConflictResolverTest.java b/iotdb-core/node-commons/src/test/java/org/apache/iotdb/commons/consensus/iotv2/consistency/repair/RepairConflictResolverTest.java new file mode 100644 index 0000000000000..5ce0d21e285d7 --- /dev/null +++ b/iotdb-core/node-commons/src/test/java/org/apache/iotdb/commons/consensus/iotv2/consistency/repair/RepairConflictResolverTest.java @@ -0,0 +1,76 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.iotdb.commons.consensus.iotv2.consistency.repair; + +import org.apache.iotdb.commons.consensus.iotv2.consistency.ibf.DataPointLocator; + +import org.junit.Assert; +import org.junit.Test; + +import java.util.Collections; + +public class RepairConflictResolverTest { + + @Test + public void leaderHasWithoutNewerFollowerDeletionShouldSendToFollower() { + RepairConflictResolver resolver = + new RepairConflictResolver(Collections.emptyList(), Collections.emptyList()); + + Assert.assertEquals( + RepairAction.SEND_TO_FOLLOWER, + resolver.resolveLeaderHas(new DataPointLocator("root.sg.d1", "s1", 100L), 10L)); + } + + @Test + public void leaderHasWithNewerFollowerDeletionShouldDeleteOnLeader() { + RepairConflictResolver resolver = + new RepairConflictResolver( + Collections.emptyList(), + Collections.singletonList( + new ModEntrySummary("root.sg.d1", "s1", 0L, 200L, 20L))); + + Assert.assertEquals( + RepairAction.DELETE_ON_LEADER, + resolver.resolveLeaderHas(new DataPointLocator("root.sg.d1", "s1", 100L), 10L)); + } + + @Test + public void followerHasWithoutNewerLeaderDeletionShouldSendToLeader() { + RepairConflictResolver resolver = + new RepairConflictResolver(Collections.emptyList(), Collections.emptyList()); + + Assert.assertEquals( + RepairAction.SEND_TO_LEADER, + resolver.resolveFollowerHas(new DataPointLocator("root.sg.d1", "s1", 100L), 10L)); + } + + @Test + public void followerHasWithNewerLeaderDeletionShouldDeleteOnFollower() { + RepairConflictResolver resolver = + new RepairConflictResolver( + Collections.singletonList( + new ModEntrySummary("root.sg.d1", "s1", 0L, 200L, 20L)), + Collections.emptyList()); + + Assert.assertEquals( + RepairAction.DELETE_ON_FOLLOWER, + resolver.resolveFollowerHas(new DataPointLocator("root.sg.d1", "s1", 100L), 10L)); + } +} diff --git a/iotdb-core/node-commons/src/test/java/org/apache/iotdb/commons/consensus/iotv2/consistency/repair/RepairSessionTest.java b/iotdb-core/node-commons/src/test/java/org/apache/iotdb/commons/consensus/iotv2/consistency/repair/RepairSessionTest.java new file mode 100644 index 0000000000000..2c91b46a72ff9 --- /dev/null +++ b/iotdb-core/node-commons/src/test/java/org/apache/iotdb/commons/consensus/iotv2/consistency/repair/RepairSessionTest.java @@ -0,0 +1,80 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.iotdb.commons.consensus.iotv2.consistency.repair; + +import org.apache.iotdb.commons.consensus.iotv2.consistency.ibf.DataPointLocator; + +import org.junit.Assert; +import org.junit.Test; + +import java.util.ArrayList; +import java.util.Arrays; +import java.util.Collections; +import java.util.List; + +public class RepairSessionTest { + + @Test + public void shouldApplyAndJournalRecordsAtomically() { + List journalEvents = new ArrayList<>(); + List appliedInserts = new ArrayList<>(); + List appliedDeletes = new ArrayList<>(); + RepairSession session = + new RepairSession( + 5L, + (sessionId, partitionId, inserts, deletes) -> { + appliedInserts.addAll(inserts); + appliedDeletes.addAll(deletes); + }, + new RepairSession.RepairSessionJournal() { + @Override + public void append(String sessionId, RepairRecord record) { + journalEvents.add("append:" + record.getType()); + } + + @Override + public void markCommitted(String sessionId) { + journalEvents.add("commit"); + } + + @Override + public void delete(String sessionId) { + journalEvents.add("delete"); + } + }); + RepairRecord insert = + RepairRecord.insert(new DataPointLocator("root.sg.d1", "s1", 100L), 1L, "v1", 100L); + RepairRecord delete = + RepairRecord.delete(new DataPointLocator("root.sg.d1", "s1", 101L), 2L, 101L); + + session.stage(insert); + session.stage(delete); + + Assert.assertTrue(session.promoteAtomically()); + Assert.assertEquals(RepairSession.SessionState.COMMITTED, session.getState()); + Assert.assertEquals(Collections.singletonList(insert), appliedInserts); + Assert.assertEquals(Collections.singletonList(delete), appliedDeletes); + + session.cleanup(); + + Assert.assertEquals( + Arrays.asList("append:INSERT", "append:DELETE", "commit", "delete"), journalEvents); + } +} diff --git a/iotdb-protocol/thrift-confignode/src/main/thrift/confignode.thrift b/iotdb-protocol/thrift-confignode/src/main/thrift/confignode.thrift index 92312ee81a307..c34bd594f5204 100644 --- a/iotdb-protocol/thrift-confignode/src/main/thrift/confignode.thrift +++ b/iotdb-protocol/thrift-confignode/src/main/thrift/confignode.thrift @@ -173,6 +173,10 @@ struct TSetDataNodeStatusReq { 2: required string status } +struct TTriggerRegionConsistencyRepairReq { + 1: required common.TConsensusGroupId consensusGroupId +} + // Database struct TDeleteDatabaseReq { 1: required string prefixPath @@ -1753,6 +1757,9 @@ service IConfigNodeRPCService { common.TSStatus removeRegion(TRemoveRegionReq req) + /** Trigger replica consistency check and repair for a single DataRegion */ + common.TSStatus triggerRegionConsistencyRepair(TTriggerRegionConsistencyRepairReq req) + /** Kill query */ common.TSStatus killQuery(string queryId, i32 dataNodeId, string allowedUsername) @@ -2055,4 +2062,3 @@ service IConfigNodeRPCService { common.TSStatus createTableView(TCreateTableViewReq req) } - diff --git a/iotdb-protocol/thrift-datanode/src/main/thrift/datanode.thrift b/iotdb-protocol/thrift-datanode/src/main/thrift/datanode.thrift index cca7110f28d40..ae67e9b9f8163 100644 --- a/iotdb-protocol/thrift-datanode/src/main/thrift/datanode.thrift +++ b/iotdb-protocol/thrift-datanode/src/main/thrift/datanode.thrift @@ -428,6 +428,33 @@ struct TLoadResp { 3: optional common.TSStatus status } +struct TConsistencyMerkleFile { + 1: required string sourceTsFilePath + 2: required i64 tsFileSize + 3: required i64 fileXorHash + 4: required i64 fileAddHash +} + +struct TTimePartitionConsistencyView { + 1: required i64 timePartitionId + 2: required list merkleFiles +} + +struct TDataRegionConsistencySnapshotReq { + 1: required common.TConsensusGroupId consensusGroupId +} + +struct TDataRegionConsistencySnapshotResp { + 1: required common.TSStatus status + 2: optional list timePartitionViews +} + +struct TRepairTransferTsFileReq { + 1: required common.TConsensusGroupId consensusGroupId + 2: required string sourceTsFilePath + 3: required list targetDataNodes +} + struct TConstructSchemaBlackListReq { 1: required list schemaRegionIdList 2: required binary pathPatternTree @@ -815,6 +842,11 @@ service IDataNodeRPCService { TLoadResp sendLoadCommand(TLoadCommandReq req); + TDataRegionConsistencySnapshotResp getDataRegionConsistencySnapshot( + TDataRegionConsistencySnapshotReq req); + + common.TSStatus repairTransferTsFile(TRepairTransferTsFileReq req); + common.TSStatus updateAttribute(TAttributeUpdateReq req); @@ -1291,4 +1323,4 @@ service MPPDataExchangeService { /** Empty rpc, only for connection test */ common.TSStatus testConnectionEmptyRPC() -} \ No newline at end of file +} diff --git a/replica_consistency_check_repair_review.md b/replica_consistency_check_repair_review.md new file mode 100644 index 0000000000000..45dfb456a6e5a --- /dev/null +++ b/replica_consistency_check_repair_review.md @@ -0,0 +1,43 @@ +# Review: Replica Consistency Check and Repair Plan + +## 主要发现(按严重级别) + +- **P0:IBF 解码结果与修复输入不闭环,当前定义下无法直接落地** + - 方案中 `Key=xxHash64(deviceId||measurement||timestamp)`,但后续流式修复直接使用 `deviceId/measurement/timestamp` 查询原始点;中间缺少“哈希键 -> 原始点位”的反查索引设计。 + - 建议:明确一种可实现路径(例如在参与 IBF 的窗口内构建临时 `key -> rowRef` 映射,或将可逆编码扩展为携带定位信息)。 + +- **P0:`last_repaired_watermark` 单一水位可能掩盖局部失败,造成永久漏检** + - 有效范围定义为 `(lastRepairedWatermark, T_safe]`,但失败仅记录在 `failed_partitions` 文本字段;如果本轮部分分区失败但全局水位仍推进,失败区间可能被跳过。 + - 建议:改为“分区级/子范围级 checkpoint”(至少 `partition_id + repaired_to`),或仅在范围内全成功时推进全局水位。 + +- **P1:XOR 聚合存在可抵消性,存在误判为一致的理论窗口** + - XOR 作为集合聚合摘要存在信息损失,不同差异组合可能相互抵消,导致误判一致。 + - 建议:至少使用双摘要(如 `xor + sum(mod 2^64)`),或改为真正树形聚合(`parent=H(left||right)`)。 + +- **P1:`Leader wins` 与弱一致/领导权切换语义可能冲突** + - 方案中多处采用“Leader 版本覆盖”,但 leader 转移与复制延迟窗口下,若不统一按 `ProgressIndex/term` 做因果裁决,可能覆盖掉更“新”的删除或写入。 + - 建议:将冲突裁决统一定义为“更高 `ProgressIndex` 获胜”,并落实到所有 repair record 的 apply 逻辑。 + +- **P1:修复应用阶段缺少明确原子提交边界** + - Procedure 有状态机和 checkpoint,但 streaming repair 的“写入可见性边界”与“重试幂等边界”尚不清晰,存在部分应用后重放副作用风险。 + - 建议:引入 repair session staging + 原子 promote,或最少定义严格幂等键、去重窗口与 WAL 持久化顺序。 + +- **P2:性能估算偏乐观,可能触发锁竞争** + - 视图构建若持有 `resourceListLock` 读锁执行重扫描/加载,可能抑制 compaction 写锁并放大尾延迟。 + - 建议:采用“短读锁抓快照 + 锁外重活”,并增加单分区最大处理时长与中断点。 + +## 测试缺口(建议作为准入门槛) + +- 故障注入:`EXECUTE_REPAIR` 中点崩溃、网络抖动、leader 切换,验证无重复/无遗漏/无回退。 +- 语义冲突:同 key 上插入/删除并发与乱序,验证 `ProgressIndex` 裁决一致。 +- 正确性对照:随机数据集做全量扫描真值对比,量化 false-negative(尤其 XOR 聚合方案)。 +- 大分歧退化:分区级缺失/整段缺失,验证快速跳过 IBF 直接全量修复。 +- 幂等回放:同一 repair session 重放多次结果不变。 + +## 结论 + +方案方向正确、工程化程度高,但建议在上线前优先收敛三点: + +1. 打通 IBF 解码到修复输入的可实现闭环; +2. 引入分区级 checkpoint,避免全局水位推进掩盖失败; +3. 将冲突裁决统一为 `ProgressIndex`,替代“Leader wins”口径。 From 9c22bd3967958b415ca8876563f531b7daad6d75 Mon Sep 17 00:00:00 2001 From: Peng Junzhi <201250214@smail.nju.edu.cn> Date: Wed, 18 Mar 2026 23:19:44 +0800 Subject: [PATCH 2/4] fix some test & spotless --- .../IoTDBIoTConsensusV23C3DBasicITBase.java | 68 +++++++++++++------ .../client/sync/SyncDataNodeClientPool.java | 2 +- .../confignode/manager/ProcedureManager.java | 6 +- .../consistency/RepairRegionProcedure.java | 3 +- .../RepairRegionProcedureTest.java | 21 +++--- .../impl/DataNodeInternalRPCServiceImpl.java | 8 +-- .../DataRegionConsistencyRepairService.java | 57 ++++++++++------ .../consistency/ConsistencyMerkleTree.java | 8 ++- .../consistency/repair/RepairSession.java | 6 +- .../ConsistencyMerkleTreeTest.java | 6 +- .../repair/RepairConflictResolverTest.java | 6 +- 11 files changed, 114 insertions(+), 77 deletions(-) diff --git a/integration-test/src/test/java/org/apache/iotdb/db/it/iotconsensusv2/IoTDBIoTConsensusV23C3DBasicITBase.java b/integration-test/src/test/java/org/apache/iotdb/db/it/iotconsensusv2/IoTDBIoTConsensusV23C3DBasicITBase.java index d2394eac57249..34a4b67bec3fd 100644 --- a/integration-test/src/test/java/org/apache/iotdb/db/it/iotconsensusv2/IoTDBIoTConsensusV23C3DBasicITBase.java +++ b/integration-test/src/test/java/org/apache/iotdb/db/it/iotconsensusv2/IoTDBIoTConsensusV23C3DBasicITBase.java @@ -26,13 +26,13 @@ import org.apache.iotdb.confignode.it.regionmigration.IoTDBRegionOperationReliabilityITFramework; import org.apache.iotdb.confignode.rpc.thrift.TTriggerRegionConsistencyRepairReq; import org.apache.iotdb.consensus.ConsensusFactory; +import org.apache.iotdb.db.storageengine.dataregion.modification.ModificationFile; +import org.apache.iotdb.db.storageengine.dataregion.modification.v1.ModificationFileV1; +import org.apache.iotdb.db.storageengine.dataregion.tsfile.TsFileResource; import org.apache.iotdb.isession.SessionConfig; import org.apache.iotdb.it.env.EnvFactory; import org.apache.iotdb.it.env.cluster.node.DataNodeWrapper; import org.apache.iotdb.itbase.env.BaseEnv; -import org.apache.iotdb.db.storageengine.dataregion.modification.ModificationFile; -import org.apache.iotdb.db.storageengine.dataregion.modification.v1.ModificationFileV1; -import org.apache.iotdb.db.storageengine.dataregion.tsfile.TsFileResource; import org.apache.tsfile.utils.Pair; import org.awaitility.Awaitility; @@ -42,12 +42,13 @@ import org.slf4j.LoggerFactory; import java.io.File; -import java.sql.Connection; -import java.sql.ResultSet; -import java.sql.Statement; import java.nio.file.Files; import java.nio.file.Path; import java.nio.file.Paths; +import java.sql.Connection; +import java.sql.ResultSet; +import java.sql.SQLException; +import java.sql.Statement; import java.util.Collections; import java.util.Map; import java.util.Optional; @@ -224,7 +225,8 @@ public void testReplicaConsistencyAfterDeleteAndLeaderStop() throws Exception { Awaitility.await() .pollDelay(2, TimeUnit.SECONDS) .atMost(2, TimeUnit.MINUTES) - .untilAsserted(() -> verifyPostDeleteConsistencyOnNode(regionReplicaSelection.followerNode)); + .untilAsserted( + () -> verifyPostDeleteConsistencyOnNode(regionReplicaSelection.followerNode)); LOGGER.info( "Replica consistency verified after delete and failover on follower DataNode {}", @@ -248,7 +250,8 @@ public void testReplicaConsistencyRepairAfterFollowerLosesSealedTsFile() throws regionReplicaSelection = selectReplicatedDataRegion(statement); waitForReplicationComplete(regionReplicaSelection.leaderNode); deletedTsFile = - findLatestSealedTsFile(regionReplicaSelection.followerNode, regionReplicaSelection.regionId); + findLatestSealedTsFile( + regionReplicaSelection.followerNode, regionReplicaSelection.regionId); } LOGGER.info( @@ -261,6 +264,7 @@ public void testReplicaConsistencyRepairAfterFollowerLosesSealedTsFile() throws deleteTsFileArtifacts(deletedTsFile); regionReplicaSelection.followerNode.start(); + waitForNodeConnectionReady(regionReplicaSelection.followerNode); Awaitility.await() .pollDelay(2, TimeUnit.SECONDS) .atMost(2, TimeUnit.MINUTES) @@ -295,7 +299,8 @@ protected void insertAndFlushTestData(Statement statement) throws Exception { statement.execute(FLUSH_COMMAND); } - protected RegionReplicaSelection selectReplicatedDataRegion(Statement statement) throws Exception { + protected RegionReplicaSelection selectReplicatedDataRegion(Statement statement) + throws Exception { Map>> dataRegionMap = getDataRegionMapWithLeader(statement); for (Map.Entry>> entry : dataRegionMap.entrySet()) { @@ -323,11 +328,7 @@ protected RegionReplicaSelection selectReplicatedDataRegion(Statement statement) .dataNodeIdToWrapper(followerDataNodeId) .orElseThrow(() -> new AssertionError("Follower DataNode not found in cluster")); return new RegionReplicaSelection( - entry.getKey(), - leaderDataNodeId, - followerDataNodeId, - leaderNode, - followerNode); + entry.getKey(), leaderDataNodeId, followerDataNodeId, leaderNode, followerNode); } Assert.fail("Should find a replicated data region with a leader for root.sg"); @@ -493,6 +494,30 @@ protected void assertDataInconsistentOnNode(DataNodeWrapper targetNode) throws E } } + protected void waitForNodeConnectionReady(DataNodeWrapper targetNode) { + Awaitility.await() + .pollDelay(1, TimeUnit.SECONDS) + .pollInterval(1, TimeUnit.SECONDS) + .atMost(2, TimeUnit.MINUTES) + .untilAsserted( + () -> { + try (Connection ignored = + makeItCloseQuietly( + EnvFactory.getEnv() + .getConnection( + targetNode, + SessionConfig.DEFAULT_USER, + SessionConfig.DEFAULT_PASSWORD, + BaseEnv.TREE_SQL_DIALECT))) { + Assert.assertNotNull( + "Expected a JDBC connection for DataNode " + targetNode.getId(), ignored); + } catch (SQLException e) { + throw new AssertionError( + "DataNode " + targetNode.getId() + " is not accepting JDBC connections yet", e); + } + }); + } + protected static void assertNullValue(String value) { Assert.assertTrue( "Expected deleted value to be null, but was " + value, @@ -541,7 +566,9 @@ private Path findLatestSealedTsFile(DataNodeWrapper dataNodeWrapper, int regionI .filter(path -> path.getFileName().toString().endsWith(".tsfile")) .filter(path -> path.toString().contains(File.separator + "root.sg" + File.separator)) .filter(path -> belongsToRegion(path, regionId)) - .max((left, right) -> Long.compare(left.toFile().lastModified(), right.toFile().lastModified())); + .max( + (left, right) -> + Long.compare(left.toFile().lastModified(), right.toFile().lastModified())); if (candidate.isPresent()) { return candidate.get(); } @@ -561,21 +588,18 @@ private void deleteTsFileArtifacts(Path tsFile) throws Exception { Files.deleteIfExists(Paths.get(tsFile.toString() + ModificationFile.FILE_SUFFIX)); Files.deleteIfExists(Paths.get(tsFile.toString() + ModificationFile.COMPACTION_FILE_SUFFIX)); Files.deleteIfExists(Paths.get(tsFile.toString() + ModificationFileV1.FILE_SUFFIX)); - Files.deleteIfExists( - Paths.get(tsFile.toString() + ModificationFileV1.COMPACTION_FILE_SUFFIX)); + Files.deleteIfExists(Paths.get(tsFile.toString() + ModificationFileV1.COMPACTION_FILE_SUFFIX)); } private void triggerRegionConsistencyRepair(int regionId) throws Exception { - TConsensusGroupId consensusGroupId = new TConsensusGroupId(TConsensusGroupType.DataRegion, regionId); + TConsensusGroupId consensusGroupId = + new TConsensusGroupId(TConsensusGroupType.DataRegion, regionId); try (SyncConfigNodeIServiceClient client = (SyncConfigNodeIServiceClient) EnvFactory.getEnv().getLeaderConfigNodeConnection()) { TSStatus status = client.triggerRegionConsistencyRepair( new TTriggerRegionConsistencyRepairReq(consensusGroupId)); - Assert.assertEquals( - "Replica consistency repair should succeed", - 200, - status.getCode()); + Assert.assertEquals("Replica consistency repair should succeed", 200, status.getCode()); } } } diff --git a/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/client/sync/SyncDataNodeClientPool.java b/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/client/sync/SyncDataNodeClientPool.java index 588046c1f6af2..69b514a06976a 100644 --- a/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/client/sync/SyncDataNodeClientPool.java +++ b/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/client/sync/SyncDataNodeClientPool.java @@ -33,13 +33,13 @@ import org.apache.iotdb.mpp.rpc.thrift.TCreatePeerReq; import org.apache.iotdb.mpp.rpc.thrift.TCreateSchemaRegionReq; import org.apache.iotdb.mpp.rpc.thrift.TDataRegionConsistencySnapshotReq; -import org.apache.iotdb.mpp.rpc.thrift.TRepairTransferTsFileReq; import org.apache.iotdb.mpp.rpc.thrift.TInvalidateCacheReq; import org.apache.iotdb.mpp.rpc.thrift.TInvalidatePermissionCacheReq; import org.apache.iotdb.mpp.rpc.thrift.TKillQueryInstanceReq; import org.apache.iotdb.mpp.rpc.thrift.TMaintainPeerReq; import org.apache.iotdb.mpp.rpc.thrift.TRegionLeaderChangeReq; import org.apache.iotdb.mpp.rpc.thrift.TRegionLeaderChangeResp; +import org.apache.iotdb.mpp.rpc.thrift.TRepairTransferTsFileReq; import org.apache.iotdb.mpp.rpc.thrift.TResetPeerListReq; import org.apache.iotdb.mpp.rpc.thrift.TUpdateTableReq; import org.apache.iotdb.mpp.rpc.thrift.TUpdateTemplateReq; diff --git a/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/manager/ProcedureManager.java b/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/manager/ProcedureManager.java index e5d70c308666d..88ee234c8fc0b 100644 --- a/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/manager/ProcedureManager.java +++ b/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/manager/ProcedureManager.java @@ -63,6 +63,7 @@ import org.apache.iotdb.confignode.procedure.env.RegionMaintainHandler; import org.apache.iotdb.confignode.procedure.env.RemoveDataNodeHandler; import org.apache.iotdb.confignode.procedure.impl.consistency.LiveDataRegionRepairExecutionContext; +import org.apache.iotdb.confignode.procedure.impl.consistency.RepairRegionProcedure; import org.apache.iotdb.confignode.procedure.impl.cq.CreateCQProcedure; import org.apache.iotdb.confignode.procedure.impl.node.AddConfigNodeProcedure; import org.apache.iotdb.confignode.procedure.impl.node.RemoveAINodeProcedure; @@ -85,7 +86,6 @@ import org.apache.iotdb.confignode.procedure.impl.region.RegionMigrationPlan; import org.apache.iotdb.confignode.procedure.impl.region.RegionOperationProcedure; import org.apache.iotdb.confignode.procedure.impl.region.RemoveRegionPeerProcedure; -import org.apache.iotdb.confignode.procedure.impl.consistency.RepairRegionProcedure; import org.apache.iotdb.confignode.procedure.impl.schema.AlterEncodingCompressorProcedure; import org.apache.iotdb.confignode.procedure.impl.schema.AlterLogicalViewProcedure; import org.apache.iotdb.confignode.procedure.impl.schema.AlterTimeSeriesDataTypeProcedure; @@ -1402,9 +1402,7 @@ public TSStatus triggerRegionConsistencyRepair(final TConsensusGroupId consensus return waitingProcedureFinished(procedure); } catch (Exception e) { LOGGER.warn( - "Failed to trigger replica consistency repair for region {}", - consensusGroupId, - e); + "Failed to trigger replica consistency repair for region {}", consensusGroupId, e); return new TSStatus(TSStatusCode.EXECUTE_STATEMENT_ERROR.getStatusCode()) .setMessage(e.getMessage()); } diff --git a/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/procedure/impl/consistency/RepairRegionProcedure.java b/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/procedure/impl/consistency/RepairRegionProcedure.java index e3405e98040af..3c0d6008c50f1 100644 --- a/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/procedure/impl/consistency/RepairRegionProcedure.java +++ b/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/procedure/impl/consistency/RepairRegionProcedure.java @@ -810,7 +810,8 @@ private RepairProgressTable getOrCreateRepairProgressTable() { if (repairProgressTable == null) { RepairExecutionContext context = getExecutionContextIfPresent(); if (context != null) { - repairProgressTable = context.loadRepairProgressTable(toConsensusGroupKey(consensusGroupId)); + repairProgressTable = + context.loadRepairProgressTable(toConsensusGroupKey(consensusGroupId)); } if (repairProgressTable == null) { repairProgressTable = new RepairProgressTable(toConsensusGroupKey(consensusGroupId)); diff --git a/iotdb-core/confignode/src/test/java/org/apache/iotdb/confignode/procedure/impl/consistency/RepairRegionProcedureTest.java b/iotdb-core/confignode/src/test/java/org/apache/iotdb/confignode/procedure/impl/consistency/RepairRegionProcedureTest.java index 7237bcf6586a2..54533991a9cd8 100644 --- a/iotdb-core/confignode/src/test/java/org/apache/iotdb/confignode/procedure/impl/consistency/RepairRegionProcedureTest.java +++ b/iotdb-core/confignode/src/test/java/org/apache/iotdb/confignode/procedure/impl/consistency/RepairRegionProcedureTest.java @@ -28,9 +28,9 @@ import org.apache.iotdb.commons.consensus.iotv2.consistency.ibf.RowRefIndex; import org.apache.iotdb.commons.consensus.iotv2.consistency.merkle.MerkleEntry; import org.apache.iotdb.commons.consensus.iotv2.consistency.merkle.MerkleFileContent; +import org.apache.iotdb.commons.consensus.iotv2.consistency.repair.ModEntrySummary; import org.apache.iotdb.commons.consensus.iotv2.consistency.repair.RepairAction; import org.apache.iotdb.commons.consensus.iotv2.consistency.repair.RepairConflictResolver; -import org.apache.iotdb.commons.consensus.iotv2.consistency.repair.ModEntrySummary; import org.apache.iotdb.commons.consensus.iotv2.consistency.repair.RepairPlan; import org.apache.iotdb.commons.consensus.iotv2.consistency.repair.RepairRecord; import org.apache.iotdb.commons.consensus.iotv2.consistency.repair.RepairSession; @@ -62,8 +62,7 @@ public void serDeTest() throws Exception { TConsensusGroupId groupId = new TConsensusGroupId(TConsensusGroupType.DataRegion, 10); RepairRegionProcedure procedure = new RepairRegionProcedure( - groupId, - new TestExecutionContext(Collections.emptyMap(), new SimulatedReplicaState())); + groupId, new TestExecutionContext(Collections.emptyMap(), new SimulatedReplicaState())); try (PublicBAOS byteArrayOutputStream = new PublicBAOS(); DataOutputStream outputStream = new DataOutputStream(byteArrayOutputStream)) { procedure.serialize(outputStream); @@ -165,7 +164,8 @@ public void executeRepairFlowTest() throws Exception { } @Test - public void executeRepairFlowDeletesFollowerOnlyDataWhenLeaderDeletionWinsTest() throws Exception { + public void executeRepairFlowDeletesFollowerOnlyDataWhenLeaderDeletionWinsTest() + throws Exception { TConsensusGroupId groupId = new TConsensusGroupId(TConsensusGroupType.DataRegion, 2); RowRefIndex rowRefIndex = new RowRefIndex.Builder() @@ -735,15 +735,13 @@ private void applyInserts(List inserts) { for (RepairRecord record : inserts) { if (record.getTargetReplica() == RepairRecord.TargetReplica.LEADER) { SimulatedPoint followerPoint = followerPoints.get(record.getLocator()); - String tsFilePath = followerPoint == null ? "stream.tsfile" : followerPoint.getTsFilePath(); + String tsFilePath = + followerPoint == null ? "stream.tsfile" : followerPoint.getTsFilePath(); leaderTsFiles.add(tsFilePath); leaderPoints.put( record.getLocator(), new SimulatedPoint( - tsFilePath, - record.getLocator(), - record.getProgressIndex(), - record.getValue())); + tsFilePath, record.getLocator(), record.getProgressIndex(), record.getValue())); } else { SimulatedPoint leaderPoint = leaderPoints.get(record.getLocator()); String tsFilePath = leaderPoint == null ? "stream.tsfile" : leaderPoint.getTsFilePath(); @@ -751,10 +749,7 @@ private void applyInserts(List inserts) { followerPoints.put( record.getLocator(), new SimulatedPoint( - tsFilePath, - record.getLocator(), - record.getProgressIndex(), - record.getValue())); + tsFilePath, record.getLocator(), record.getProgressIndex(), record.getValue())); } } } diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/protocol/thrift/impl/DataNodeInternalRPCServiceImpl.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/protocol/thrift/impl/DataNodeInternalRPCServiceImpl.java index 2e87d24144d2f..7edf9fde0d3d8 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/protocol/thrift/impl/DataNodeInternalRPCServiceImpl.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/protocol/thrift/impl/DataNodeInternalRPCServiceImpl.java @@ -193,11 +193,11 @@ import org.apache.iotdb.db.service.externalservice.ExternalServiceManagementService; import org.apache.iotdb.db.service.metrics.FileMetrics; import org.apache.iotdb.db.storageengine.StorageEngine; -import org.apache.iotdb.db.storageengine.dataregion.consistency.DataRegionConsistencyRepairService; import org.apache.iotdb.db.storageengine.dataregion.compaction.repair.RepairTaskStatus; import org.apache.iotdb.db.storageengine.dataregion.compaction.schedule.CompactionScheduleTaskManager; import org.apache.iotdb.db.storageengine.dataregion.compaction.schedule.CompactionTaskManager; import org.apache.iotdb.db.storageengine.dataregion.compaction.settle.SettleRequestHandler; +import org.apache.iotdb.db.storageengine.dataregion.consistency.DataRegionConsistencyRepairService; import org.apache.iotdb.db.storageengine.dataregion.flush.CompressionRatio; import org.apache.iotdb.db.storageengine.dataregion.modification.DeletionPredicate; import org.apache.iotdb.db.storageengine.dataregion.modification.IDPredicate; @@ -239,10 +239,10 @@ import org.apache.iotdb.mpp.rpc.thrift.TCreatePipePluginInstanceReq; import org.apache.iotdb.mpp.rpc.thrift.TCreateSchemaRegionReq; import org.apache.iotdb.mpp.rpc.thrift.TCreateTriggerInstanceReq; -import org.apache.iotdb.mpp.rpc.thrift.TDataRegionConsistencySnapshotReq; -import org.apache.iotdb.mpp.rpc.thrift.TDataRegionConsistencySnapshotResp; import org.apache.iotdb.mpp.rpc.thrift.TDataNodeHeartbeatReq; import org.apache.iotdb.mpp.rpc.thrift.TDataNodeHeartbeatResp; +import org.apache.iotdb.mpp.rpc.thrift.TDataRegionConsistencySnapshotReq; +import org.apache.iotdb.mpp.rpc.thrift.TDataRegionConsistencySnapshotResp; import org.apache.iotdb.mpp.rpc.thrift.TDeactivateTemplateReq; import org.apache.iotdb.mpp.rpc.thrift.TDeleteColumnDataReq; import org.apache.iotdb.mpp.rpc.thrift.TDeleteDataForDeleteSchemaReq; @@ -289,11 +289,11 @@ import org.apache.iotdb.mpp.rpc.thrift.TPushTopicMetaReq; import org.apache.iotdb.mpp.rpc.thrift.TPushTopicMetaResp; import org.apache.iotdb.mpp.rpc.thrift.TPushTopicMetaRespExceptionMessage; -import org.apache.iotdb.mpp.rpc.thrift.TRepairTransferTsFileReq; import org.apache.iotdb.mpp.rpc.thrift.TRegionLeaderChangeReq; import org.apache.iotdb.mpp.rpc.thrift.TRegionLeaderChangeResp; import org.apache.iotdb.mpp.rpc.thrift.TRegionMigrateResult; import org.apache.iotdb.mpp.rpc.thrift.TRegionRouteReq; +import org.apache.iotdb.mpp.rpc.thrift.TRepairTransferTsFileReq; import org.apache.iotdb.mpp.rpc.thrift.TResetPeerListReq; import org.apache.iotdb.mpp.rpc.thrift.TRollbackSchemaBlackListReq; import org.apache.iotdb.mpp.rpc.thrift.TRollbackSchemaBlackListWithTemplateReq; diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/consistency/DataRegionConsistencyRepairService.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/consistency/DataRegionConsistencyRepairService.java index 2a9700c3cbd18..3964d46486161 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/consistency/DataRegionConsistencyRepairService.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/consistency/DataRegionConsistencyRepairService.java @@ -31,6 +31,7 @@ import org.apache.iotdb.commons.consensus.DataRegionId; import org.apache.iotdb.commons.consensus.index.ProgressIndex; import org.apache.iotdb.db.conf.IoTDBDescriptor; +import org.apache.iotdb.db.exception.load.LoadFileException; import org.apache.iotdb.db.queryengine.plan.planner.plan.node.PlanNodeId; import org.apache.iotdb.db.queryengine.plan.planner.plan.node.load.LoadTsFilePieceNode; import org.apache.iotdb.db.queryengine.plan.scheduler.load.LoadTsFileScheduler; @@ -75,11 +76,10 @@ public class DataRegionConsistencyRepairService { private final StorageEngine storageEngine = StorageEngine.getInstance(); - private final IClientManager - clientManager = - new IClientManager.Factory() - .createClientManager( - new ClientPoolFactory.SyncDataNodeInternalServiceClientPoolFactory()); + private final IClientManager clientManager = + new IClientManager.Factory() + .createClientManager( + new ClientPoolFactory.SyncDataNodeInternalServiceClientPoolFactory()); public TDataRegionConsistencySnapshotResp getSnapshot(TDataRegionConsistencySnapshotReq req) { DataRegion dataRegion = getDataRegion(req.getConsensusGroupId()); @@ -105,9 +105,7 @@ public TDataRegionConsistencySnapshotResp getSnapshot(TDataRegionConsistencySnap .setTimePartitionViews(partitionViews); } catch (Exception e) { LOGGER.warn( - "Failed to build consistency snapshot for region {}", - req.getConsensusGroupId(), - e); + "Failed to build consistency snapshot for region {}", req.getConsensusGroupId(), e); return new TDataRegionConsistencySnapshotResp( RpcUtils.getStatus(TSStatusCode.EXECUTE_STATEMENT_ERROR, e.getMessage())); } @@ -145,7 +143,8 @@ public TSStatus repairTransferTsFile(TRepairTransferTsFileReq req) { } for (TDataNodeLocation targetDataNode : req.getTargetDataNodes()) { - TSStatus status = transferOneTarget(req.getConsensusGroupId(), tsFileResource, targetDataNode); + TSStatus status = + transferOneTarget(req.getConsensusGroupId(), tsFileResource, targetDataNode); if (status.getCode() != TSStatusCode.SUCCESS_STATUS.getStatusCode()) { return status; } @@ -212,16 +211,26 @@ private TSStatus transferOneTarget( targetDataNode.getDataNodeId(), e); if (tracker.hasSentPieces) { - TSStatus rollbackStatus = - sendLoadCommand( - targetDataNode.getInternalEndPoint(), - buildLoadCommandReq(uuid, tsFileResource, LoadTsFileScheduler.LoadCommand.ROLLBACK)); - if (rollbackStatus.getCode() != TSStatusCode.SUCCESS_STATUS.getStatusCode()) { - rollbackStatus.setMessage( - rollbackStatus.getMessage() + try { + TSStatus rollbackStatus = + sendLoadCommand( + targetDataNode.getInternalEndPoint(), + buildLoadCommandReq( + uuid, tsFileResource, LoadTsFileScheduler.LoadCommand.ROLLBACK)); + if (rollbackStatus.getCode() != TSStatusCode.SUCCESS_STATUS.getStatusCode()) { + rollbackStatus.setMessage( + rollbackStatus.getMessage() + ", original transfer failure: " + e.getMessage()); + return rollbackStatus; + } + } catch (IOException rollbackBuildException) { + return RpcUtils.getStatus( + TSStatusCode.LOAD_FILE_ERROR, + "Failed to build rollback command for TsFile " + + tsFileResource.getTsFilePath() + + ": " + + rollbackBuildException.getMessage() + ", original transfer failure: " + e.getMessage()); - return rollbackStatus; } } return RpcUtils.getStatus( @@ -241,7 +250,7 @@ private void sendAllPieces( TConsensusGroupId consensusGroupId, TEndPoint targetEndPoint, TransferTracker tracker) - throws Exception { + throws IOException, LoadFileException { final LoadTsFilePieceNode[] pieceHolder = { new LoadTsFilePieceNode(new PlanNodeId("repair-tsfile-piece"), tsFile) }; @@ -271,20 +280,24 @@ private void dispatchPieceNode( String uuid, TConsensusGroupId consensusGroupId, LoadTsFilePieceNode pieceNode) - throws Exception { + throws LoadFileException { TTsFilePieceReq request = new TTsFilePieceReq(pieceNode.serializeToByteBuffer(), uuid, consensusGroupId); try (SyncDataNodeInternalServiceClient client = clientManager.borrowClient(targetEndPoint)) { TLoadResp response = client.sendTsFilePieceNode(request); if (!response.isAccepted()) { - throw new IllegalStateException( + throw new LoadFileException( response.isSetStatus() ? response.getStatus().getMessage() : response.getMessage()); } + } catch (LoadFileException e) { + throw e; + } catch (Exception e) { + throw new LoadFileException( + "Failed to dispatch TsFile piece to DataNode " + targetEndPoint, e); } } - private TSStatus sendLoadCommand( - TEndPoint targetEndPoint, TLoadCommandReq request) { + private TSStatus sendLoadCommand(TEndPoint targetEndPoint, TLoadCommandReq request) { try (SyncDataNodeInternalServiceClient client = clientManager.borrowClient(targetEndPoint)) { TLoadResp response = client.sendLoadCommand(request); if (response.isAccepted()) { diff --git a/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/consensus/iotv2/consistency/ConsistencyMerkleTree.java b/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/consensus/iotv2/consistency/ConsistencyMerkleTree.java index cd5d3ef3b3c5f..35e7935abe9c6 100644 --- a/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/consensus/iotv2/consistency/ConsistencyMerkleTree.java +++ b/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/consensus/iotv2/consistency/ConsistencyMerkleTree.java @@ -101,7 +101,9 @@ public void onCompaction( long partitionId, List sourceFileRootHashes, long targetFileRootHash) { onCompaction( partitionId, - sourceFileRootHashes.stream().map(DualDigest::fromSingleHash).collect(java.util.stream.Collectors.toList()), + sourceFileRootHashes.stream() + .map(DualDigest::fromSingleHash) + .collect(java.util.stream.Collectors.toList()), Collections.singletonList(DualDigest.fromSingleHash(targetFileRootHash))); } @@ -127,7 +129,9 @@ public void onCompaction( public void onCompaction(List sourceHashes, long targetCombinedHash, long partitionId) { onCompaction( partitionId, - sourceHashes.stream().map(DualDigest::fromSingleHash).collect(java.util.stream.Collectors.toList()), + sourceHashes.stream() + .map(DualDigest::fromSingleHash) + .collect(java.util.stream.Collectors.toList()), Collections.singletonList(DualDigest.fromSingleHash(targetCombinedHash))); } diff --git a/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/consensus/iotv2/consistency/repair/RepairSession.java b/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/consensus/iotv2/consistency/repair/RepairSession.java index 9b8fec71fc670..f795b576ed50d 100644 --- a/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/consensus/iotv2/consistency/repair/RepairSession.java +++ b/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/consensus/iotv2/consistency/repair/RepairSession.java @@ -45,7 +45,8 @@ public enum SessionState { } public interface RepairSessionApplier { - void apply(String sessionId, long partitionId, List inserts, List deletes) + void apply( + String sessionId, long partitionId, List inserts, List deletes) throws Exception; } @@ -57,7 +58,8 @@ public interface RepairSessionJournal { void delete(String sessionId); } - private static final RepairSessionApplier NO_OP_APPLIER = (sessionId, partitionId, inserts, deletes) -> {}; + private static final RepairSessionApplier NO_OP_APPLIER = + (sessionId, partitionId, inserts, deletes) -> {}; private static final RepairSessionJournal NO_OP_JOURNAL = new RepairSessionJournal() { diff --git a/iotdb-core/node-commons/src/test/java/org/apache/iotdb/commons/consensus/iotv2/consistency/ConsistencyMerkleTreeTest.java b/iotdb-core/node-commons/src/test/java/org/apache/iotdb/commons/consensus/iotv2/consistency/ConsistencyMerkleTreeTest.java index 016dcd2692a3c..c0ca0a9b0b3b5 100644 --- a/iotdb-core/node-commons/src/test/java/org/apache/iotdb/commons/consensus/iotv2/consistency/ConsistencyMerkleTreeTest.java +++ b/iotdb-core/node-commons/src/test/java/org/apache/iotdb/commons/consensus/iotv2/consistency/ConsistencyMerkleTreeTest.java @@ -36,7 +36,8 @@ public void shouldPreserveDualDigestOnFlush() { merkleTree.onTsFileFlushed(1L, first); merkleTree.onTsFileFlushed(1L, second); - Assert.assertEquals(new DualDigest(0x03L, 0x30L), merkleTree.getPartitionNode(1L).getPartitionDigest()); + Assert.assertEquals( + new DualDigest(0x03L, 0x30L), merkleTree.getPartitionNode(1L).getPartitionDigest()); Assert.assertEquals(new DualDigest(0x03L, 0x30L), merkleTree.getRegionDigest()); } @@ -50,7 +51,8 @@ public void shouldPreserveDualDigestOnCompaction() { merkleTree.onTsFileFlushed(7L, sourceOne); merkleTree.onTsFileFlushed(7L, sourceTwo); - merkleTree.onCompaction(Collections.singletonList(sourceOne), Arrays.asList(targetOne, targetTwo), 7L); + merkleTree.onCompaction( + Collections.singletonList(sourceOne), Arrays.asList(targetOne, targetTwo), 7L); Assert.assertEquals( new DualDigest(0x0EL, 0xE0L), merkleTree.getPartitionNode(7L).getPartitionDigest()); diff --git a/iotdb-core/node-commons/src/test/java/org/apache/iotdb/commons/consensus/iotv2/consistency/repair/RepairConflictResolverTest.java b/iotdb-core/node-commons/src/test/java/org/apache/iotdb/commons/consensus/iotv2/consistency/repair/RepairConflictResolverTest.java index 5ce0d21e285d7..7683764b8d515 100644 --- a/iotdb-core/node-commons/src/test/java/org/apache/iotdb/commons/consensus/iotv2/consistency/repair/RepairConflictResolverTest.java +++ b/iotdb-core/node-commons/src/test/java/org/apache/iotdb/commons/consensus/iotv2/consistency/repair/RepairConflictResolverTest.java @@ -43,8 +43,7 @@ public void leaderHasWithNewerFollowerDeletionShouldDeleteOnLeader() { RepairConflictResolver resolver = new RepairConflictResolver( Collections.emptyList(), - Collections.singletonList( - new ModEntrySummary("root.sg.d1", "s1", 0L, 200L, 20L))); + Collections.singletonList(new ModEntrySummary("root.sg.d1", "s1", 0L, 200L, 20L))); Assert.assertEquals( RepairAction.DELETE_ON_LEADER, @@ -65,8 +64,7 @@ public void followerHasWithoutNewerLeaderDeletionShouldSendToLeader() { public void followerHasWithNewerLeaderDeletionShouldDeleteOnFollower() { RepairConflictResolver resolver = new RepairConflictResolver( - Collections.singletonList( - new ModEntrySummary("root.sg.d1", "s1", 0L, 200L, 20L)), + Collections.singletonList(new ModEntrySummary("root.sg.d1", "s1", 0L, 200L, 20L)), Collections.emptyList()); Assert.assertEquals( From 7611ceb8d91ad29c898fcba54f19e4ff2d939df2 Mon Sep 17 00:00:00 2001 From: Peng Junzhi <201250214@smail.nju.edu.cn> Date: Fri, 20 Mar 2026 09:28:13 +0800 Subject: [PATCH 3/4] refactor: introduce point-level repair and tests --- .../cluster/config/MppConfigNodeConfig.java | 14 + .../remote/config/RemoteConfigNodeConfig.java | 10 + .../iotdb/itbase/env/ConfigNodeConfig.java | 4 + .../IoTDBIoTConsensusV23C3DBasicITBase.java | 712 ++++++- .../IoTDBIoTConsensusV2Batch3C3DBasicIT.java | 25 + .../IoTDBIoTConsensusV2Stream3C3DBasicIT.java | 25 + .../client/sync/CnToDnSyncRequestType.java | 9 +- .../client/sync/SyncDataNodeClientPool.java | 35 +- .../confignode/conf/ConfigNodeConfig.java | 20 + .../confignode/conf/ConfigNodeDescriptor.java | 12 + .../confignode/manager/ConfigManager.java | 75 +- .../confignode/manager/ProcedureManager.java | 39 +- .../ConsistencyCheckScheduler.java | 171 ++ .../ConsistencyProgressManager.java | 59 + .../persistence/ConsistencyProgressInfo.java | 124 ++ .../executor/ConfigPlanExecutor.java | 9 +- .../ConsistencyPartitionSelector.java | 135 ++ .../LiveDataRegionRepairExecutionContext.java | 1270 ++++++++---- .../consistency/RepairRegionProcedure.java | 532 +---- .../state/consistency/RepairState.java | 12 +- .../thrift/ConfigNodeRPCServiceProcessor.java | 6 + .../ConsistencyCheckSchedulerTest.java | 86 + .../ConsistencyProgressInfoTest.java | 60 + .../ConsistencyPartitionSelectorTest.java | 141 ++ .../RepairRegionProcedureTest.java | 808 ++------ .../IoTConsensusV2AsyncSink.java | 9 +- .../db/protocol/client/ConfigNodeClient.java | 7 + .../impl/DataNodeInternalRPCServiceImpl.java | 50 +- ...formationSchemaContentSupplierFactory.java | 64 + .../DataNodeLocationSupplierFactory.java | 1 + .../storageengine/dataregion/DataRegion.java | 58 + .../DataRegionConsistencyManager.java | 1781 +++++++++++++++++ .../DataRegionConsistencyRepairService.java | 710 ++++--- .../LogicalRepairSessionJournal.java | 308 +++ .../consistency/MerkleCompactionHook.java | 162 -- .../consistency/MerkleDeletionHook.java | 97 - .../consistency/MerkleFlushListener.java | 95 - .../consistency/MerkleHashComputer.java | 232 --- .../read/control/QueryResourceManager.java | 10 + .../dataregion/tsfile/TsFileManager.java | 11 + .../IoTConsensusV2AsyncSinkTest.java | 80 + .../DataNodeLocationSupplierFactoryTest.java} | 22 +- .../DataRegionConsistencyManagerTest.java | 444 ++++ .../LogicalRepairSessionJournalTest.java | 179 ++ .../ConsistencyCheckScheduler.java | 217 -- .../consistency/ConsistencyMerkleTree.java | 242 --- .../consistency/ConsistencyViewBuilder.java | 133 -- .../consistency/LogicalMismatchScope.java | 273 +++ .../consistency/RepairProgressTable.java | 780 ++++++-- .../consistency/TimePartitionMerkleNode.java | 129 -- .../consistency/ibf/CompositeKeyCodec.java | 88 - .../consistency/ibf/DataPointLocator.java | 71 - .../iotv2/consistency/ibf/DiffEntry.java | 59 - .../iotv2/consistency/ibf/IBFCell.java | 98 - .../consistency/ibf/IBFDecodeResult.java | 58 - .../ibf/InvertibleBloomFilter.java | 210 -- .../iotv2/consistency/ibf/RowRefIndex.java | 160 -- .../consistency/ibf/StrataEstimator.java | 108 - .../iotv2/consistency/merkle/MerkleEntry.java | 73 - .../consistency/merkle/MerkleFileCache.java | 124 -- .../consistency/merkle/MerkleFileContent.java | 82 - .../consistency/merkle/MerkleFileReader.java | 148 -- .../consistency/merkle/MerkleFileWriter.java | 136 -- .../repair/AdaptiveRepairExecutor.java | 204 -- .../consistency/repair/DiffAttribution.java | 111 - .../consistency/repair/ModEntrySummary.java | 97 - .../consistency/repair/RepairAction.java | 38 - .../repair/RepairConflictResolver.java | 145 -- .../consistency/repair/RepairCostModel.java | 119 -- .../iotv2/consistency/repair/RepairPlan.java | 80 - .../consistency/repair/RepairRecord.java | 132 -- .../consistency/repair/RepairSession.java | 197 -- .../consistency/repair/RepairStrategy.java | 28 - .../schema/column/ColumnHeaderConstant.java | 14 + .../schema/table/InformationSchema.java | 46 + .../ConsistencyMerkleTreeTest.java | 61 - .../consistency/LogicalMismatchScopeTest.java | 98 + .../consistency/RepairProgressTableTest.java | 129 ++ .../repair/DiffAttributionTest.java | 70 - .../repair/RepairConflictResolverTest.java | 74 - .../consistency/repair/RepairSessionTest.java | 80 - .../src/main/thrift/confignode.thrift | 28 + .../src/main/thrift/datanode.thrift | 169 +- replica_consistency_check_repair_review.md | 43 - 84 files changed, 7771 insertions(+), 6094 deletions(-) create mode 100644 iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/manager/consistency/ConsistencyCheckScheduler.java create mode 100644 iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/manager/consistency/ConsistencyProgressManager.java create mode 100644 iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/persistence/ConsistencyProgressInfo.java create mode 100644 iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/procedure/impl/consistency/ConsistencyPartitionSelector.java create mode 100644 iotdb-core/confignode/src/test/java/org/apache/iotdb/confignode/manager/consistency/ConsistencyCheckSchedulerTest.java create mode 100644 iotdb-core/confignode/src/test/java/org/apache/iotdb/confignode/persistence/ConsistencyProgressInfoTest.java create mode 100644 iotdb-core/confignode/src/test/java/org/apache/iotdb/confignode/procedure/impl/consistency/ConsistencyPartitionSelectorTest.java create mode 100644 iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/consistency/DataRegionConsistencyManager.java create mode 100644 iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/consistency/LogicalRepairSessionJournal.java delete mode 100644 iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/consistency/MerkleCompactionHook.java delete mode 100644 iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/consistency/MerkleDeletionHook.java delete mode 100644 iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/consistency/MerkleFlushListener.java delete mode 100644 iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/consistency/MerkleHashComputer.java create mode 100644 iotdb-core/datanode/src/test/java/org/apache/iotdb/db/pipe/sink/protocol/iotconsensusv2/IoTConsensusV2AsyncSinkTest.java rename iotdb-core/{node-commons/src/main/java/org/apache/iotdb/commons/consensus/iotv2/consistency/StaleSnapshotException.java => datanode/src/test/java/org/apache/iotdb/db/queryengine/plan/relational/planner/optimizations/DataNodeLocationSupplierFactoryTest.java} (61%) create mode 100644 iotdb-core/datanode/src/test/java/org/apache/iotdb/db/storageengine/dataregion/consistency/DataRegionConsistencyManagerTest.java create mode 100644 iotdb-core/datanode/src/test/java/org/apache/iotdb/db/storageengine/dataregion/consistency/LogicalRepairSessionJournalTest.java delete mode 100644 iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/consensus/iotv2/consistency/ConsistencyCheckScheduler.java delete mode 100644 iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/consensus/iotv2/consistency/ConsistencyMerkleTree.java delete mode 100644 iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/consensus/iotv2/consistency/ConsistencyViewBuilder.java create mode 100644 iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/consensus/iotv2/consistency/LogicalMismatchScope.java delete mode 100644 iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/consensus/iotv2/consistency/TimePartitionMerkleNode.java delete mode 100644 iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/consensus/iotv2/consistency/ibf/CompositeKeyCodec.java delete mode 100644 iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/consensus/iotv2/consistency/ibf/DataPointLocator.java delete mode 100644 iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/consensus/iotv2/consistency/ibf/DiffEntry.java delete mode 100644 iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/consensus/iotv2/consistency/ibf/IBFCell.java delete mode 100644 iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/consensus/iotv2/consistency/ibf/IBFDecodeResult.java delete mode 100644 iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/consensus/iotv2/consistency/ibf/InvertibleBloomFilter.java delete mode 100644 iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/consensus/iotv2/consistency/ibf/RowRefIndex.java delete mode 100644 iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/consensus/iotv2/consistency/ibf/StrataEstimator.java delete mode 100644 iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/consensus/iotv2/consistency/merkle/MerkleEntry.java delete mode 100644 iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/consensus/iotv2/consistency/merkle/MerkleFileCache.java delete mode 100644 iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/consensus/iotv2/consistency/merkle/MerkleFileContent.java delete mode 100644 iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/consensus/iotv2/consistency/merkle/MerkleFileReader.java delete mode 100644 iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/consensus/iotv2/consistency/merkle/MerkleFileWriter.java delete mode 100644 iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/consensus/iotv2/consistency/repair/AdaptiveRepairExecutor.java delete mode 100644 iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/consensus/iotv2/consistency/repair/DiffAttribution.java delete mode 100644 iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/consensus/iotv2/consistency/repair/ModEntrySummary.java delete mode 100644 iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/consensus/iotv2/consistency/repair/RepairAction.java delete mode 100644 iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/consensus/iotv2/consistency/repair/RepairConflictResolver.java delete mode 100644 iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/consensus/iotv2/consistency/repair/RepairCostModel.java delete mode 100644 iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/consensus/iotv2/consistency/repair/RepairPlan.java delete mode 100644 iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/consensus/iotv2/consistency/repair/RepairRecord.java delete mode 100644 iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/consensus/iotv2/consistency/repair/RepairSession.java delete mode 100644 iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/consensus/iotv2/consistency/repair/RepairStrategy.java delete mode 100644 iotdb-core/node-commons/src/test/java/org/apache/iotdb/commons/consensus/iotv2/consistency/ConsistencyMerkleTreeTest.java create mode 100644 iotdb-core/node-commons/src/test/java/org/apache/iotdb/commons/consensus/iotv2/consistency/LogicalMismatchScopeTest.java create mode 100644 iotdb-core/node-commons/src/test/java/org/apache/iotdb/commons/consensus/iotv2/consistency/RepairProgressTableTest.java delete mode 100644 iotdb-core/node-commons/src/test/java/org/apache/iotdb/commons/consensus/iotv2/consistency/repair/DiffAttributionTest.java delete mode 100644 iotdb-core/node-commons/src/test/java/org/apache/iotdb/commons/consensus/iotv2/consistency/repair/RepairConflictResolverTest.java delete mode 100644 iotdb-core/node-commons/src/test/java/org/apache/iotdb/commons/consensus/iotv2/consistency/repair/RepairSessionTest.java delete mode 100644 replica_consistency_check_repair_review.md diff --git a/integration-test/src/main/java/org/apache/iotdb/it/env/cluster/config/MppConfigNodeConfig.java b/integration-test/src/main/java/org/apache/iotdb/it/env/cluster/config/MppConfigNodeConfig.java index 4385097ce045b..cfca2502cb38a 100644 --- a/integration-test/src/main/java/org/apache/iotdb/it/env/cluster/config/MppConfigNodeConfig.java +++ b/integration-test/src/main/java/org/apache/iotdb/it/env/cluster/config/MppConfigNodeConfig.java @@ -73,4 +73,18 @@ public ConfigNodeConfig setLeaderDistributionPolicy(String policy) { properties.setProperty("leader_distribution_policy", policy); return this; } + + @Override + public ConfigNodeConfig setConsistencyCheckSchedulerInitialDelayInMs(long initialDelayInMs) { + properties.setProperty( + "consistency_check_scheduler_initial_delay_in_ms", String.valueOf(initialDelayInMs)); + return this; + } + + @Override + public ConfigNodeConfig setConsistencyCheckSchedulerIntervalInMs(long intervalInMs) { + properties.setProperty( + "consistency_check_scheduler_interval_in_ms", String.valueOf(intervalInMs)); + return this; + } } diff --git a/integration-test/src/main/java/org/apache/iotdb/it/env/remote/config/RemoteConfigNodeConfig.java b/integration-test/src/main/java/org/apache/iotdb/it/env/remote/config/RemoteConfigNodeConfig.java index c16722f4bd943..387020363cefb 100644 --- a/integration-test/src/main/java/org/apache/iotdb/it/env/remote/config/RemoteConfigNodeConfig.java +++ b/integration-test/src/main/java/org/apache/iotdb/it/env/remote/config/RemoteConfigNodeConfig.java @@ -43,4 +43,14 @@ public ConfigNodeConfig setMetricPrometheusReporterPassword(String password) { public ConfigNodeConfig setLeaderDistributionPolicy(String policy) { return this; } + + @Override + public ConfigNodeConfig setConsistencyCheckSchedulerInitialDelayInMs(long initialDelayInMs) { + return this; + } + + @Override + public ConfigNodeConfig setConsistencyCheckSchedulerIntervalInMs(long intervalInMs) { + return this; + } } diff --git a/integration-test/src/main/java/org/apache/iotdb/itbase/env/ConfigNodeConfig.java b/integration-test/src/main/java/org/apache/iotdb/itbase/env/ConfigNodeConfig.java index 4af35f6f56a9d..60431ddba13f0 100644 --- a/integration-test/src/main/java/org/apache/iotdb/itbase/env/ConfigNodeConfig.java +++ b/integration-test/src/main/java/org/apache/iotdb/itbase/env/ConfigNodeConfig.java @@ -31,4 +31,8 @@ public interface ConfigNodeConfig { ConfigNodeConfig setMetricPrometheusReporterPassword(String password); ConfigNodeConfig setLeaderDistributionPolicy(String policy); + + ConfigNodeConfig setConsistencyCheckSchedulerInitialDelayInMs(long initialDelayInMs); + + ConfigNodeConfig setConsistencyCheckSchedulerIntervalInMs(long intervalInMs); } diff --git a/integration-test/src/test/java/org/apache/iotdb/db/it/iotconsensusv2/IoTDBIoTConsensusV23C3DBasicITBase.java b/integration-test/src/test/java/org/apache/iotdb/db/it/iotconsensusv2/IoTDBIoTConsensusV23C3DBasicITBase.java index 34a4b67bec3fd..a44f1ba020ec1 100644 --- a/integration-test/src/test/java/org/apache/iotdb/db/it/iotconsensusv2/IoTDBIoTConsensusV23C3DBasicITBase.java +++ b/integration-test/src/test/java/org/apache/iotdb/db/it/iotconsensusv2/IoTDBIoTConsensusV23C3DBasicITBase.java @@ -21,8 +21,12 @@ import org.apache.iotdb.common.rpc.thrift.TConsensusGroupId; import org.apache.iotdb.common.rpc.thrift.TConsensusGroupType; +import org.apache.iotdb.common.rpc.thrift.TEndPoint; import org.apache.iotdb.common.rpc.thrift.TSStatus; +import org.apache.iotdb.commons.client.ClientPoolFactory; +import org.apache.iotdb.commons.client.IClientManager; import org.apache.iotdb.commons.client.sync.SyncConfigNodeIServiceClient; +import org.apache.iotdb.commons.client.sync.SyncDataNodeInternalServiceClient; import org.apache.iotdb.confignode.it.regionmigration.IoTDBRegionOperationReliabilityITFramework; import org.apache.iotdb.confignode.rpc.thrift.TTriggerRegionConsistencyRepairReq; import org.apache.iotdb.consensus.ConsensusFactory; @@ -33,6 +37,9 @@ import org.apache.iotdb.it.env.EnvFactory; import org.apache.iotdb.it.env.cluster.node.DataNodeWrapper; import org.apache.iotdb.itbase.env.BaseEnv; +import org.apache.iotdb.mpp.rpc.thrift.TGetConsistencyEligibilityReq; +import org.apache.iotdb.mpp.rpc.thrift.TGetConsistencyEligibilityResp; +import org.apache.iotdb.mpp.rpc.thrift.TPartitionConsistencyEligibility; import org.apache.tsfile.utils.Pair; import org.awaitility.Awaitility; @@ -49,13 +56,14 @@ import java.sql.ResultSet; import java.sql.SQLException; import java.sql.Statement; +import java.util.ArrayList; import java.util.Collections; +import java.util.Comparator; +import java.util.List; import java.util.Map; import java.util.Optional; import java.util.Set; import java.util.concurrent.TimeUnit; -import java.util.regex.Matcher; -import java.util.regex.Pattern; import java.util.stream.Stream; import static org.apache.iotdb.util.MagicUtils.makeItCloseQuietly; @@ -80,9 +88,19 @@ public abstract class IoTDBIoTConsensusV23C3DBasicITBase protected static final int DATA_REPLICATION_FACTOR = 2; protected static final int SCHEMA_REPLICATION_FACTOR = 3; + private static final IClientManager + DATA_NODE_INTERNAL_CLIENT_MANAGER = + new IClientManager.Factory() + .createClientManager( + new ClientPoolFactory.SyncDataNodeInternalServiceClientPoolFactory()); + /** Timeout in seconds for 3C3D cluster init. */ protected static final int CLUSTER_INIT_TIMEOUT_SECONDS = 300; + protected static final long TIME_PARTITION_INTERVAL = 100L; + protected static final long CONSISTENCY_CHECK_INITIAL_DELAY_MS = 1_000L; + protected static final long CONSISTENCY_CHECK_INTERVAL_MS = 1_000L; + protected static final String INSERTION1 = "INSERT INTO root.sg.d1(timestamp,speed,temperature) values(100, 1, 2)"; protected static final String INSERTION2 = @@ -90,6 +108,7 @@ public abstract class IoTDBIoTConsensusV23C3DBasicITBase protected static final String INSERTION3 = "INSERT INTO root.sg.d1(timestamp,speed,temperature) values(102, 5, 6)"; protected static final String FLUSH_COMMAND = "flush on cluster"; + protected static final String LOCAL_FLUSH_COMMAND = "flush on local"; protected static final String COUNT_QUERY = "select count(*) from root.sg.**"; protected static final String DELETE_SPEED_UP_TO_101 = "DELETE FROM root.sg.d1.speed WHERE time <= 101"; @@ -112,6 +131,7 @@ public void setUp() throws Exception { .getCommonConfig() .setDataReplicationFactor(DATA_REPLICATION_FACTOR) .setSchemaReplicationFactor(SCHEMA_REPLICATION_FACTOR) + .setTimePartitionInterval(TIME_PARTITION_INTERVAL) .setIoTConsensusV2Mode(getIoTConsensusV2Mode()); EnvFactory.getEnv() @@ -119,6 +139,12 @@ public void setUp() throws Exception { .getDataNodeConfig() .setMetricReporterType(Collections.singletonList("PROMETHEUS")); + EnvFactory.getEnv() + .getConfig() + .getConfigNodeConfig() + .setConsistencyCheckSchedulerInitialDelayInMs(CONSISTENCY_CHECK_INITIAL_DELAY_MS) + .setConsistencyCheckSchedulerIntervalInMs(CONSISTENCY_CHECK_INTERVAL_MS); + EnvFactory.getEnv() .initClusterEnvironment(CONFIG_NODE_NUM, DATA_NODE_NUM, CLUSTER_INIT_TIMEOUT_SECONDS); } @@ -146,7 +172,8 @@ public void testFollowerCanReadConsistentDataAfterCatchUp() throws Exception { verifyDataConsistency(statement); RegionReplicaSelection regionReplicaSelection = selectReplicatedDataRegion(statement); - waitForReplicationComplete(regionReplicaSelection.leaderNode); + waitForReplicationComplete( + regionReplicaSelection.leaderNode, regionReplicaSelection.regionId); LOGGER.info( "Verifying logical view from follower DataNode {} for region {} after catch-up", @@ -169,7 +196,8 @@ public void testReplicaConsistencyAfterLeaderStop() throws Exception { verifyDataConsistency(statement); regionReplicaSelection = selectReplicatedDataRegion(statement); - waitForReplicationComplete(regionReplicaSelection.leaderNode); + waitForReplicationComplete( + regionReplicaSelection.leaderNode, regionReplicaSelection.regionId); } LOGGER.info( @@ -212,7 +240,8 @@ public void testReplicaConsistencyAfterDeleteAndLeaderStop() throws Exception { statement.execute(FLUSH_COMMAND); verifyPostDeleteConsistency(statement); - waitForReplicationComplete(regionReplicaSelection.leaderNode); + waitForReplicationComplete( + regionReplicaSelection.leaderNode, regionReplicaSelection.regionId); verifyPostDeleteConsistencyOnNode(regionReplicaSelection.followerNode); } @@ -234,48 +263,259 @@ public void testReplicaConsistencyAfterDeleteAndLeaderStop() throws Exception { } /** - * Simulate a follower missing a sealed TsFile, trigger replica consistency repair through - * ConfigNode, and verify the repaired follower still serves the correct data after the leader is - * stopped. + * Background consistency check should skip hot partitions before flush and only verify them after + * they become cold and safe. + */ + public void testBackgroundConsistencyCheckOnlyRunsOnColdPartitions() throws Exception { + RegionReplicaSelection regionReplicaSelection; + long partitionId = timePartitionId(100L); + + try (Connection connection = makeItCloseQuietly(EnvFactory.getEnv().getConnection()); + Statement statement = makeItCloseQuietly(connection.createStatement())) { + insertPartitionData(statement, 100L); + regionReplicaSelection = selectReplicatedDataRegion(statement); + + TimeUnit.MILLISECONDS.sleep( + CONSISTENCY_CHECK_INITIAL_DELAY_MS + CONSISTENCY_CHECK_INTERVAL_MS * 2); + assertRepairProgressEmpty(regionReplicaSelection.regionId); + + statement.execute(FLUSH_COMMAND); + waitForReplicationComplete( + regionReplicaSelection.leaderNode, regionReplicaSelection.regionId); + } + + RepairProgressRow row = + waitForCheckState(regionReplicaSelection.regionId, partitionId, "VERIFIED"); + Assert.assertEquals("IDLE", row.repairState); + } + + /** + * Restarting a follower should allow the background checker to rebuild its logical snapshot and + * keep the partition verified after the node rejoins. + */ + public void testBackgroundConsistencyCheckRebuildsLogicalSnapshotAfterFollowerRestart() + throws Exception { + RegionReplicaSelection regionReplicaSelection; + long partitionId = timePartitionId(100L); + + try (Connection connection = makeItCloseQuietly(EnvFactory.getEnv().getConnection()); + Statement statement = makeItCloseQuietly(connection.createStatement())) { + insertAndFlushTestData(statement); + regionReplicaSelection = selectReplicatedDataRegion(statement); + waitForReplicationComplete( + regionReplicaSelection.leaderNode, regionReplicaSelection.regionId); + waitForCheckState(regionReplicaSelection.regionId, partitionId, "VERIFIED"); + } + + RepairProgressRow previousRow = + getRepairProgressRow(regionReplicaSelection.regionId, partitionId); + Assert.assertNotNull(previousRow); + + regionReplicaSelection.followerNode.stopForcibly(); + Assert.assertFalse("Follower should be stopped", regionReplicaSelection.followerNode.isAlive()); + + regionReplicaSelection.followerNode.start(); + waitForNodeConnectionReady(regionReplicaSelection.followerNode); + waitForProgressRefresh(regionReplicaSelection.regionId, partitionId, previousRow.lastCheckedAt); + RepairProgressRow refreshedRow = + waitForCheckState(regionReplicaSelection.regionId, partitionId, "VERIFIED"); + Assert.assertEquals("READY", refreshedRow.snapshotState); + Assert.assertTrue(refreshedRow.snapshotEpoch >= previousRow.snapshotEpoch); + } + + /** + * Background check must not advance progress while the leader reports non-zero sync lag. Once the + * lagging follower catches up again, the same partition can be checked and verified in a new + * round. + */ + public void testBackgroundConsistencyCheckWaitsForSyncLagToClear() throws Exception { + RegionReplicaSelection regionReplicaSelection; + long partitionId = timePartitionId(100L); + RepairProgressRow baselineRow; + + try (Connection connection = makeItCloseQuietly(EnvFactory.getEnv().getConnection()); + Statement statement = makeItCloseQuietly(connection.createStatement())) { + insertAndFlushTestData(statement); + regionReplicaSelection = selectReplicatedDataRegion(statement); + waitForReplicationComplete( + regionReplicaSelection.leaderNode, regionReplicaSelection.regionId); + baselineRow = waitForCheckState(regionReplicaSelection.regionId, partitionId, "VERIFIED"); + } + + regionReplicaSelection.followerNode.stopForcibly(); + Assert.assertFalse("Follower should be stopped", regionReplicaSelection.followerNode.isAlive()); + + try (Connection connection = + makeItCloseQuietly( + EnvFactory.getEnv() + .getConnection( + regionReplicaSelection.leaderNode, + SessionConfig.DEFAULT_USER, + SessionConfig.DEFAULT_PASSWORD, + BaseEnv.TREE_SQL_DIALECT)); + Statement statement = makeItCloseQuietly(connection.createStatement())) { + insertPartitionData(statement, 103L); + if (shouldFlushLaggingWritesBeforeFollowerRestart()) { + statement.execute(LOCAL_FLUSH_COMMAND); + } + } + + waitForReplicationLag(regionReplicaSelection.leaderNode, regionReplicaSelection.regionId); + assertDataPointCountOnNode(regionReplicaSelection.leaderNode, 12L); + + TimeUnit.MILLISECONDS.sleep(CONSISTENCY_CHECK_INTERVAL_MS * 3); + RepairProgressRow laggingRow = + getRepairProgressRow( + regionReplicaSelection.leaderNode, regionReplicaSelection.regionId, partitionId); + Assert.assertNotNull(laggingRow); + Assert.assertEquals( + "Background checker should keep the last verified result while syncLag > 0", + "VERIFIED", + laggingRow.checkState); + Assert.assertEquals( + "Background checker should not advance last_checked_at while syncLag > 0", + baselineRow.lastCheckedAt, + laggingRow.lastCheckedAt); + Assert.assertEquals( + "Background checker should not persist a new mutation epoch while syncLag > 0", + baselineRow.partitionMutationEpoch, + laggingRow.partitionMutationEpoch); + Assert.assertEquals( + "Background checker should not persist a new snapshot epoch while syncLag > 0", + baselineRow.snapshotEpoch, + laggingRow.snapshotEpoch); + + regionReplicaSelection.followerNode.start(); + waitForNodeConnectionReady(regionReplicaSelection.followerNode); + waitForReplicationComplete(regionReplicaSelection.leaderNode, regionReplicaSelection.regionId); + if (!shouldFlushLaggingWritesBeforeFollowerRestart()) { + localFlushOnNode(regionReplicaSelection.leaderNode); + waitForReplicationComplete( + regionReplicaSelection.leaderNode, regionReplicaSelection.regionId); + } + waitForProgressRefresh(regionReplicaSelection.regionId, partitionId, baselineRow.lastCheckedAt); + RepairProgressRow refreshedRow = + waitForCheckState(regionReplicaSelection.regionId, partitionId, "VERIFIED"); + Assert.assertTrue( + "Expected a new mutation epoch after the lagging writes are replicated", + refreshedRow.partitionMutationEpoch > baselineRow.partitionMutationEpoch); + Assert.assertTrue( + "Expected a rebuilt snapshot epoch after sync lag is cleared", + refreshedRow.snapshotEpoch > baselineRow.snapshotEpoch); + assertDataPointCountOnNode(regionReplicaSelection.followerNode, 12L); + } + + /** + * Verified progress should remain queryable after the current ConfigNode leader is restarted. + * This validates that progress is durably persisted rather than kept only in the live leader's + * memory. + */ + public void testRepairProgressSurvivesConfigNodeLeaderRestart() throws Exception { + RegionReplicaSelection regionReplicaSelection; + long partitionId = timePartitionId(100L); + RepairProgressRow baselineRow; + + try (Connection connection = makeItCloseQuietly(EnvFactory.getEnv().getConnection()); + Statement statement = makeItCloseQuietly(connection.createStatement())) { + insertAndFlushTestData(statement); + regionReplicaSelection = selectReplicatedDataRegion(statement); + waitForReplicationComplete( + regionReplicaSelection.leaderNode, regionReplicaSelection.regionId); + baselineRow = waitForCheckState(regionReplicaSelection.regionId, partitionId, "VERIFIED"); + } + + int leaderConfigNodeIndex = EnvFactory.getEnv().getLeaderConfigNodeIndex(); + EnvFactory.getEnv().getConfigNodeWrapperList().get(leaderConfigNodeIndex).stopForcibly(); + + Awaitility.await() + .pollDelay(1, TimeUnit.SECONDS) + .pollInterval(1, TimeUnit.SECONDS) + .atMost(2, TimeUnit.MINUTES) + .untilAsserted( + () -> { + RepairProgressRow row = + getRepairProgressRow(regionReplicaSelection.regionId, partitionId); + Assert.assertNotNull(row); + Assert.assertEquals("VERIFIED", row.checkState); + Assert.assertTrue(row.lastCheckedAt >= baselineRow.lastCheckedAt); + Assert.assertTrue(row.snapshotEpoch >= baselineRow.snapshotEpoch); + }); + + EnvFactory.getEnv().getConfigNodeWrapperList().get(leaderConfigNodeIndex).start(); + + Awaitility.await() + .pollDelay(1, TimeUnit.SECONDS) + .pollInterval(1, TimeUnit.SECONDS) + .atMost(2, TimeUnit.MINUTES) + .untilAsserted( + () -> { + RepairProgressRow row = + getRepairProgressRow(regionReplicaSelection.regionId, partitionId); + Assert.assertNotNull(row); + Assert.assertEquals("VERIFIED", row.checkState); + Assert.assertTrue(row.lastCheckedAt >= baselineRow.lastCheckedAt); + Assert.assertTrue(row.snapshotEpoch >= baselineRow.snapshotEpoch); + }); + } + + /** + * Background check should only mark mismatches. Manual repair should then consume only the + * mismatched partition scope and restore the follower. */ public void testReplicaConsistencyRepairAfterFollowerLosesSealedTsFile() throws Exception { RegionReplicaSelection regionReplicaSelection; + long firstPartitionId = timePartitionId(100L); + long secondPartitionId = timePartitionId(200L); Path deletedTsFile; try (Connection connection = makeItCloseQuietly(EnvFactory.getEnv().getConnection()); Statement statement = makeItCloseQuietly(connection.createStatement())) { - insertAndFlushTestData(statement); - verifyDataConsistency(statement); + insertAndFlushPartitionData(statement, 100L); + insertAndFlushPartitionData(statement, 200L); regionReplicaSelection = selectReplicatedDataRegion(statement); - waitForReplicationComplete(regionReplicaSelection.leaderNode); + waitForReplicationComplete( + regionReplicaSelection.leaderNode, regionReplicaSelection.regionId); + waitForCheckState(regionReplicaSelection.regionId, firstPartitionId, "VERIFIED"); + waitForCheckState(regionReplicaSelection.regionId, secondPartitionId, "VERIFIED"); deletedTsFile = findLatestSealedTsFile( - regionReplicaSelection.followerNode, regionReplicaSelection.regionId); + regionReplicaSelection.followerNode, + regionReplicaSelection.regionId, + secondPartitionId); } - LOGGER.info( - "Stopping follower DataNode {} and deleting sealed TsFile {} for region {}", - regionReplicaSelection.followerDataNodeId, - deletedTsFile, - regionReplicaSelection.regionId); regionReplicaSelection.followerNode.stopForcibly(); Assert.assertFalse("Follower should be stopped", regionReplicaSelection.followerNode.isAlive()); deleteTsFileArtifacts(deletedTsFile); regionReplicaSelection.followerNode.start(); waitForNodeConnectionReady(regionReplicaSelection.followerNode); - Awaitility.await() - .pollDelay(2, TimeUnit.SECONDS) - .atMost(2, TimeUnit.MINUTES) - .untilAsserted(() -> assertDataInconsistentOnNode(regionReplicaSelection.followerNode)); + waitForCheckState(regionReplicaSelection.regionId, secondPartitionId, "MISMATCH"); + assertPartitionViewMismatch(regionReplicaSelection); + assertDataPointCountOnNode(regionReplicaSelection.followerNode, 6L); + + RepairProgressRow firstPartition = + getRepairProgressRow(regionReplicaSelection.regionId, firstPartitionId); + RepairProgressRow secondPartition = + getRepairProgressRow(regionReplicaSelection.regionId, secondPartitionId); + Assert.assertNotNull(firstPartition); + Assert.assertNotNull(secondPartition); + Assert.assertEquals("VERIFIED", firstPartition.checkState); + Assert.assertEquals("MISMATCH", secondPartition.checkState); + + TimeUnit.MILLISECONDS.sleep(CONSISTENCY_CHECK_INTERVAL_MS * 3); + Assert.assertEquals( + "Background checker should stay check-only and not auto-repair mismatches", + "MISMATCH", + getRepairProgressRow(regionReplicaSelection.regionId, secondPartitionId).checkState); + assertDataPointCountOnNode(regionReplicaSelection.followerNode, 6L); triggerRegionConsistencyRepair(regionReplicaSelection.regionId); - Awaitility.await() - .pollDelay(2, TimeUnit.SECONDS) - .atMost(2, TimeUnit.MINUTES) - .untilAsserted(() -> verifyDataConsistencyOnNode(regionReplicaSelection.followerNode)); + waitForCheckState(regionReplicaSelection.regionId, secondPartitionId, "VERIFIED"); + assertPartitionViewMatched(regionReplicaSelection); + assertDataPointCountOnNode(regionReplicaSelection.followerNode, 12L); LOGGER.info( "Stopping leader DataNode {} after repair to verify repaired follower serves local data", @@ -286,19 +526,47 @@ public void testReplicaConsistencyRepairAfterFollowerLosesSealedTsFile() throws Awaitility.await() .pollDelay(2, TimeUnit.SECONDS) .atMost(2, TimeUnit.MINUTES) - .untilAsserted(() -> verifyDataConsistencyOnNode(regionReplicaSelection.followerNode)); + .untilAsserted(() -> assertDataPointCountOnNode(regionReplicaSelection.followerNode, 12L)); } protected void insertAndFlushTestData(Statement statement) throws Exception { - LOGGER.info("Writing data to 3C3D cluster (mode: {})...", getIoTConsensusV2Mode()); - statement.execute(INSERTION1); - statement.execute(INSERTION2); - statement.execute(INSERTION3); + insertAndFlushPartitionData(statement, 100L); + } - LOGGER.info("Executing flush on cluster..."); + protected void insertAndFlushPartitionData(Statement statement, long baseTimestamp) + throws Exception { + insertPartitionData(statement, baseTimestamp); statement.execute(FLUSH_COMMAND); } + protected void insertPartitionData(Statement statement, long baseTimestamp) throws Exception { + LOGGER.info( + "Writing partition-scoped data at baseTimestamp={} to 3C3D cluster (mode: {})...", + baseTimestamp, + getIoTConsensusV2Mode()); + statement.execute(insertSql(baseTimestamp, 1L, 2L)); + statement.execute(insertSql(baseTimestamp + 1, 3L, 4L)); + statement.execute(insertSql(baseTimestamp + 2, 5L, 6L)); + } + + protected boolean shouldFlushLaggingWritesBeforeFollowerRestart() { + return ConsensusFactory.IOT_CONSENSUS_V2_BATCH_MODE.equals(getIoTConsensusV2Mode()); + } + + protected void localFlushOnNode(DataNodeWrapper targetNode) throws Exception { + try (Connection connection = + makeItCloseQuietly( + EnvFactory.getEnv() + .getConnection( + targetNode, + SessionConfig.DEFAULT_USER, + SessionConfig.DEFAULT_PASSWORD, + BaseEnv.TREE_SQL_DIALECT)); + Statement statement = makeItCloseQuietly(connection.createStatement())) { + statement.execute(LOCAL_FLUSH_COMMAND); + } + } + protected RegionReplicaSelection selectReplicatedDataRegion(Statement statement) throws Exception { Map>> dataRegionMap = getDataRegionMapWithLeader(statement); @@ -363,42 +631,57 @@ protected void verifyPostDeleteConsistencyOnNode(DataNodeWrapper targetNode) thr } } - private static final Pattern SYNC_LAG_PATTERN = - Pattern.compile("iot_consensus_v2\\{[^}]*type=\"syncLag\"[^}]*}\\s+(\\S+)"); - /** - * Wait until all consensus pipe syncLag metrics on the given leader DataNode reach 0, meaning - * replication is fully caught up. Queries the leader's Prometheus metrics endpoint periodically. + * Wait until the target region's eligibility reports syncLag == 0, matching the production + * scheduler's gating logic. */ - protected void waitForReplicationComplete(DataNodeWrapper leaderNode) { + protected void waitForReplicationComplete(DataNodeWrapper leaderNode, int regionId) { final long timeoutSeconds = 120; - final String metricsUrl = - "http://" + leaderNode.getIp() + ":" + leaderNode.getMetricPort() + "/metrics"; LOGGER.info( - "Waiting for consensus pipe syncLag to reach 0 on leader DataNode (url: {}, timeout: {}s)...", - metricsUrl, + "Waiting for region {} syncLag to reach 0 on leader DataNode {} (timeout: {}s)...", + regionId, + leaderNode.getId(), timeoutSeconds); Awaitility.await() .pollInterval(500, TimeUnit.MILLISECONDS) .atMost(timeoutSeconds, TimeUnit.SECONDS) .untilAsserted( () -> { - String metricsContent = EnvFactory.getEnv().getUrlContent(metricsUrl, null); - Assert.assertNotNull( - "Failed to fetch metrics from leader DataNode at " + metricsUrl, metricsContent); - Matcher matcher = SYNC_LAG_PATTERN.matcher(metricsContent); - boolean found = false; - while (matcher.find()) { - found = true; - double syncLag = Double.parseDouble(matcher.group(1)); - LOGGER.debug("Found syncLag metric value: {}", syncLag); - Assert.assertEquals( - "Consensus pipe syncLag should be 0.0 but was " + syncLag, 0.0, syncLag, 0.001); - } + long syncLag = getConsistencyEligibility(leaderNode, regionId).getSyncLag(); + LOGGER.debug( + "Observed region {} syncLag={} on leader DataNode {}", + regionId, + syncLag, + leaderNode.getId()); + Assert.assertEquals( + "Region " + regionId + " syncLag should be 0 but was " + syncLag, 0L, syncLag); + }); + LOGGER.info("Region {} syncLag == 0 on leader, replication is complete", regionId); + } + + protected void waitForReplicationLag(DataNodeWrapper leaderNode, int regionId) { + final long timeoutSeconds = 60; + LOGGER.info( + "Waiting for region {} syncLag to become positive on leader DataNode {} (timeout: {}s)...", + regionId, + leaderNode.getId(), + timeoutSeconds); + Awaitility.await() + .pollInterval(500, TimeUnit.MILLISECONDS) + .atMost(timeoutSeconds, TimeUnit.SECONDS) + .untilAsserted( + () -> { + long syncLag = getConsistencyEligibility(leaderNode, regionId).getSyncLag(); + LOGGER.debug( + "Observed region {} syncLag={} while waiting for lag on leader DataNode {}", + regionId, + syncLag, + leaderNode.getId()); Assert.assertTrue( - "No iot_consensus_v2 syncLag metric found in leader DataNode metrics", found); + "Expected region " + regionId + " syncLag > 0 while follower is lagging", + syncLag > 0L); }); - LOGGER.info("All consensus pipe syncLag == 0 on leader, replication is complete"); + LOGGER.info("Observed region {} syncLag > 0 on leader while follower is lagging", regionId); } protected void verifyDataConsistency(Statement statement) throws Exception { @@ -437,6 +720,20 @@ protected void verifyDataConsistency(Statement statement) throws Exception { Assert.assertEquals("Expected 3 rows from select *", 3, rowCount); } + protected void verifyDataPointCount(Statement statement, long expectedTotalCount) + throws Exception { + try (ResultSet countResult = statement.executeQuery(COUNT_QUERY)) { + Assert.assertTrue("Count query should return results", countResult.next()); + + int columnCount = countResult.getMetaData().getColumnCount(); + long totalCount = 0; + for (int i = 1; i <= columnCount; i++) { + totalCount += parseLongFromString(countResult.getString(i)); + } + Assert.assertEquals("Unexpected total data point count", expectedTotalCount, totalCount); + } + } + protected void verifyPostDeleteConsistency(Statement statement) throws Exception { LOGGER.info("Querying data to verify replicated delete success..."); try (ResultSet countResult = statement.executeQuery(COUNT_AFTER_DELETE_QUERY)) { @@ -494,6 +791,21 @@ protected void assertDataInconsistentOnNode(DataNodeWrapper targetNode) throws E } } + protected void assertDataPointCountOnNode(DataNodeWrapper targetNode, long expectedTotalCount) + throws Exception { + try (Connection targetConnection = + makeItCloseQuietly( + EnvFactory.getEnv() + .getConnection( + targetNode, + SessionConfig.DEFAULT_USER, + SessionConfig.DEFAULT_PASSWORD, + BaseEnv.TREE_SQL_DIALECT)); + Statement targetStatement = makeItCloseQuietly(targetConnection.createStatement())) { + verifyDataPointCount(targetStatement, expectedTotalCount); + } + } + protected void waitForNodeConnectionReady(DataNodeWrapper targetNode) { Awaitility.await() .pollDelay(1, TimeUnit.SECONDS) @@ -559,6 +871,11 @@ private RegionReplicaSelection( private Path findLatestSealedTsFile(DataNodeWrapper dataNodeWrapper, int regionId) throws Exception { + return findLatestSealedTsFile(dataNodeWrapper, regionId, null); + } + + private Path findLatestSealedTsFile( + DataNodeWrapper dataNodeWrapper, int regionId, Long timePartitionId) throws Exception { try (Stream tsFiles = Files.walk(Paths.get(dataNodeWrapper.getDataPath()))) { Optional candidate = tsFiles @@ -566,6 +883,8 @@ private Path findLatestSealedTsFile(DataNodeWrapper dataNodeWrapper, int regionI .filter(path -> path.getFileName().toString().endsWith(".tsfile")) .filter(path -> path.toString().contains(File.separator + "root.sg" + File.separator)) .filter(path -> belongsToRegion(path, regionId)) + .filter( + path -> timePartitionId == null || belongsToTimePartition(path, timePartitionId)) .max( (left, right) -> Long.compare(left.toFile().lastModified(), right.toFile().lastModified())); @@ -573,7 +892,10 @@ private Path findLatestSealedTsFile(DataNodeWrapper dataNodeWrapper, int regionI return candidate.get(); } } - throw new AssertionError("No sealed TsFile found for region " + regionId); + throw new AssertionError( + "No sealed TsFile found for region " + + regionId + + (timePartitionId == null ? "" : (" partition " + timePartitionId))); } private boolean belongsToRegion(Path tsFile, int regionId) { @@ -582,6 +904,12 @@ private boolean belongsToRegion(Path tsFile, int regionId) { return regionDir != null && String.valueOf(regionId).equals(regionDir.getFileName().toString()); } + private boolean belongsToTimePartition(Path tsFile, long timePartitionId) { + Path timePartitionDir = tsFile.getParent(); + return timePartitionDir != null + && String.valueOf(timePartitionId).equals(timePartitionDir.getFileName().toString()); + } + private void deleteTsFileArtifacts(Path tsFile) throws Exception { Files.deleteIfExists(tsFile); Files.deleteIfExists(Paths.get(tsFile.toString() + TsFileResource.RESOURCE_SUFFIX)); @@ -602,4 +930,276 @@ private void triggerRegionConsistencyRepair(int regionId) throws Exception { Assert.assertEquals("Replica consistency repair should succeed", 200, status.getCode()); } } + + private void assertPartitionViewMismatch(RegionReplicaSelection regionReplicaSelection) + throws Exception { + Assert.assertNotEquals( + "Expected leader and follower logical snapshot roots to differ before repair", + partitionSnapshotSignature( + getConsistencyEligibility( + regionReplicaSelection.leaderNode, regionReplicaSelection.regionId)), + partitionSnapshotSignature( + getConsistencyEligibility( + regionReplicaSelection.followerNode, regionReplicaSelection.regionId))); + } + + private void assertPartitionViewMatched(RegionReplicaSelection regionReplicaSelection) + throws Exception { + Assert.assertEquals( + "Expected leader and follower logical snapshot roots to match after repair", + partitionSnapshotSignature( + getConsistencyEligibility( + regionReplicaSelection.leaderNode, regionReplicaSelection.regionId)), + partitionSnapshotSignature( + getConsistencyEligibility( + regionReplicaSelection.followerNode, regionReplicaSelection.regionId))); + } + + private TGetConsistencyEligibilityResp getConsistencyEligibility( + DataNodeWrapper dataNodeWrapper, int regionId) throws Exception { + try (SyncDataNodeInternalServiceClient client = + DATA_NODE_INTERNAL_CLIENT_MANAGER.borrowClient( + new TEndPoint( + dataNodeWrapper.getInternalAddress(), dataNodeWrapper.getInternalPort()))) { + TGetConsistencyEligibilityResp response = + client.getConsistencyEligibility( + new TGetConsistencyEligibilityReq( + new TConsensusGroupId(TConsensusGroupType.DataRegion, regionId))); + Assert.assertEquals( + "Consistency eligibility RPC should succeed on DataNode " + dataNodeWrapper.getId(), + 200, + response.getStatus().getCode()); + return response; + } + } + + private List partitionSnapshotSignature( + TGetConsistencyEligibilityResp eligibilityResponse) { + if (eligibilityResponse == null || !eligibilityResponse.isSetPartitions()) { + return Collections.emptyList(); + } + + List partitionSignatures = new ArrayList<>(); + for (TPartitionConsistencyEligibility partition : eligibilityResponse.getPartitions()) { + partitionSignatures.add( + partition.getTimePartitionId() + + "|mutation=" + + partition.getPartitionMutationEpoch() + + "|snapshot=" + + partition.getSnapshotEpoch() + + "|state=" + + partition.getSnapshotState() + + "|live=" + + partition.getLiveRootXorHash() + + ":" + + partition.getLiveRootAddHash() + + "|tombstone=" + + partition.getTombstoneRootXorHash() + + ":" + + partition.getTombstoneRootAddHash()); + } + Collections.sort(partitionSignatures); + return partitionSignatures; + } + + private void assertRepairProgressEmpty(int regionId) throws Exception { + Assert.assertTrue( + "Expected no repair progress rows yet for region " + regionId, + getRepairProgressRows(regionId).isEmpty()); + } + + private RepairProgressRow waitForCheckState(int regionId, long timePartitionId, String checkState) + throws Exception { + Awaitility.await() + .pollDelay(1, TimeUnit.SECONDS) + .pollInterval(1, TimeUnit.SECONDS) + .atMost(2, TimeUnit.MINUTES) + .untilAsserted( + () -> { + RepairProgressRow row = getRepairProgressRow(regionId, timePartitionId); + Assert.assertNotNull( + "Expected repair progress row for region " + + regionId + + " partition " + + timePartitionId, + row); + Assert.assertEquals(checkState, row.checkState); + }); + return getRepairProgressRow(regionId, timePartitionId); + } + + private void waitForProgressRefresh(int regionId, long timePartitionId, long previousCheckedAt) + throws Exception { + Awaitility.await() + .pollDelay(1, TimeUnit.SECONDS) + .pollInterval(1, TimeUnit.SECONDS) + .atMost(2, TimeUnit.MINUTES) + .untilAsserted( + () -> { + RepairProgressRow row = getRepairProgressRow(regionId, timePartitionId); + Assert.assertNotNull(row); + Assert.assertTrue( + "Expected last_checked_at to advance after a new background round", + row.lastCheckedAt > previousCheckedAt); + }); + } + + private RepairProgressRow getRepairProgressRow(int regionId, long timePartitionId) + throws Exception { + return getRepairProgressRows(regionId).stream() + .filter(row -> row.timePartition == timePartitionId) + .findFirst() + .orElse(null); + } + + private RepairProgressRow getRepairProgressRow( + DataNodeWrapper targetNode, int regionId, long timePartitionId) throws Exception { + return getRepairProgressRows(targetNode, regionId).stream() + .filter(row -> row.timePartition == timePartitionId) + .findFirst() + .orElse(null); + } + + private List getRepairProgressRows(int regionId) throws Exception { + try (Connection connection = + makeItCloseQuietly(EnvFactory.getEnv().getConnection(BaseEnv.TABLE_SQL_DIALECT)); + Statement statement = makeItCloseQuietly(connection.createStatement()); + ResultSet resultSet = + statement.executeQuery( + "select region_id, time_partition, check_state, repair_state, last_checked_at, " + + "last_safe_watermark, partition_mutation_epoch, snapshot_epoch, snapshot_state, " + + "last_mismatch_at, mismatch_scope_ref, mismatch_leaf_count, repair_epoch, " + + "last_error_code, last_error_message from information_schema.repair_progress " + + "where region_id = " + + regionId + + " order by time_partition")) { + List rows = new ArrayList<>(); + while (resultSet.next()) { + rows.add( + new RepairProgressRow( + resultSet.getInt(1), + resultSet.getLong(2), + resultSet.getString(3), + resultSet.getString(4), + resultSet.getLong(5), + resultSet.getLong(6), + resultSet.getLong(7), + resultSet.getLong(8), + resultSet.getString(9), + resultSet.getLong(10), + resultSet.getString(11), + resultSet.getInt(12), + resultSet.getString(13), + resultSet.getString(14), + resultSet.getString(15))); + } + rows.sort(Comparator.comparingLong(row -> row.timePartition)); + return rows; + } + } + + private List getRepairProgressRows(DataNodeWrapper targetNode, int regionId) + throws Exception { + try (Connection connection = + makeItCloseQuietly( + EnvFactory.getEnv() + .getConnection( + targetNode, + SessionConfig.DEFAULT_USER, + SessionConfig.DEFAULT_PASSWORD, + BaseEnv.TABLE_SQL_DIALECT)); + Statement statement = makeItCloseQuietly(connection.createStatement()); + ResultSet resultSet = + statement.executeQuery( + "select region_id, time_partition, check_state, repair_state, last_checked_at, " + + "last_safe_watermark, partition_mutation_epoch, snapshot_epoch, snapshot_state, " + + "last_mismatch_at, mismatch_scope_ref, mismatch_leaf_count, repair_epoch, " + + "last_error_code, last_error_message from information_schema.repair_progress " + + "where region_id = " + + regionId + + " order by time_partition")) { + List rows = new ArrayList<>(); + while (resultSet.next()) { + rows.add( + new RepairProgressRow( + resultSet.getInt(1), + resultSet.getLong(2), + resultSet.getString(3), + resultSet.getString(4), + resultSet.getLong(5), + resultSet.getLong(6), + resultSet.getLong(7), + resultSet.getLong(8), + resultSet.getString(9), + resultSet.getLong(10), + resultSet.getString(11), + resultSet.getInt(12), + resultSet.getString(13), + resultSet.getString(14), + resultSet.getString(15))); + } + return rows; + } + } + + private long timePartitionId(long timestamp) { + return Math.floorDiv(timestamp, TIME_PARTITION_INTERVAL); + } + + private String insertSql(long timestamp, long speed, long temperature) { + return String.format( + "INSERT INTO root.sg.d1(timestamp,speed,temperature) values(%d, %d, %d)", + timestamp, speed, temperature); + } + + private static final class RepairProgressRow { + private final int regionId; + private final long timePartition; + private final String checkState; + private final String repairState; + private final long lastCheckedAt; + private final long lastSafeWatermark; + private final long partitionMutationEpoch; + private final long snapshotEpoch; + private final String snapshotState; + private final long lastMismatchAt; + private final String mismatchScopeRef; + private final int mismatchLeafCount; + private final String repairEpoch; + private final String lastErrorCode; + private final String lastErrorMessage; + + private RepairProgressRow( + int regionId, + long timePartition, + String checkState, + String repairState, + long lastCheckedAt, + long lastSafeWatermark, + long partitionMutationEpoch, + long snapshotEpoch, + String snapshotState, + long lastMismatchAt, + String mismatchScopeRef, + int mismatchLeafCount, + String repairEpoch, + String lastErrorCode, + String lastErrorMessage) { + this.regionId = regionId; + this.timePartition = timePartition; + this.checkState = checkState; + this.repairState = repairState; + this.lastCheckedAt = lastCheckedAt; + this.lastSafeWatermark = lastSafeWatermark; + this.partitionMutationEpoch = partitionMutationEpoch; + this.snapshotEpoch = snapshotEpoch; + this.snapshotState = snapshotState; + this.lastMismatchAt = lastMismatchAt; + this.mismatchScopeRef = mismatchScopeRef; + this.mismatchLeafCount = mismatchLeafCount; + this.repairEpoch = repairEpoch; + this.lastErrorCode = lastErrorCode; + this.lastErrorMessage = lastErrorMessage; + } + } } diff --git a/integration-test/src/test/java/org/apache/iotdb/db/it/iotconsensusv2/batch/IoTDBIoTConsensusV2Batch3C3DBasicIT.java b/integration-test/src/test/java/org/apache/iotdb/db/it/iotconsensusv2/batch/IoTDBIoTConsensusV2Batch3C3DBasicIT.java index 2302378924e0d..837789d3cdf13 100644 --- a/integration-test/src/test/java/org/apache/iotdb/db/it/iotconsensusv2/batch/IoTDBIoTConsensusV2Batch3C3DBasicIT.java +++ b/integration-test/src/test/java/org/apache/iotdb/db/it/iotconsensusv2/batch/IoTDBIoTConsensusV2Batch3C3DBasicIT.java @@ -62,6 +62,31 @@ public void testReplicaConsistencyRepairAfterFollowerLosesSealedTsFile() throws super.testReplicaConsistencyRepairAfterFollowerLosesSealedTsFile(); } + @Override + @Test + public void testBackgroundConsistencyCheckOnlyRunsOnColdPartitions() throws Exception { + super.testBackgroundConsistencyCheckOnlyRunsOnColdPartitions(); + } + + @Override + @Test + public void testBackgroundConsistencyCheckRebuildsLogicalSnapshotAfterFollowerRestart() + throws Exception { + super.testBackgroundConsistencyCheckRebuildsLogicalSnapshotAfterFollowerRestart(); + } + + @Override + @Test + public void testBackgroundConsistencyCheckWaitsForSyncLagToClear() throws Exception { + super.testBackgroundConsistencyCheckWaitsForSyncLagToClear(); + } + + @Override + @Test + public void testRepairProgressSurvivesConfigNodeLeaderRestart() throws Exception { + super.testRepairProgressSurvivesConfigNodeLeaderRestart(); + } + @Override @Test public void test3C3DWriteFlushAndQuery() throws Exception { diff --git a/integration-test/src/test/java/org/apache/iotdb/db/it/iotconsensusv2/stream/IoTDBIoTConsensusV2Stream3C3DBasicIT.java b/integration-test/src/test/java/org/apache/iotdb/db/it/iotconsensusv2/stream/IoTDBIoTConsensusV2Stream3C3DBasicIT.java index 2c19b642d21a1..c188b99e6853a 100644 --- a/integration-test/src/test/java/org/apache/iotdb/db/it/iotconsensusv2/stream/IoTDBIoTConsensusV2Stream3C3DBasicIT.java +++ b/integration-test/src/test/java/org/apache/iotdb/db/it/iotconsensusv2/stream/IoTDBIoTConsensusV2Stream3C3DBasicIT.java @@ -62,6 +62,31 @@ public void testReplicaConsistencyRepairAfterFollowerLosesSealedTsFile() throws super.testReplicaConsistencyRepairAfterFollowerLosesSealedTsFile(); } + @Override + @Test + public void testBackgroundConsistencyCheckOnlyRunsOnColdPartitions() throws Exception { + super.testBackgroundConsistencyCheckOnlyRunsOnColdPartitions(); + } + + @Override + @Test + public void testBackgroundConsistencyCheckRebuildsLogicalSnapshotAfterFollowerRestart() + throws Exception { + super.testBackgroundConsistencyCheckRebuildsLogicalSnapshotAfterFollowerRestart(); + } + + @Override + @Test + public void testBackgroundConsistencyCheckWaitsForSyncLagToClear() throws Exception { + super.testBackgroundConsistencyCheckWaitsForSyncLagToClear(); + } + + @Override + @Test + public void testRepairProgressSurvivesConfigNodeLeaderRestart() throws Exception { + super.testRepairProgressSurvivesConfigNodeLeaderRestart(); + } + @Override @Test public void test3C3DWriteFlushAndQuery() throws Exception { diff --git a/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/client/sync/CnToDnSyncRequestType.java b/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/client/sync/CnToDnSyncRequestType.java index 858afcd602289..3841bb97b2f4d 100644 --- a/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/client/sync/CnToDnSyncRequestType.java +++ b/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/client/sync/CnToDnSyncRequestType.java @@ -36,8 +36,13 @@ public enum CnToDnSyncRequestType { REMOVE_REGION_PEER, DELETE_OLD_REGION_PEER, RESET_PEER_LIST, - GET_DATA_REGION_CONSISTENCY_SNAPSHOT, - REPAIR_TRANSFER_TSFILE, + GET_CONSISTENCY_ELIGIBILITY, + GET_SNAPSHOT_SUBTREE, + ESTIMATE_LEAF_DIFF, + DECODE_LEAF_DIFF, + STREAM_LOGICAL_REPAIR, + APPLY_LOGICAL_REPAIR_BATCH, + FINISH_LOGICAL_REPAIR_SESSION, // PartitionCache INVALIDATE_PARTITION_CACHE, diff --git a/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/client/sync/SyncDataNodeClientPool.java b/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/client/sync/SyncDataNodeClientPool.java index 69b514a06976a..bd6a391420071 100644 --- a/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/client/sync/SyncDataNodeClientPool.java +++ b/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/client/sync/SyncDataNodeClientPool.java @@ -28,19 +28,24 @@ import org.apache.iotdb.commons.client.exception.ClientManagerException; import org.apache.iotdb.commons.client.sync.SyncDataNodeInternalServiceClient; import org.apache.iotdb.commons.exception.UncheckedStartupException; +import org.apache.iotdb.mpp.rpc.thrift.TApplyLogicalRepairBatchReq; import org.apache.iotdb.mpp.rpc.thrift.TCleanDataNodeCacheReq; import org.apache.iotdb.mpp.rpc.thrift.TCreateDataRegionReq; import org.apache.iotdb.mpp.rpc.thrift.TCreatePeerReq; import org.apache.iotdb.mpp.rpc.thrift.TCreateSchemaRegionReq; -import org.apache.iotdb.mpp.rpc.thrift.TDataRegionConsistencySnapshotReq; +import org.apache.iotdb.mpp.rpc.thrift.TDecodeLeafDiffReq; +import org.apache.iotdb.mpp.rpc.thrift.TEstimateLeafDiffReq; +import org.apache.iotdb.mpp.rpc.thrift.TFinishLogicalRepairSessionReq; +import org.apache.iotdb.mpp.rpc.thrift.TGetConsistencyEligibilityReq; +import org.apache.iotdb.mpp.rpc.thrift.TGetSnapshotSubtreeReq; import org.apache.iotdb.mpp.rpc.thrift.TInvalidateCacheReq; import org.apache.iotdb.mpp.rpc.thrift.TInvalidatePermissionCacheReq; import org.apache.iotdb.mpp.rpc.thrift.TKillQueryInstanceReq; import org.apache.iotdb.mpp.rpc.thrift.TMaintainPeerReq; import org.apache.iotdb.mpp.rpc.thrift.TRegionLeaderChangeReq; import org.apache.iotdb.mpp.rpc.thrift.TRegionLeaderChangeResp; -import org.apache.iotdb.mpp.rpc.thrift.TRepairTransferTsFileReq; import org.apache.iotdb.mpp.rpc.thrift.TResetPeerListReq; +import org.apache.iotdb.mpp.rpc.thrift.TStreamLogicalRepairReq; import org.apache.iotdb.mpp.rpc.thrift.TUpdateTableReq; import org.apache.iotdb.mpp.rpc.thrift.TUpdateTemplateReq; import org.apache.iotdb.rpc.TSStatusCode; @@ -101,9 +106,26 @@ private void buildActionMap() { CnToDnSyncRequestType.DELETE_REGION, (req, client) -> client.deleteRegion((TConsensusGroupId) req)); actionMapBuilder.put( - CnToDnSyncRequestType.GET_DATA_REGION_CONSISTENCY_SNAPSHOT, - (req, client) -> - client.getDataRegionConsistencySnapshot((TDataRegionConsistencySnapshotReq) req)); + CnToDnSyncRequestType.GET_CONSISTENCY_ELIGIBILITY, + (req, client) -> client.getConsistencyEligibility((TGetConsistencyEligibilityReq) req)); + actionMapBuilder.put( + CnToDnSyncRequestType.GET_SNAPSHOT_SUBTREE, + (req, client) -> client.getSnapshotSubtree((TGetSnapshotSubtreeReq) req)); + actionMapBuilder.put( + CnToDnSyncRequestType.ESTIMATE_LEAF_DIFF, + (req, client) -> client.estimateLeafDiff((TEstimateLeafDiffReq) req)); + actionMapBuilder.put( + CnToDnSyncRequestType.DECODE_LEAF_DIFF, + (req, client) -> client.decodeLeafDiff((TDecodeLeafDiffReq) req)); + actionMapBuilder.put( + CnToDnSyncRequestType.STREAM_LOGICAL_REPAIR, + (req, client) -> client.streamLogicalRepair((TStreamLogicalRepairReq) req)); + actionMapBuilder.put( + CnToDnSyncRequestType.APPLY_LOGICAL_REPAIR_BATCH, + (req, client) -> client.applyLogicalRepairBatch((TApplyLogicalRepairBatchReq) req)); + actionMapBuilder.put( + CnToDnSyncRequestType.FINISH_LOGICAL_REPAIR_SESSION, + (req, client) -> client.finishLogicalRepairSession((TFinishLogicalRepairSessionReq) req)); actionMapBuilder.put( CnToDnSyncRequestType.INVALIDATE_PERMISSION_CACHE, (req, client) -> client.invalidatePermissionCache((TInvalidatePermissionCacheReq) req)); @@ -140,9 +162,6 @@ private void buildActionMap() { actionMapBuilder.put( CnToDnSyncRequestType.RESET_PEER_LIST, (req, client) -> client.resetPeerList((TResetPeerListReq) req)); - actionMapBuilder.put( - CnToDnSyncRequestType.REPAIR_TRANSFER_TSFILE, - (req, client) -> client.repairTransferTsFile((TRepairTransferTsFileReq) req)); actionMapBuilder.put( CnToDnSyncRequestType.SHOW_CONFIGURATION, (req, client) -> client.showConfiguration()); actionMapBuilder.put( diff --git a/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/conf/ConfigNodeConfig.java b/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/conf/ConfigNodeConfig.java index 3abb322d08472..f51033ea67cde 100644 --- a/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/conf/ConfigNodeConfig.java +++ b/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/conf/ConfigNodeConfig.java @@ -317,6 +317,9 @@ public class ConfigNodeConfig { /** The getOrCreatePartitionTable interface will log new created Partition if set true. */ private boolean isEnablePrintingNewlyCreatedPartition = false; + private long consistencyCheckSchedulerInitialDelayInMs = 60_000L; + private long consistencyCheckSchedulerIntervalInMs = 900_000L; + private long forceWalPeriodForConfigNodeSimpleInMs = 100; public ConfigNodeConfig() { @@ -1174,6 +1177,23 @@ public void setEnablePrintingNewlyCreatedPartition(boolean enablePrintingNewlyCr isEnablePrintingNewlyCreatedPartition = enablePrintingNewlyCreatedPartition; } + public long getConsistencyCheckSchedulerInitialDelayInMs() { + return consistencyCheckSchedulerInitialDelayInMs; + } + + public void setConsistencyCheckSchedulerInitialDelayInMs( + long consistencyCheckSchedulerInitialDelayInMs) { + this.consistencyCheckSchedulerInitialDelayInMs = consistencyCheckSchedulerInitialDelayInMs; + } + + public long getConsistencyCheckSchedulerIntervalInMs() { + return consistencyCheckSchedulerIntervalInMs; + } + + public void setConsistencyCheckSchedulerIntervalInMs(long consistencyCheckSchedulerIntervalInMs) { + this.consistencyCheckSchedulerIntervalInMs = consistencyCheckSchedulerIntervalInMs; + } + public long getForceWalPeriodForConfigNodeSimpleInMs() { return forceWalPeriodForConfigNodeSimpleInMs; } diff --git a/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/conf/ConfigNodeDescriptor.java b/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/conf/ConfigNodeDescriptor.java index 77790dae1a903..b4655a0056c69 100644 --- a/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/conf/ConfigNodeDescriptor.java +++ b/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/conf/ConfigNodeDescriptor.java @@ -348,6 +348,18 @@ private void loadProperties(TrimProperties properties) throws BadNodeUrlExceptio "enable_auto_leader_balance_for_iot_consensus", String.valueOf(conf.isEnableAutoLeaderBalanceForIoTConsensus())))); + conf.setConsistencyCheckSchedulerInitialDelayInMs( + Long.parseLong( + properties.getProperty( + "consistency_check_scheduler_initial_delay_in_ms", + String.valueOf(conf.getConsistencyCheckSchedulerInitialDelayInMs())))); + + conf.setConsistencyCheckSchedulerIntervalInMs( + Long.parseLong( + properties.getProperty( + "consistency_check_scheduler_interval_in_ms", + String.valueOf(conf.getConsistencyCheckSchedulerIntervalInMs())))); + String routePriorityPolicy = properties.getProperty("route_priority_policy", conf.getRoutePriorityPolicy()); if (IPriorityBalancer.GREEDY_POLICY.equals(routePriorityPolicy) diff --git a/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/manager/ConfigManager.java b/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/manager/ConfigManager.java index cc20be5f61462..cd16b1345938a 100644 --- a/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/manager/ConfigManager.java +++ b/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/manager/ConfigManager.java @@ -47,6 +47,7 @@ import org.apache.iotdb.commons.conf.ConfigurationFileUtils; import org.apache.iotdb.commons.conf.IoTDBConstant; import org.apache.iotdb.commons.conf.TrimProperties; +import org.apache.iotdb.commons.consensus.iotv2.consistency.RepairProgressTable; import org.apache.iotdb.commons.exception.IllegalPathException; import org.apache.iotdb.commons.exception.MetadataException; import org.apache.iotdb.commons.path.MeasurementPath; @@ -109,6 +110,7 @@ import org.apache.iotdb.confignode.consensus.response.ttl.ShowTTLResp; import org.apache.iotdb.confignode.consensus.statemachine.ConfigRegionStateMachine; import org.apache.iotdb.confignode.manager.consensus.ConsensusManager; +import org.apache.iotdb.confignode.manager.consistency.ConsistencyProgressManager; import org.apache.iotdb.confignode.manager.cq.CQManager; import org.apache.iotdb.confignode.manager.externalservice.ExternalServiceInfo; import org.apache.iotdb.confignode.manager.externalservice.ExternalServiceManager; @@ -125,6 +127,7 @@ import org.apache.iotdb.confignode.manager.schema.ClusterSchemaQuotaStatistics; import org.apache.iotdb.confignode.manager.subscription.SubscriptionManager; import org.apache.iotdb.confignode.persistence.ClusterInfo; +import org.apache.iotdb.confignode.persistence.ConsistencyProgressInfo; import org.apache.iotdb.confignode.persistence.ProcedureInfo; import org.apache.iotdb.confignode.persistence.TTLInfo; import org.apache.iotdb.confignode.persistence.TriggerInfo; @@ -217,6 +220,7 @@ import org.apache.iotdb.confignode.rpc.thrift.TReconstructRegionReq; import org.apache.iotdb.confignode.rpc.thrift.TRegionRouteMapResp; import org.apache.iotdb.confignode.rpc.thrift.TRemoveRegionReq; +import org.apache.iotdb.confignode.rpc.thrift.TRepairProgressInfo; import org.apache.iotdb.confignode.rpc.thrift.TSchemaNodeManagementResp; import org.apache.iotdb.confignode.rpc.thrift.TSchemaPartitionTableResp; import org.apache.iotdb.confignode.rpc.thrift.TSetDataNodeStatusReq; @@ -232,6 +236,7 @@ import org.apache.iotdb.confignode.rpc.thrift.TShowPipePluginReq; import org.apache.iotdb.confignode.rpc.thrift.TShowPipeReq; import org.apache.iotdb.confignode.rpc.thrift.TShowPipeResp; +import org.apache.iotdb.confignode.rpc.thrift.TShowRepairProgressResp; import org.apache.iotdb.confignode.rpc.thrift.TShowSubscriptionReq; import org.apache.iotdb.confignode.rpc.thrift.TShowSubscriptionResp; import org.apache.iotdb.confignode.rpc.thrift.TShowTable4InformationSchemaResp; @@ -344,6 +349,8 @@ public class ConfigManager implements IManager { /** Subscription */ private final SubscriptionManager subscriptionManager; + private final ConsistencyProgressManager consistencyProgressManager; + private final ConfigRegionStateMachine stateMachine; private final RetryFailedTasksThread retryFailedTasksThread; @@ -367,6 +374,7 @@ public ConfigManager() throws IOException { QuotaInfo quotaInfo = new QuotaInfo(); TTLInfo ttlInfo = new TTLInfo(); SubscriptionInfo subscriptionInfo = new SubscriptionInfo(); + ConsistencyProgressInfo consistencyProgressInfo = new ConsistencyProgressInfo(); // Build state machine and executor ConfigPlanExecutor executor = @@ -384,7 +392,8 @@ public ConfigManager() throws IOException { pipeInfo, subscriptionInfo, quotaInfo, - ttlInfo); + ttlInfo, + consistencyProgressInfo); this.stateMachine = new ConfigRegionStateMachine(this, executor); // Build the manager module @@ -405,6 +414,7 @@ public ConfigManager() throws IOException { this.cqManager = new CQManager(this); this.pipeManager = new PipeManager(this, pipeInfo); this.subscriptionManager = new SubscriptionManager(this, subscriptionInfo); + this.consistencyProgressManager = new ConsistencyProgressManager(consistencyProgressInfo); this.auditLogger = new CNAuditLogger(this); // 1. keep PipeManager initialization before LoadManager initialization, because @@ -1291,6 +1301,10 @@ public SubscriptionManager getSubscriptionManager() { return subscriptionManager; } + public ConsistencyProgressManager getConsistencyProgressManager() { + return consistencyProgressManager; + } + @Override public CNAuditLogger getAuditLogger() { return auditLogger; @@ -1997,6 +2011,63 @@ public TShowConfigNodes4InformationSchemaResp showConfigNodes4InformationSchema( } } + public TShowRepairProgressResp showRepairProgress() { + final TSStatus status = confirmLeader(); + final TShowRepairProgressResp resp = new TShowRepairProgressResp(); + if (status.getCode() != TSStatusCode.SUCCESS_STATUS.getStatusCode()) { + return resp.setStatus(status); + } + + List progressInfoList = new ArrayList<>(); + for (RepairProgressTable table : consistencyProgressManager.getAllTables()) { + int regionId = parseRegionId(table.getConsensusGroupId()); + for (RepairProgressTable.PartitionProgress progress : table.getAllPartitions()) { + TRepairProgressInfo progressInfo = + new TRepairProgressInfo( + regionId, + progress.getPartitionId(), + progress.getCheckState().name(), + progress.getRepairState().name(), + progress.getLastCheckedAt(), + progress.getLastSafeWatermark(), + progress.getPartitionMutationEpoch(), + progress.getSnapshotEpoch(), + progress.getSnapshotState().name(), + progress.getLastMismatchAt(), + progress.getMismatchLeafCount()); + if (progress.getMismatchScopeRef() != null) { + progressInfo.setMismatchScopeRef(progress.getMismatchScopeRef()); + } + if (progress.getRepairEpoch() != null) { + progressInfo.setRepairEpoch(progress.getRepairEpoch()); + } + if (progress.getLastErrorCode() != null) { + progressInfo.setLastErrorCode(progress.getLastErrorCode()); + } + if (progress.getLastErrorMessage() != null) { + progressInfo.setLastErrorMessage(progress.getLastErrorMessage()); + } + progressInfoList.add(progressInfo); + } + } + progressInfoList.sort( + Comparator.comparingInt(TRepairProgressInfo::getRegionId) + .thenComparingLong(TRepairProgressInfo::getTimePartition)); + return resp.setRepairProgressInfoList(progressInfoList).setStatus(StatusUtils.OK); + } + + private int parseRegionId(String consensusGroupKey) { + int separatorIndex = consensusGroupKey.lastIndexOf('-'); + if (separatorIndex < 0 || separatorIndex == consensusGroupKey.length() - 1) { + return -1; + } + try { + return Integer.parseInt(consensusGroupKey.substring(separatorIndex + 1)); + } catch (NumberFormatException e) { + return -1; + } + } + @Override public TShowDatabaseResp showDatabase(final TGetDatabaseReq req) { final TSStatus status = confirmLeader(); @@ -2605,7 +2676,7 @@ public TSStatus removeRegion(TRemoveRegionReq req) { public TSStatus triggerRegionConsistencyRepair(TTriggerRegionConsistencyRepairReq req) { TSStatus status = confirmLeader(); return status.getCode() == TSStatusCode.SUCCESS_STATUS.getStatusCode() - ? procedureManager.triggerRegionConsistencyRepair(req.getConsensusGroupId()) + ? procedureManager.triggerRegionConsistencyRepair(req) : status; } diff --git a/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/manager/ProcedureManager.java b/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/manager/ProcedureManager.java index 88ee234c8fc0b..a918fd86fbc33 100644 --- a/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/manager/ProcedureManager.java +++ b/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/manager/ProcedureManager.java @@ -53,6 +53,7 @@ import org.apache.iotdb.confignode.consensus.request.write.datanode.RemoveDataNodePlan; import org.apache.iotdb.confignode.consensus.request.write.procedure.UpdateProcedurePlan; import org.apache.iotdb.confignode.consensus.request.write.region.CreateRegionGroupsPlan; +import org.apache.iotdb.confignode.manager.consistency.ConsistencyCheckScheduler; import org.apache.iotdb.confignode.manager.partition.PartitionManager; import org.apache.iotdb.confignode.persistence.ProcedureInfo; import org.apache.iotdb.confignode.procedure.PartitionTableAutoCleaner; @@ -151,6 +152,7 @@ import org.apache.iotdb.confignode.rpc.thrift.TReconstructRegionReq; import org.apache.iotdb.confignode.rpc.thrift.TRemoveRegionReq; import org.apache.iotdb.confignode.rpc.thrift.TSubscribeReq; +import org.apache.iotdb.confignode.rpc.thrift.TTriggerRegionConsistencyRepairReq; import org.apache.iotdb.confignode.rpc.thrift.TUnsubscribeReq; import org.apache.iotdb.consensus.ConsensusFactory; import org.apache.iotdb.db.exception.BatchProcessException; @@ -173,6 +175,7 @@ import java.nio.ByteBuffer; import java.util.ArrayList; import java.util.Arrays; +import java.util.Collections; import java.util.HashMap; import java.util.List; import java.util.Map; @@ -207,6 +210,7 @@ public class ProcedureManager { private ProcedureMetrics procedureMetrics; private final PartitionTableAutoCleaner partitionTableCleaner; + private final ConsistencyCheckScheduler consistencyCheckScheduler; private final ReentrantLock tableLock = new ReentrantLock(); @@ -223,6 +227,7 @@ public ProcedureManager(ConfigManager configManager, ProcedureInfo procedureInfo - IoTDBConstant.RAFT_LOG_BASIC_SIZE; this.procedureMetrics = new ProcedureMetrics(this); this.partitionTableCleaner = new PartitionTableAutoCleaner<>(configManager); + this.consistencyCheckScheduler = new ConsistencyCheckScheduler(configManager, this); } public void startExecutor() { @@ -234,12 +239,14 @@ public void startExecutor() { CONFIG_NODE_CONFIG.getProcedureCompletedEvictTTL()); executor.addInternalProcedure(partitionTableCleaner); store.start(); + consistencyCheckScheduler.start(); LOGGER.info("ProcedureManager is started successfully."); } } public void stopExecutor() { if (executor.isRunning()) { + consistencyCheckScheduler.stop(); executor.stop(); if (!executor.isRunning()) { executor.join(); @@ -1376,7 +1383,21 @@ public TSStatus createRegionGroups( } } + public TSStatus triggerRegionConsistencyRepair(final TTriggerRegionConsistencyRepairReq req) { + return triggerRegionConsistencyRepair( + req.getConsensusGroupId(), + req.isSetPartitionFilter() ? req.getPartitionFilter() : Collections.emptyList(), + req.isSetRepairEpoch() ? req.getRepairEpoch() : null); + } + public TSStatus triggerRegionConsistencyRepair(final TConsensusGroupId consensusGroupId) { + return triggerRegionConsistencyRepair(consensusGroupId, Collections.emptyList(), null); + } + + public TSStatus triggerRegionConsistencyRepair( + final TConsensusGroupId consensusGroupId, + final List partitionFilter, + final String repairEpoch) { if (consensusGroupId == null || consensusGroupId.getType() != TConsensusGroupType.DataRegion) { return new TSStatus(TSStatusCode.ILLEGAL_PARAMETER.getStatusCode()) .setMessage("Replica consistency repair currently only supports DataRegion"); @@ -1397,7 +1418,12 @@ public TSStatus triggerRegionConsistencyRepair(final TConsensusGroupId consensus RepairRegionProcedure procedure = new RepairRegionProcedure( consensusGroupId, - new LiveDataRegionRepairExecutionContext(configManager, consensusGroupId)); + new LiveDataRegionRepairExecutionContext( + configManager, + consensusGroupId, + partitionFilter == null ? Collections.emptyList() : partitionFilter, + repairEpoch, + true)); executor.submitProcedure(procedure); return waitingProcedureFinished(procedure); } catch (Exception e) { @@ -1409,6 +1435,17 @@ public TSStatus triggerRegionConsistencyRepair(final TConsensusGroupId consensus } } + public boolean hasRunningRepairProcedure(final TConsensusGroupId consensusGroupId) { + for (Procedure procedure : executor.getProcedures().values()) { + if (procedure instanceof RepairRegionProcedure + && !procedure.isFinished() + && consensusGroupId.equals(((RepairRegionProcedure) procedure).getConsensusGroupId())) { + return true; + } + } + return false; + } + /** * Generate {@link CreateTriggerProcedure} and wait until it finished. * diff --git a/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/manager/consistency/ConsistencyCheckScheduler.java b/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/manager/consistency/ConsistencyCheckScheduler.java new file mode 100644 index 0000000000000..57c272ba538b9 --- /dev/null +++ b/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/manager/consistency/ConsistencyCheckScheduler.java @@ -0,0 +1,171 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.iotdb.confignode.manager.consistency; + +import org.apache.iotdb.common.rpc.thrift.TConsensusGroupId; +import org.apache.iotdb.common.rpc.thrift.TConsensusGroupType; +import org.apache.iotdb.commons.concurrent.IoTDBThreadPoolFactory; +import org.apache.iotdb.commons.concurrent.threadpool.ScheduledExecutorUtil; +import org.apache.iotdb.commons.consensus.iotv2.consistency.RepairProgressTable; +import org.apache.iotdb.commons.utils.TestOnly; +import org.apache.iotdb.confignode.conf.ConfigNodeDescriptor; +import org.apache.iotdb.confignode.manager.ConfigManager; +import org.apache.iotdb.confignode.manager.ProcedureManager; +import org.apache.iotdb.confignode.procedure.impl.consistency.LiveDataRegionRepairExecutionContext; + +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.util.Collections; +import java.util.Comparator; +import java.util.concurrent.Future; +import java.util.concurrent.ScheduledExecutorService; +import java.util.concurrent.TimeUnit; +import java.util.concurrent.atomic.AtomicBoolean; + +public class ConsistencyCheckScheduler { + + private static final Logger LOGGER = LoggerFactory.getLogger(ConsistencyCheckScheduler.class); + + private final ConfigManager configManager; + private final ProcedureManager procedureManager; + private final ScheduledExecutorService executorService = + IoTDBThreadPoolFactory.newSingleThreadScheduledExecutor("ConfigNode-Consistency-Check"); + private final RegionCheckExecutor regionCheckExecutor; + private final long initialDelayMs; + private final long intervalMs; + private final AtomicBoolean roundRunning = new AtomicBoolean(false); + private final Object scheduleMonitor = new Object(); + + private Future scheduledFuture; + + public ConsistencyCheckScheduler(ConfigManager configManager, ProcedureManager procedureManager) { + this( + configManager, + procedureManager, + ConfigNodeDescriptor.getInstance().getConf().getConsistencyCheckSchedulerInitialDelayInMs(), + ConfigNodeDescriptor.getInstance().getConf().getConsistencyCheckSchedulerIntervalInMs(), + (manager, consensusGroupId, progressTable) -> { + LiveDataRegionRepairExecutionContext executionContext = + new LiveDataRegionRepairExecutionContext( + manager, consensusGroupId, Collections.emptySet(), null, false); + executionContext.collectPendingPartitions( + executionContext.computeSafeWatermark(), progressTable); + }); + } + + @TestOnly + ConsistencyCheckScheduler( + ConfigManager configManager, + ProcedureManager procedureManager, + long initialDelayMs, + long intervalMs, + RegionCheckExecutor regionCheckExecutor) { + this.configManager = configManager; + this.procedureManager = procedureManager; + this.initialDelayMs = initialDelayMs; + this.intervalMs = intervalMs; + this.regionCheckExecutor = regionCheckExecutor; + } + + public void start() { + synchronized (scheduleMonitor) { + if (scheduledFuture != null) { + return; + } + scheduledFuture = + ScheduledExecutorUtil.safelyScheduleWithFixedDelay( + executorService, + this::runOneRound, + initialDelayMs, + intervalMs, + TimeUnit.MILLISECONDS); + LOGGER.info( + "Consistency check scheduler started with initialDelay={}ms interval={}ms", + initialDelayMs, + intervalMs); + } + } + + public void stop() { + synchronized (scheduleMonitor) { + if (scheduledFuture == null) { + return; + } + scheduledFuture.cancel(false); + scheduledFuture = null; + LOGGER.info("Consistency check scheduler stopped"); + } + } + + void runOneRound() { + if (!roundRunning.compareAndSet(false, true)) { + LOGGER.debug("Skip consistency check round because the previous round is still running"); + return; + } + + try { + configManager + .getPartitionManager() + .getAllReplicaSetsMap(TConsensusGroupType.DataRegion) + .keySet() + .stream() + .sorted(Comparator.comparingInt(TConsensusGroupId::getId)) + .forEach(this::checkOneRegion); + } finally { + roundRunning.set(false); + } + } + + private void checkOneRegion(TConsensusGroupId consensusGroupId) { + if (procedureManager.hasRunningRepairProcedure(consensusGroupId)) { + LOGGER.debug( + "Skip background consistency check for region {} because a repair is running", + consensusGroupId); + return; + } + + try { + ConsistencyProgressManager consistencyProgressManager = + configManager.getConsistencyProgressManager(); + if (consistencyProgressManager == null) { + LOGGER.debug( + "Skip background consistency check for region {} because progress manager is not ready", + consensusGroupId); + return; + } + RepairProgressTable progressTable = + consistencyProgressManager.loadRepairProgressTable(consensusGroupId); + regionCheckExecutor.execute(configManager, consensusGroupId, progressTable); + consistencyProgressManager.persistRepairProgressTable(progressTable); + } catch (Exception e) { + LOGGER.warn("Background consistency check failed for region {}", consensusGroupId, e); + } + } + + @FunctionalInterface + interface RegionCheckExecutor { + void execute( + ConfigManager configManager, + TConsensusGroupId consensusGroupId, + RepairProgressTable progressTable) + throws Exception; + } +} diff --git a/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/manager/consistency/ConsistencyProgressManager.java b/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/manager/consistency/ConsistencyProgressManager.java new file mode 100644 index 0000000000000..ff395d7854e9d --- /dev/null +++ b/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/manager/consistency/ConsistencyProgressManager.java @@ -0,0 +1,59 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.iotdb.confignode.manager.consistency; + +import org.apache.iotdb.common.rpc.thrift.TConsensusGroupId; +import org.apache.iotdb.commons.consensus.iotv2.consistency.RepairProgressTable; +import org.apache.iotdb.confignode.persistence.ConsistencyProgressInfo; + +import java.util.List; + +public class ConsistencyProgressManager { + + private final ConsistencyProgressInfo consistencyProgressInfo; + + public ConsistencyProgressManager(ConsistencyProgressInfo consistencyProgressInfo) { + this.consistencyProgressInfo = consistencyProgressInfo; + } + + public RepairProgressTable loadRepairProgressTable(String consensusGroupKey) { + RepairProgressTable table = consistencyProgressInfo.getTable(consensusGroupKey); + return table == null ? new RepairProgressTable(consensusGroupKey) : table; + } + + public RepairProgressTable loadRepairProgressTable(TConsensusGroupId consensusGroupId) { + return loadRepairProgressTable(toConsensusGroupKey(consensusGroupId)); + } + + public void persistRepairProgressTable(RepairProgressTable repairProgressTable) { + consistencyProgressInfo.updateTable(repairProgressTable); + } + + public List getAllTables() { + return consistencyProgressInfo.getAllTables(); + } + + public static String toConsensusGroupKey(TConsensusGroupId consensusGroupId) { + if (consensusGroupId == null) { + return "unknown"; + } + return consensusGroupId.getType() + "-" + consensusGroupId.getId(); + } +} diff --git a/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/persistence/ConsistencyProgressInfo.java b/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/persistence/ConsistencyProgressInfo.java new file mode 100644 index 0000000000000..80b9a117a2525 --- /dev/null +++ b/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/persistence/ConsistencyProgressInfo.java @@ -0,0 +1,124 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.iotdb.confignode.persistence; + +import org.apache.iotdb.commons.consensus.iotv2.consistency.RepairProgressTable; +import org.apache.iotdb.commons.snapshot.SnapshotProcessor; + +import org.apache.thrift.TException; +import org.apache.tsfile.utils.ReadWriteIOUtils; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.io.File; +import java.io.FileInputStream; +import java.io.FileOutputStream; +import java.io.IOException; +import java.util.ArrayList; +import java.util.List; +import java.util.Map; +import java.util.UUID; +import java.util.concurrent.ConcurrentHashMap; + +public class ConsistencyProgressInfo implements SnapshotProcessor { + + private static final Logger LOGGER = LoggerFactory.getLogger(ConsistencyProgressInfo.class); + private static final String SNAPSHOT_FILENAME = "consistency_progress.bin"; + + private final ConcurrentHashMap progressTables = + new ConcurrentHashMap<>(); + + public RepairProgressTable getOrCreateTable(String consensusGroupKey) { + return progressTables.computeIfAbsent(consensusGroupKey, RepairProgressTable::new); + } + + public RepairProgressTable getTable(String consensusGroupKey) { + RepairProgressTable table = progressTables.get(consensusGroupKey); + return table == null ? null : table.copy(); + } + + public void updateTable(RepairProgressTable repairProgressTable) { + if (repairProgressTable == null) { + return; + } + progressTables.put(repairProgressTable.getConsensusGroupId(), repairProgressTable.copy()); + } + + public List getAllTables() { + List tables = new ArrayList<>(); + for (RepairProgressTable table : progressTables.values()) { + tables.add(table.copy()); + } + tables.sort((left, right) -> left.getConsensusGroupId().compareTo(right.getConsensusGroupId())); + return tables; + } + + @Override + public boolean processTakeSnapshot(File snapshotDir) throws TException, IOException { + File snapshotFile = new File(snapshotDir, SNAPSHOT_FILENAME); + if (snapshotFile.exists() && snapshotFile.isFile()) { + LOGGER.error( + "Failed to take consistency progress snapshot because [{}] already exists", + snapshotFile.getAbsolutePath()); + return false; + } + + File tmpFile = new File(snapshotFile.getAbsolutePath() + "-" + UUID.randomUUID()); + try (FileOutputStream outputStream = new FileOutputStream(tmpFile)) { + List tables = getAllTables(); + ReadWriteIOUtils.write(tables.size(), outputStream); + for (RepairProgressTable table : tables) { + table.serialize(outputStream); + } + outputStream.getFD().sync(); + } + return tmpFile.renameTo(snapshotFile); + } + + @Override + public void processLoadSnapshot(File snapshotDir) throws TException, IOException { + File snapshotFile = new File(snapshotDir, SNAPSHOT_FILENAME); + if (!snapshotFile.exists() || !snapshotFile.isFile()) { + LOGGER.info( + "Consistency progress snapshot [{}] does not exist, skip loading", + snapshotFile.getAbsolutePath()); + return; + } + + ConcurrentHashMap recovered = new ConcurrentHashMap<>(); + try (FileInputStream inputStream = new FileInputStream(snapshotFile)) { + int size = ReadWriteIOUtils.readInt(inputStream); + for (int i = 0; i < size; i++) { + RepairProgressTable table = RepairProgressTable.deserialize(inputStream); + recovered.put(table.getConsensusGroupId(), table); + } + } + progressTables.clear(); + progressTables.putAll(recovered); + } + + public int size() { + return progressTables.size(); + } + + public Map view() { + return progressTables; + } +} diff --git a/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/persistence/executor/ConfigPlanExecutor.java b/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/persistence/executor/ConfigPlanExecutor.java index 8016690d17c9a..59355fba0add1 100644 --- a/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/persistence/executor/ConfigPlanExecutor.java +++ b/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/persistence/executor/ConfigPlanExecutor.java @@ -151,6 +151,7 @@ import org.apache.iotdb.confignode.manager.externalservice.ExternalServiceInfo; import org.apache.iotdb.confignode.manager.pipe.agent.PipeConfigNodeAgent; import org.apache.iotdb.confignode.persistence.ClusterInfo; +import org.apache.iotdb.confignode.persistence.ConsistencyProgressInfo; import org.apache.iotdb.confignode.persistence.ProcedureInfo; import org.apache.iotdb.confignode.persistence.TTLInfo; import org.apache.iotdb.confignode.persistence.TriggerInfo; @@ -220,6 +221,8 @@ public class ConfigPlanExecutor { private final TTLInfo ttlInfo; + private final ConsistencyProgressInfo consistencyProgressInfo; + public ConfigPlanExecutor( ClusterInfo clusterInfo, NodeInfo nodeInfo, @@ -234,7 +237,8 @@ public ConfigPlanExecutor( PipeInfo pipeInfo, SubscriptionInfo subscriptionInfo, QuotaInfo quotaInfo, - TTLInfo ttlInfo) { + TTLInfo ttlInfo, + ConsistencyProgressInfo consistencyProgressInfo) { this.snapshotProcessorList = new ArrayList<>(); @@ -280,6 +284,9 @@ public ConfigPlanExecutor( this.ttlInfo = ttlInfo; this.snapshotProcessorList.add(ttlInfo); + this.consistencyProgressInfo = consistencyProgressInfo; + this.snapshotProcessorList.add(consistencyProgressInfo); + this.snapshotProcessorList.add(PipeConfigNodeAgent.runtime().listener()); } diff --git a/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/procedure/impl/consistency/ConsistencyPartitionSelector.java b/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/procedure/impl/consistency/ConsistencyPartitionSelector.java new file mode 100644 index 0000000000000..64f165b8c13a1 --- /dev/null +++ b/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/procedure/impl/consistency/ConsistencyPartitionSelector.java @@ -0,0 +1,135 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.iotdb.confignode.procedure.impl.consistency; + +import org.apache.iotdb.commons.consensus.iotv2.consistency.RepairProgressTable; +import org.apache.iotdb.mpp.rpc.thrift.TPartitionConsistencyEligibility; + +import java.util.Collections; +import java.util.List; +import java.util.Map; +import java.util.Set; +import java.util.stream.Collectors; + +final class ConsistencyPartitionSelector { + + private ConsistencyPartitionSelector() {} + + static List selectCandidatePartitions( + Map leaderEligibilityByPartition, + Set requestedPartitions, + boolean repairMode, + String requestedRepairEpoch, + RepairProgressTable repairProgressTable) { + return selectCandidatePartitions( + leaderEligibilityByPartition, + requestedPartitions, + repairMode, + requestedRepairEpoch, + repairProgressTable, + Collections.emptyMap()); + } + + static List selectCandidatePartitions( + Map leaderEligibilityByPartition, + Set requestedPartitions, + boolean repairMode, + String requestedRepairEpoch, + RepairProgressTable repairProgressTable, + Map replicaObservationTokens) { + List eligiblePartitions = + leaderEligibilityByPartition.values().stream() + .map(TPartitionConsistencyEligibility::getTimePartitionId) + .filter( + partitionId -> + requestedPartitions == null + || requestedPartitions.isEmpty() + || requestedPartitions.contains(partitionId)) + .sorted() + .collect(Collectors.toList()); + if (eligiblePartitions.isEmpty()) { + return Collections.emptyList(); + } + + if (repairMode && (requestedPartitions == null || requestedPartitions.isEmpty())) { + List mismatchPartitions = + eligiblePartitions.stream() + .filter( + partitionId -> { + RepairProgressTable.PartitionProgress progress = + repairProgressTable.getPartition(partitionId); + return progress != null + && progress.getCheckState() == RepairProgressTable.CheckState.MISMATCH + && (requestedRepairEpoch == null + || requestedRepairEpoch.equals(progress.getRepairEpoch())); + }) + .collect(Collectors.toList()); + if (!mismatchPartitions.isEmpty()) { + return mismatchPartitions; + } + } + + return eligiblePartitions.stream() + .filter( + partitionId -> + shouldInspectPartition( + leaderEligibilityByPartition.get(partitionId), + repairProgressTable.getPartition(partitionId), + replicaObservationTokens.get(partitionId))) + .collect(Collectors.toList()); + } + + static boolean shouldInspectPartition( + TPartitionConsistencyEligibility eligibility, + RepairProgressTable.PartitionProgress progress) { + return shouldInspectPartition(eligibility, progress, null); + } + + static boolean shouldInspectPartition( + TPartitionConsistencyEligibility eligibility, + RepairProgressTable.PartitionProgress progress, + String replicaObservationToken) { + if (eligibility == null) { + return false; + } + if (progress == null) { + return true; + } + RepairProgressTable.SnapshotState snapshotState = + RepairProgressTable.SnapshotState.valueOf(eligibility.getSnapshotState()); + if (snapshotState != RepairProgressTable.SnapshotState.READY) { + return true; + } + switch (progress.getCheckState()) { + case VERIFIED: + return progress.shouldCheck( + eligibility.getPartitionMutationEpoch(), + eligibility.getSnapshotEpoch(), + snapshotState, + replicaObservationToken); + case PENDING: + case DIRTY: + case MISMATCH: + case FAILED: + default: + return true; + } + } +} diff --git a/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/procedure/impl/consistency/LiveDataRegionRepairExecutionContext.java b/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/procedure/impl/consistency/LiveDataRegionRepairExecutionContext.java index dd46f4cf7e462..e56d8db5c8ddd 100644 --- a/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/procedure/impl/consistency/LiveDataRegionRepairExecutionContext.java +++ b/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/procedure/impl/consistency/LiveDataRegionRepairExecutionContext.java @@ -22,29 +22,40 @@ import org.apache.iotdb.common.rpc.thrift.TConsensusGroupId; import org.apache.iotdb.common.rpc.thrift.TConsensusGroupType; import org.apache.iotdb.common.rpc.thrift.TDataNodeLocation; +import org.apache.iotdb.common.rpc.thrift.TEndPoint; import org.apache.iotdb.common.rpc.thrift.TRegionReplicaSet; import org.apache.iotdb.common.rpc.thrift.TSStatus; +import org.apache.iotdb.commons.consensus.iotv2.consistency.LogicalMismatchScope; import org.apache.iotdb.commons.consensus.iotv2.consistency.RepairProgressTable; -import org.apache.iotdb.commons.consensus.iotv2.consistency.ibf.DiffEntry; -import org.apache.iotdb.commons.consensus.iotv2.consistency.ibf.RowRefIndex; -import org.apache.iotdb.commons.consensus.iotv2.consistency.merkle.MerkleFileContent; -import org.apache.iotdb.commons.consensus.iotv2.consistency.repair.RepairConflictResolver; -import org.apache.iotdb.commons.consensus.iotv2.consistency.repair.RepairRecord; import org.apache.iotdb.confignode.client.sync.CnToDnSyncRequestType; import org.apache.iotdb.confignode.client.sync.SyncDataNodeClientPool; import org.apache.iotdb.confignode.manager.ConfigManager; -import org.apache.iotdb.mpp.rpc.thrift.TConsistencyMerkleFile; -import org.apache.iotdb.mpp.rpc.thrift.TDataRegionConsistencySnapshotReq; -import org.apache.iotdb.mpp.rpc.thrift.TDataRegionConsistencySnapshotResp; -import org.apache.iotdb.mpp.rpc.thrift.TRepairTransferTsFileReq; -import org.apache.iotdb.mpp.rpc.thrift.TTimePartitionConsistencyView; +import org.apache.iotdb.confignode.manager.consistency.ConsistencyProgressManager; +import org.apache.iotdb.mpp.rpc.thrift.TApplyLogicalRepairBatchReq; +import org.apache.iotdb.mpp.rpc.thrift.TDecodeLeafDiffReq; +import org.apache.iotdb.mpp.rpc.thrift.TDecodeLeafDiffResp; +import org.apache.iotdb.mpp.rpc.thrift.TEstimateLeafDiffReq; +import org.apache.iotdb.mpp.rpc.thrift.TEstimateLeafDiffResp; +import org.apache.iotdb.mpp.rpc.thrift.TFinishLogicalRepairSessionReq; +import org.apache.iotdb.mpp.rpc.thrift.TGetConsistencyEligibilityReq; +import org.apache.iotdb.mpp.rpc.thrift.TGetConsistencyEligibilityResp; +import org.apache.iotdb.mpp.rpc.thrift.TGetSnapshotSubtreeReq; +import org.apache.iotdb.mpp.rpc.thrift.TGetSnapshotSubtreeResp; +import org.apache.iotdb.mpp.rpc.thrift.TLeafDiffEntry; +import org.apache.iotdb.mpp.rpc.thrift.TLeafDiffEstimate; +import org.apache.iotdb.mpp.rpc.thrift.TLogicalRepairBatch; +import org.apache.iotdb.mpp.rpc.thrift.TLogicalRepairLeafSelector; +import org.apache.iotdb.mpp.rpc.thrift.TPartitionConsistencyEligibility; +import org.apache.iotdb.mpp.rpc.thrift.TSnapshotSubtreeNode; +import org.apache.iotdb.mpp.rpc.thrift.TStreamLogicalRepairReq; +import org.apache.iotdb.mpp.rpc.thrift.TStreamLogicalRepairResp; import org.apache.iotdb.rpc.TSStatusCode; import java.util.ArrayList; import java.util.Collection; import java.util.Collections; import java.util.Comparator; -import java.util.HashMap; +import java.util.HashSet; import java.util.LinkedHashMap; import java.util.LinkedHashSet; import java.util.List; @@ -52,45 +63,69 @@ import java.util.Objects; import java.util.Set; import java.util.TreeMap; +import java.util.UUID; import java.util.stream.Collectors; -/** - * Live runtime bridge for {@link RepairRegionProcedure}. The first production path only supports - * leader-authoritative repair when one or more followers are missing sealed TsFiles that still - * exist on the leader. - */ +/** Runtime bridge between the repair procedure and the logical snapshot RPCs. */ public class LiveDataRegionRepairExecutionContext implements RepairRegionProcedure.RepairExecutionContext { + private static final String TREE_KIND_LIVE = "LIVE"; + private static final String TREE_KIND_TOMBSTONE = "TOMBSTONE"; + private static final long EXACT_DIFF_DECODE_THRESHOLD = 2048L; + + private final ConfigManager configManager; + private final ConsistencyProgressManager consistencyProgressManager; private final TConsensusGroupId consensusGroupId; private final TDataNodeLocation leaderLocation; private final List followerLocations; - private final Map partitionContexts; - private final Map> transferTargetsByTsFile; + private final String routeVersionToken; + private final Set requestedPartitions; + private final String requestedRepairEpoch; + private final boolean repairMode; + + private final Map leaderEligibilityByPartition = + new TreeMap<>(); + private final Map> + followerEligibilityByNodeId = new LinkedHashMap<>(); + private final Map partitionContexts = new TreeMap<>(); + private final Map repairOperationsById = new LinkedHashMap<>(); + + private long syncLag = Long.MAX_VALUE; + private long safeWatermark = Long.MIN_VALUE; public LiveDataRegionRepairExecutionContext( ConfigManager configManager, TConsensusGroupId consensusGroupId) { + this(configManager, consensusGroupId, Collections.emptySet(), null, true); + } + + public LiveDataRegionRepairExecutionContext( + ConfigManager configManager, + TConsensusGroupId consensusGroupId, + Collection partitionFilter, + String requestedRepairEpoch, + boolean repairMode) { + this.configManager = configManager; + this.consistencyProgressManager = configManager.getConsistencyProgressManager(); this.consensusGroupId = requireDataRegion(consensusGroupId); + this.requestedPartitions = + partitionFilter == null ? Collections.emptySet() : new LinkedHashSet<>(partitionFilter); + this.requestedRepairEpoch = requestedRepairEpoch; + this.repairMode = repairMode; TRegionReplicaSet replicaSet = configManager .getPartitionManager() .getAllReplicaSetsMap(TConsensusGroupType.DataRegion) .get(this.consensusGroupId); - if (replicaSet == null) { + if (replicaSet == null || replicaSet.getDataNodeLocations() == null) { throw new IllegalStateException("DataRegion " + consensusGroupId + " does not exist"); } - if (replicaSet.getDataNodeLocations() == null || replicaSet.getDataNodeLocations().size() < 2) { - throw new IllegalStateException( - "DataRegion " + consensusGroupId + " has fewer than 2 replicas, no repair target exists"); - } - int leaderId = configManager.getLoadManager().getRegionLeaderMap().getOrDefault(this.consensusGroupId, -1); if (leaderId <= 0) { throw new IllegalStateException("Cannot determine leader for DataRegion " + consensusGroupId); } - this.leaderLocation = replicaSet.getDataNodeLocations().stream() .filter(location -> location.getDataNodeId() == leaderId) @@ -102,448 +137,1003 @@ public LiveDataRegionRepairExecutionContext( + leaderId + " is not part of replica set for " + consensusGroupId)); - this.followerLocations = replicaSet.getDataNodeLocations().stream() .filter(location -> location.getDataNodeId() != leaderId) .sorted(Comparator.comparingInt(TDataNodeLocation::getDataNodeId)) .collect(Collectors.toList()); - - Map> leaderSnapshot = fetchSnapshot(leaderLocation); - Map>> followerSnapshots = new LinkedHashMap<>(); - for (TDataNodeLocation followerLocation : followerLocations) { - followerSnapshots.put(followerLocation.getDataNodeId(), fetchSnapshot(followerLocation)); - } - - this.partitionContexts = new TreeMap<>(); - this.transferTargetsByTsFile = new LinkedHashMap<>(); - initializePartitionContexts(leaderSnapshot, followerSnapshots); + this.routeVersionToken = buildRouteBindingToken(leaderId); + refreshLeaderEligibility(); } @Override public boolean isReplicationComplete() { - return true; + return syncLag == 0L; } @Override public long computeSafeWatermark() { - return Long.MAX_VALUE; + return safeWatermark; } @Override public List collectPendingPartitions( - long globalRepairedWatermark, long safeWatermark, RepairProgressTable repairProgressTable) { - return new ArrayList<>(partitionContexts.keySet()); - } - - @Override - public RepairRegionProcedure.PartitionRepairContext getPartitionContext(long partitionId) { - return partitionContexts.get(partitionId); - } + long currentSafeWatermark, RepairProgressTable repairProgressTable) { + if (!isRouteBindingStillValid()) { + return Collections.emptyList(); + } + refreshLeaderEligibility(); + partitionContexts.clear(); + repairOperationsById.clear(); - @Override - public RepairRecord buildRepairRecord( - RepairRegionProcedure.PartitionRepairContext partitionContext, - DiffEntry diffEntry, - RowRefIndex rowRefIndex, - RepairConflictResolver conflictResolver) { - return null; - } + if (!isReplicationComplete() || safeWatermark == Long.MIN_VALUE) { + return Collections.emptyList(); + } - @Override - public void transferTsFile(String tsFilePath) { - List targetDataNodes = transferTargetsByTsFile.get(tsFilePath); - if (targetDataNodes == null || targetDataNodes.isEmpty()) { - throw new IllegalStateException("No repair target found for TsFile " + tsFilePath); + followerEligibilityByNodeId.clear(); + for (TDataNodeLocation followerLocation : followerLocations) { + EligibilitySnapshot snapshot = fetchEligibility(followerLocation, !repairMode); + if (!snapshot.isAvailable()) { + return Collections.emptyList(); + } + followerEligibilityByNodeId.put( + followerLocation.getDataNodeId(), snapshot.getPartitionsMap()); } - TRepairTransferTsFileReq request = - new TRepairTransferTsFileReq(consensusGroupId, tsFilePath, targetDataNodes); - Object response = - SyncDataNodeClientPool.getInstance() - .sendSyncRequestToDataNodeWithRetry( - leaderLocation.getInternalEndPoint(), - request, - CnToDnSyncRequestType.REPAIR_TRANSFER_TSFILE); - - TSStatus status = - response instanceof TSStatus - ? (TSStatus) response - : new TSStatus(TSStatusCode.INTERNAL_SERVER_ERROR.getStatusCode()) - .setMessage("Unexpected response type for repair TsFile transfer: " + response); - if (status.getCode() != TSStatusCode.SUCCESS_STATUS.getStatusCode()) { - throw new IllegalStateException( - "Repair transfer of TsFile " + tsFilePath + " failed: " + status.getMessage()); + List candidatePartitions = + ConsistencyPartitionSelector.selectCandidatePartitions( + leaderEligibilityByPartition, + requestedPartitions, + repairMode, + requestedRepairEpoch, + repairProgressTable, + buildReplicaObservationTokens()); + if (candidatePartitions.isEmpty()) { + return Collections.emptyList(); } - } - private void initializePartitionContexts( - Map> leaderSnapshot, - Map>> followerSnapshots) { - Set partitionIds = new LinkedHashSet<>(leaderSnapshot.keySet()); - followerSnapshots.values().forEach(snapshot -> partitionIds.addAll(snapshot.keySet())); + long now = System.currentTimeMillis(); + List repairablePartitions = new ArrayList<>(); + for (Long partitionId : candidatePartitions) { + TPartitionConsistencyEligibility leaderEligibility = + leaderEligibilityByPartition.get(partitionId); + RepairProgressTable.PartitionProgress progress = + repairProgressTable.getPartition(partitionId); + if (leaderEligibility == null) { + continue; + } + if (!isSnapshotComparable(leaderEligibility)) { + repairProgressTable.markDirty(partitionId); + continue; + } - for (Long partitionId : partitionIds.stream().sorted().collect(Collectors.toList())) { - List leaderFiles = - leaderSnapshot.getOrDefault(partitionId, Collections.emptyList()); - if (leaderFiles.isEmpty()) { + boolean allFollowersReady = true; + boolean rootMatched = true; + Set unionScopes = new LinkedHashSet<>(); + try { for (TDataNodeLocation followerLocation : followerLocations) { - List followerFiles = - followerSnapshots + TPartitionConsistencyEligibility followerEligibility = + followerEligibilityByNodeId .getOrDefault(followerLocation.getDataNodeId(), Collections.emptyMap()) - .getOrDefault(partitionId, Collections.emptyList()); - if (!followerFiles.isEmpty()) { - throw new IllegalStateException( - String.format( - "Partition %d of region %s has follower-only data on DataNode %d", - partitionId, consensusGroupId, followerLocation.getDataNodeId())); + .get(partitionId); + if (!isSnapshotComparable(followerEligibility)) { + allFollowersReady = false; + break; + } + boolean liveMatched = sameRootDigest(leaderEligibility, followerEligibility); + boolean tombstoneMatched = sameTombstoneDigest(leaderEligibility, followerEligibility); + if (!liveMatched || !tombstoneMatched) { + rootMatched = false; + if (repairMode + && progress != null + && progress.getCheckState() == RepairProgressTable.CheckState.MISMATCH + && progress.getMismatchScopeRef() != null + && Objects.equals( + buildRepairEpoch(partitionId, leaderEligibility), progress.getRepairEpoch())) { + unionScopes.addAll(LogicalMismatchScope.deserialize(progress.getMismatchScopeRef())); + } else { + if (!liveMatched) { + unionScopes.addAll(comparePartition(followerLocation, partitionId, TREE_KIND_LIVE)); + } + if (!tombstoneMatched) { + unionScopes.addAll( + comparePartition(followerLocation, partitionId, TREE_KIND_TOMBSTONE)); + } + } } } + } catch (StaleSnapshotCompareException e) { + repairProgressTable.markDirty(partitionId); + continue; + } + + if (!allFollowersReady) { + repairProgressTable.markDirty(partitionId); + continue; + } + + if (rootMatched) { + String replicaObservationToken = buildReplicaObservationToken(partitionId); + repairProgressTable.markVerified( + partitionId, + now, + safeWatermark, + leaderEligibility.getPartitionMutationEpoch(), + leaderEligibility.getSnapshotEpoch(), + RepairProgressTable.SnapshotState.READY, + replicaObservationToken); continue; } - Map leaderCounts = buildDigestCounts(leaderFiles); - Map> leaderFilesByDigest = groupFilesByDigest(leaderFiles); - Map> transferTargets = new LinkedHashMap<>(); - boolean matched = true; + String repairEpochRef = buildRepairEpoch(partitionId, leaderEligibility); + if (repairMode + && progress != null + && progress.getCheckState() == RepairProgressTable.CheckState.MISMATCH + && progress.getMismatchScopeRef() != null + && !Objects.equals(repairEpochRef, progress.getRepairEpoch())) { + repairProgressTable.markDirty(partitionId); + continue; + } + if (unionScopes.isEmpty()) { + String replicaObservationToken = buildReplicaObservationToken(partitionId); + repairProgressTable.markVerified( + partitionId, + now, + safeWatermark, + leaderEligibility.getPartitionMutationEpoch(), + leaderEligibility.getSnapshotEpoch(), + RepairProgressTable.SnapshotState.READY, + replicaObservationToken); + continue; + } + String mismatchScopeRef = LogicalMismatchScope.serialize(unionScopes); + String replicaObservationToken = buildReplicaObservationToken(partitionId); + repairProgressTable.markMismatch( + partitionId, + now, + safeWatermark, + leaderEligibility.getPartitionMutationEpoch(), + leaderEligibility.getSnapshotEpoch(), + RepairProgressTable.SnapshotState.READY, + mismatchScopeRef, + unionScopes.size(), + repairEpochRef, + replicaObservationToken); + + if (!repairMode) { + continue; + } + + String blockingReason = buildBlockingReason(partitionId, unionScopes); + if (blockingReason != null) { + partitionContexts.put( + partitionId, + new LivePartitionRepairContext( + partitionId, false, Collections.emptyList(), repairEpochRef, blockingReason)); + repairablePartitions.add(partitionId); + continue; + } + + LivePartitionRepairContext context = + new LivePartitionRepairContext( + partitionId, false, new ArrayList<>(), repairEpochRef, null); + Set scheduledLeaves = new HashSet<>(); for (TDataNodeLocation followerLocation : followerLocations) { - List followerFiles = - followerSnapshots + TPartitionConsistencyEligibility followerEligibility = + followerEligibilityByNodeId .getOrDefault(followerLocation.getDataNodeId(), Collections.emptyMap()) - .getOrDefault(partitionId, Collections.emptyList()); - Map followerCounts = buildDigestCounts(followerFiles); - - for (Map.Entry followerEntry : followerCounts.entrySet()) { - int leaderCount = leaderCounts.getOrDefault(followerEntry.getKey(), 0); - if (followerEntry.getValue() > leaderCount) { - throw new IllegalStateException( - String.format( - "Partition %d of region %s has follower-only digest %s on DataNode %d", - partitionId, - consensusGroupId, - followerEntry.getKey(), - followerLocation.getDataNodeId())); - } + .get(partitionId); + if (followerEligibility == null + || (sameRootDigest(leaderEligibility, followerEligibility) + && sameTombstoneDigest(leaderEligibility, followerEligibility))) { + continue; } - - for (Map.Entry leaderEntry : leaderCounts.entrySet()) { - int followerCount = followerCounts.getOrDefault(leaderEntry.getKey(), 0); - int missingCount = leaderEntry.getValue() - followerCount; - if (missingCount <= 0) { + for (LogicalMismatchScope.Scope scope : unionScopes) { + RepairLeafKey repairLeafKey = + new RepairLeafKey( + followerLocation.getDataNodeId(), + scope.getTreeKind(), + scope.getLeafId(), + scope.toPersistentString()); + if (!scheduledLeaves.add(repairLeafKey)) { continue; } - - matched = false; - List candidateFiles = leaderFilesByDigest.get(leaderEntry.getKey()); - for (int i = 0; i < missingCount; i++) { - SnapshotFile snapshotFile = candidateFiles.get(i); - transferTargets - .computeIfAbsent(snapshotFile.getSourceTsFilePath(), ignored -> new ArrayList<>()) - .add(followerLocation); - } + String operationId = + partitionId + + "-" + + followerLocation.getDataNodeId() + + "-" + + scope.getTreeKind() + + "-" + + scope.getLeafId() + + "-" + + UUID.randomUUID(); + repairOperationsById.put( + operationId, + new RepairOperation(partitionId, followerLocation, scope, repairEpochRef)); + context.repairOperationIds.add(operationId); } } - - if (matched) { - continue; + if (!context.repairOperationIds.isEmpty()) { + partitionContexts.put(partitionId, context); + repairablePartitions.add(partitionId); } - - LivePartitionRepairContext partitionContext = - new LivePartitionRepairContext(partitionId, leaderFiles, transferTargets); - partitionContexts.put(partitionId, partitionContext); - transferTargets.forEach( - (tsFilePath, targets) -> - transferTargetsByTsFile - .computeIfAbsent(tsFilePath, ignored -> new ArrayList<>()) - .addAll(targets)); } + + return repairablePartitions; } - private Map> fetchSnapshot(TDataNodeLocation dataNodeLocation) { - Object response = - SyncDataNodeClientPool.getInstance() - .sendSyncRequestToDataNodeWithRetry( - dataNodeLocation.getInternalEndPoint(), - new TDataRegionConsistencySnapshotReq(consensusGroupId), - CnToDnSyncRequestType.GET_DATA_REGION_CONSISTENCY_SNAPSHOT); + @Override + public RepairRegionProcedure.PartitionRepairContext getPartitionContext(long partitionId) { + return partitionContexts.get(partitionId); + } - if (response instanceof TSStatus) { - TSStatus status = (TSStatus) response; - throw new IllegalStateException( - "Failed to fetch consistency snapshot from DataNode " - + dataNodeLocation.getDataNodeId() - + ": " - + status.getMessage()); + @Override + public void executeRepairOperation(String operationId) { + RepairOperation operation = repairOperationsById.get(operationId); + if (operation == null) { + throw new IllegalStateException("Unknown logical repair operation " + operationId); } - if (!(response instanceof TDataRegionConsistencySnapshotResp)) { + if (!isRouteBindingStillValid()) { throw new IllegalStateException( - "Unexpected snapshot response type from DataNode " - + dataNodeLocation.getDataNodeId() - + ": " - + response); + "Repair route drift detected for partition " + operation.partitionId); } - TDataRegionConsistencySnapshotResp snapshotResp = (TDataRegionConsistencySnapshotResp) response; - if (snapshotResp.getStatus().getCode() != TSStatusCode.SUCCESS_STATUS.getStatusCode()) { + TPartitionConsistencyEligibility leaderEligibility = + leaderEligibilityByPartition.get(operation.partitionId); + if (leaderEligibility == null + || !Objects.equals( + buildRepairEpoch(operation.partitionId, leaderEligibility), operation.repairEpoch)) { throw new IllegalStateException( - "Failed to fetch consistency snapshot from DataNode " - + dataNodeLocation.getDataNodeId() - + ": " - + snapshotResp.getStatus().getMessage()); - } - - Map> snapshot = new TreeMap<>(); - if (!snapshotResp.isSetTimePartitionViews()) { - return snapshot; - } - - for (TTimePartitionConsistencyView partitionView : snapshotResp.getTimePartitionViews()) { - List files = new ArrayList<>(); - if (partitionView.isSetMerkleFiles()) { - for (TConsistencyMerkleFile merkleFile : partitionView.getMerkleFiles()) { - files.add( - new SnapshotFile( - merkleFile.getSourceTsFilePath(), - merkleFile.getTsFileSize(), - new DigestKey(merkleFile.getFileXorHash(), merkleFile.getFileAddHash()))); - } + "Repair epoch drift detected for partition " + operation.partitionId); + } + + TStreamLogicalRepairResp streamResp = + sendRequestExpect( + leaderLocation.getInternalEndPoint(), + new TStreamLogicalRepairReq( + consensusGroupId, + operation.partitionId, + operation.repairEpoch, + Collections.singletonList( + new TLogicalRepairLeafSelector( + operation.scope.getTreeKind(), encodeRepairSelector(operation.scope)))), + CnToDnSyncRequestType.STREAM_LOGICAL_REPAIR, + TStreamLogicalRepairResp.class); + requireSuccess(streamResp.getStatus(), "streamLogicalRepair"); + if (Boolean.TRUE.equals(streamResp.isStale())) { + throw new IllegalStateException("Leader logical repair stream is stale"); + } + + String sessionId = null; + if (streamResp.isSetBatches()) { + for (TLogicalRepairBatch batch : streamResp.getBatches()) { + sessionId = batch.getSessionId(); + TSStatus applyStatus = + sendRequestExpect( + operation.followerLocation.getInternalEndPoint(), + new TApplyLogicalRepairBatchReq( + consensusGroupId, + operation.partitionId, + operation.repairEpoch, + batch.getSessionId(), + batch.getTreeKind(), + batch.getLeafId(), + batch.getSeqNo(), + batch.getBatchKind(), + batch.bufferForPayload()), + CnToDnSyncRequestType.APPLY_LOGICAL_REPAIR_BATCH, + TSStatus.class); + requireSuccess(applyStatus, "applyLogicalRepairBatch"); } - files.sort(Comparator.comparing(SnapshotFile::getSourceTsFilePath)); - snapshot.put(partitionView.getTimePartitionId(), files); } - return snapshot; + if (sessionId != null) { + TSStatus finishStatus = + sendRequestExpect( + operation.followerLocation.getInternalEndPoint(), + new TFinishLogicalRepairSessionReq( + consensusGroupId, operation.partitionId, operation.repairEpoch, sessionId), + CnToDnSyncRequestType.FINISH_LOGICAL_REPAIR_SESSION, + TSStatus.class); + requireSuccess(finishStatus, "finishLogicalRepairSession"); + } } - private boolean verifyPartition(long partitionId) { - Map> leaderSnapshot = fetchSnapshot(leaderLocation); - Map leaderCounts = - buildDigestCounts(leaderSnapshot.getOrDefault(partitionId, Collections.emptyList())); + @Override + public RepairProgressTable loadRepairProgressTable(String consensusGroupKey) { + return consistencyProgressManager.loadRepairProgressTable(consensusGroupKey); + } - for (TDataNodeLocation followerLocation : followerLocations) { - Map> followerSnapshot = fetchSnapshot(followerLocation); - Map followerCounts = - buildDigestCounts(followerSnapshot.getOrDefault(partitionId, Collections.emptyList())); - if (!leaderCounts.equals(followerCounts)) { - return false; + @Override + public void persistRepairProgressTable(RepairProgressTable repairProgressTable) { + consistencyProgressManager.persistRepairProgressTable(repairProgressTable); + } + + @Override + public void onPartitionCommitted( + long partitionId, long committedAt, RepairProgressTable repairProgressTable) { + TPartitionConsistencyEligibility leaderEligibility = + leaderEligibilityByPartition.get(partitionId); + if (leaderEligibility != null) { + RepairProgressTable.PartitionProgress progress = + repairProgressTable.getOrCreatePartition(partitionId); + repairProgressTable.markRepairSucceeded( + partitionId, + committedAt, + safeWatermark, + leaderEligibility.getPartitionMutationEpoch(), + leaderEligibility.getSnapshotEpoch(), + RepairProgressTable.SnapshotState.READY, + progress.getRepairEpoch(), + progress.getReplicaObservationToken()); + } + } + + private Set comparePartition( + TDataNodeLocation followerLocation, long partitionId, String treeKind) { + Set scopes = new LinkedHashSet<>(); + + Map leaderShards = + fetchSubtreeNodes(leaderLocation, partitionId, treeKind, Collections.singletonList("root")); + Map followerShards = + fetchSubtreeNodes( + followerLocation, partitionId, treeKind, Collections.singletonList("root")); + + Set shardHandles = new LinkedHashSet<>(); + shardHandles.addAll(leaderShards.keySet()); + shardHandles.addAll(followerShards.keySet()); + for (String shardHandle : shardHandles) { + TSnapshotSubtreeNode leaderShard = leaderShards.get(shardHandle); + TSnapshotSubtreeNode followerShard = followerShards.get(shardHandle); + if (sameDigest(leaderShard, followerShard)) { + continue; + } + Map leaderLeaves = + fetchSubtreeNodes( + leaderLocation, partitionId, treeKind, Collections.singletonList(shardHandle)); + Map followerLeaves = + fetchSubtreeNodes( + followerLocation, partitionId, treeKind, Collections.singletonList(shardHandle)); + Set leafHandles = new LinkedHashSet<>(); + leafHandles.addAll(leaderLeaves.keySet()); + leafHandles.addAll(followerLeaves.keySet()); + for (String leafHandle : leafHandles) { + TSnapshotSubtreeNode leaderLeaf = leaderLeaves.get(leafHandle); + TSnapshotSubtreeNode followerLeaf = followerLeaves.get(leafHandle); + if (!sameDigest(leaderLeaf, followerLeaf)) { + scopes.addAll( + compareLeaf( + followerLocation, partitionId, treeKind, leafHandle, leaderLeaf, followerLeaf)); + } } } - return true; + return scopes; } - private static TConsensusGroupId requireDataRegion(TConsensusGroupId consensusGroupId) { - if (consensusGroupId == null - || consensusGroupId.getType() == null - || consensusGroupId.getType() != TConsensusGroupType.DataRegion) { - throw new IllegalArgumentException( - "Replica consistency repair currently only supports DataRegion"); + private Set compareLeaf( + TDataNodeLocation followerLocation, + long partitionId, + String treeKind, + String leafId, + TSnapshotSubtreeNode leaderLeaf, + TSnapshotSubtreeNode followerLeaf) { + long leaderItemCount = leaderLeaf == null ? 0L : leaderLeaf.getItemCount(); + long followerItemCount = followerLeaf == null ? 0L : followerLeaf.getItemCount(); + if (Math.max(leaderItemCount, followerItemCount) > EXACT_DIFF_DECODE_THRESHOLD) { + return Collections.singleton( + new LogicalMismatchScope.Scope( + treeKind, + leafId, + chooseKeyRangeStart(leaderLeaf, followerLeaf), + chooseKeyRangeEnd(leaderLeaf, followerLeaf))); } - return consensusGroupId; + + TLeafDiffEstimate leaderEstimate = + fetchLeafEstimate(leaderLocation, partitionId, treeKind, leafId); + TLeafDiffEstimate followerEstimate = + fetchLeafEstimate(followerLocation, partitionId, treeKind, leafId); + if (leaderEstimate == null || followerEstimate == null) { + return Collections.singleton( + new LogicalMismatchScope.Scope( + treeKind, + leafId, + chooseKeyRangeStart(leaderLeaf, followerLeaf), + chooseKeyRangeEnd(leaderLeaf, followerLeaf))); + } + if (Math.max(leaderEstimate.getStrataEstimate(), followerEstimate.getStrataEstimate()) + > EXACT_DIFF_DECODE_THRESHOLD) { + return Collections.singleton( + new LogicalMismatchScope.Scope( + treeKind, + leafId, + chooseKeyRangeStart(leaderLeaf, followerLeaf), + chooseKeyRangeEnd(leaderLeaf, followerLeaf))); + } + + Set leaderKeys = fetchLeafDiffKeys(leaderLocation, partitionId, treeKind, leafId); + Set followerKeys = fetchLeafDiffKeys(followerLocation, partitionId, treeKind, leafId); + if (leaderKeys == null || followerKeys == null) { + return Collections.singleton( + new LogicalMismatchScope.Scope( + treeKind, + leafId, + chooseKeyRangeStart(leaderLeaf, followerLeaf), + chooseKeyRangeEnd(leaderLeaf, followerLeaf))); + } + + Set symmetricDiff = new LinkedHashSet<>(leaderKeys); + symmetricDiff.addAll(followerKeys); + Set intersection = new LinkedHashSet<>(leaderKeys); + intersection.retainAll(followerKeys); + symmetricDiff.removeAll(intersection); + if (symmetricDiff.isEmpty()) { + return Collections.emptySet(); + } + + List sortedDiffKeys = new ArrayList<>(symmetricDiff); + sortedDiffKeys.sort(String::compareTo); + if (TREE_KIND_TOMBSTONE.equalsIgnoreCase(treeKind)) { + Set leaderOnly = new LinkedHashSet<>(leaderKeys); + leaderOnly.removeAll(followerKeys); + List sortedLeaderOnlyKeys = new ArrayList<>(leaderOnly); + sortedLeaderOnlyKeys.sort(String::compareTo); + + Set followerOnly = new LinkedHashSet<>(followerKeys); + followerOnly.removeAll(leaderKeys); + if (!followerOnly.isEmpty()) { + return Collections.singleton( + new LogicalMismatchScope.Scope( + treeKind, + leafId, + sortedDiffKeys.get(0), + sortedDiffKeys.get(sortedDiffKeys.size() - 1), + sortedLeaderOnlyKeys, + LogicalMismatchScope.RepairDirective.FOLLOWER_EXTRA_TOMBSTONE)); + } + + return Collections.singleton( + new LogicalMismatchScope.Scope( + treeKind, + leafId, + sortedDiffKeys.get(0), + sortedDiffKeys.get(sortedDiffKeys.size() - 1), + sortedLeaderOnlyKeys)); + } + return Collections.singleton( + new LogicalMismatchScope.Scope( + treeKind, + leafId, + sortedDiffKeys.get(0), + sortedDiffKeys.get(sortedDiffKeys.size() - 1), + sortedDiffKeys)); } - private static Map buildDigestCounts(Collection files) { - Map counts = new HashMap<>(); - for (SnapshotFile file : files) { - counts.merge(file.getDigestKey(), 1, Integer::sum); + private String buildBlockingReason(long partitionId, Set scopes) { + List unsupportedScopes = + scopes.stream() + .filter(scope -> scope != null && !scope.isRepairable()) + .map(scope -> scope.getTreeKind() + "@" + scope.getLeafId()) + .sorted() + .collect(Collectors.toList()); + if (unsupportedScopes.isEmpty()) { + return null; } - return counts; + return "Partition " + + partitionId + + " contains follower-only tombstone mismatches that cannot be rolled back safely yet: " + + String.join(", ", unsupportedScopes); } - private static Map> groupFilesByDigest(List files) { - Map> grouped = new LinkedHashMap<>(); - for (SnapshotFile file : files) { - grouped.computeIfAbsent(file.getDigestKey(), ignored -> new ArrayList<>()).add(file); + private Map fetchSubtreeNodes( + TDataNodeLocation location, long partitionId, String treeKind, List nodeHandles) { + TPartitionConsistencyEligibility localEligibility = + getEligibilityForLocation(location, partitionId); + if (localEligibility == null) { + throw new StaleSnapshotCompareException( + "Missing snapshot eligibility while comparing partition " + partitionId); } - return grouped; + TGetSnapshotSubtreeResp resp = + sendRequestExpect( + location.getInternalEndPoint(), + new TGetSnapshotSubtreeReq( + consensusGroupId, + partitionId, + localEligibility.getSnapshotEpoch(), + treeKind, + nodeHandles), + CnToDnSyncRequestType.GET_SNAPSHOT_SUBTREE, + TGetSnapshotSubtreeResp.class); + requireSuccess(resp.getStatus(), "getSnapshotSubtree"); + if (Boolean.TRUE.equals(resp.isStale())) { + throw new StaleSnapshotCompareException( + "Snapshot subtree became stale while comparing partition " + partitionId); + } + Map nodes = new LinkedHashMap<>(); + if (resp.isSetNodes()) { + for (TSnapshotSubtreeNode node : resp.getNodes()) { + nodes.put(node.getNodeHandle(), node); + } + } + return nodes; } - private final class LivePartitionRepairContext - implements RepairRegionProcedure.PartitionRepairContext { + private TLeafDiffEstimate fetchLeafEstimate( + TDataNodeLocation location, long partitionId, String treeKind, String leafId) { + TPartitionConsistencyEligibility localEligibility = + getEligibilityForLocation(location, partitionId); + if (localEligibility == null) { + return null; + } + TEstimateLeafDiffResp resp = + sendRequestExpect( + location.getInternalEndPoint(), + new TEstimateLeafDiffReq( + consensusGroupId, + partitionId, + localEligibility.getSnapshotEpoch(), + treeKind, + leafId), + CnToDnSyncRequestType.ESTIMATE_LEAF_DIFF, + TEstimateLeafDiffResp.class); + requireSuccess(resp.getStatus(), "estimateLeafDiff"); + if (Boolean.TRUE.equals(resp.isStale()) || !resp.isSetLeafDiff()) { + return null; + } + return resp.getLeafDiff(); + } - private final long partitionId; - private final List leaderMerkleFiles; - private final Map leaderMerkleFileByPath; - private final Map leaderTsFileSizeByPath; - private final Map> transferTargets; + private Set fetchLeafDiffKeys( + TDataNodeLocation location, long partitionId, String treeKind, String leafId) { + TPartitionConsistencyEligibility localEligibility = + getEligibilityForLocation(location, partitionId); + if (localEligibility == null) { + return null; + } + TDecodeLeafDiffResp resp = + sendRequestExpect( + location.getInternalEndPoint(), + new TDecodeLeafDiffReq( + consensusGroupId, + partitionId, + localEligibility.getSnapshotEpoch(), + treeKind, + leafId), + CnToDnSyncRequestType.DECODE_LEAF_DIFF, + TDecodeLeafDiffResp.class); + requireSuccess(resp.getStatus(), "decodeLeafDiff"); + if (Boolean.TRUE.equals(resp.isStale())) { + return null; + } + Set logicalKeys = new LinkedHashSet<>(); + if (resp.isSetDiffEntries()) { + for (TLeafDiffEntry entry : resp.getDiffEntries()) { + logicalKeys.add(entry.getLogicalKey()); + } + } + return logicalKeys; + } - private LivePartitionRepairContext( - long partitionId, - List leaderFiles, - Map> transferTargets) { - this.partitionId = partitionId; - this.leaderMerkleFiles = - leaderFiles.stream() - .map( - file -> - new MerkleFileContent( - file.getDigestKey().getFileXorHash(), - file.getDigestKey().getFileAddHash(), - Collections.emptyList(), - file.getSourceTsFilePath())) - .collect(Collectors.toList()); - this.leaderMerkleFileByPath = - this.leaderMerkleFiles.stream() - .collect( - Collectors.toMap( - MerkleFileContent::getSourceTsFilePath, - file -> file, - (left, right) -> left, - LinkedHashMap::new)); - this.leaderTsFileSizeByPath = - leaderFiles.stream() - .collect( - Collectors.toMap( - SnapshotFile::getSourceTsFilePath, - SnapshotFile::getTsFileSize, - (left, right) -> left, - LinkedHashMap::new)); - this.transferTargets = - transferTargets.entrySet().stream() - .collect( - Collectors.toMap( - Map.Entry::getKey, - entry -> Collections.unmodifiableList(new ArrayList<>(entry.getValue())), - (left, right) -> left, - LinkedHashMap::new)); + private EligibilitySnapshot fetchEligibility( + TDataNodeLocation location, boolean tolerateUnavailable) { + Object response = + sendRequest( + location.getInternalEndPoint(), + new TGetConsistencyEligibilityReq(consensusGroupId), + CnToDnSyncRequestType.GET_CONSISTENCY_ELIGIBILITY); + if (response == null) { + if (tolerateUnavailable) { + return EligibilitySnapshot.unavailable(); + } + throw new IllegalStateException("getConsistencyEligibility failed: null response"); + } + if (response instanceof TSStatus) { + TSStatus status = (TSStatus) response; + if (tolerateUnavailable) { + return EligibilitySnapshot.unavailable(); + } + throw new IllegalStateException("getConsistencyEligibility failed: " + status.getMessage()); } + if (!(response instanceof TGetConsistencyEligibilityResp)) { + throw new IllegalStateException( + "GET_CONSISTENCY_ELIGIBILITY returned unexpected response type " + + response.getClass().getSimpleName()); + } + TGetConsistencyEligibilityResp resp = (TGetConsistencyEligibilityResp) response; + requireSuccess(resp.getStatus(), "getConsistencyEligibility"); + return new EligibilitySnapshot(resp); + } - @Override - public long getPartitionId() { - return partitionId; + private void refreshLeaderEligibility() { + if (!isRouteBindingStillValid()) { + syncLag = Long.MAX_VALUE; + safeWatermark = Long.MIN_VALUE; + leaderEligibilityByPartition.clear(); + followerEligibilityByNodeId.clear(); + return; } + EligibilitySnapshot snapshot = fetchEligibility(leaderLocation, !repairMode); + if (!snapshot.isAvailable()) { + syncLag = Long.MAX_VALUE; + safeWatermark = Long.MIN_VALUE; + leaderEligibilityByPartition.clear(); + followerEligibilityByNodeId.clear(); + return; + } + syncLag = snapshot.syncLag; + safeWatermark = snapshot.safeWatermark; + leaderEligibilityByPartition.clear(); + leaderEligibilityByPartition.putAll(snapshot.partitionsById); + } - @Override - public boolean isRootHashMatched() { - return false; + private Map buildReplicaObservationTokens() { + Map tokens = new LinkedHashMap<>(); + for (Long partitionId : leaderEligibilityByPartition.keySet()) { + tokens.put(partitionId, buildReplicaObservationToken(partitionId)); } + return tokens; + } - @Override - public List getLeaderMerkleFiles() { - return leaderMerkleFiles; + private String buildReplicaObservationToken(long partitionId) { + StringBuilder builder = new StringBuilder(); + appendEligibilityToken( + builder, leaderLocation.getDataNodeId(), leaderEligibilityByPartition.get(partitionId)); + for (TDataNodeLocation followerLocation : followerLocations) { + builder.append('|'); + appendEligibilityToken( + builder, + followerLocation.getDataNodeId(), + followerEligibilityByNodeId + .getOrDefault(followerLocation.getDataNodeId(), Collections.emptyMap()) + .get(partitionId)); } + return builder.toString(); + } - @Override - public List getMismatchedLeaderMerkleFiles() { - return transferTargets.keySet().stream() - .map(leaderMerkleFileByPath::get) - .filter(Objects::nonNull) - .collect(Collectors.toList()); + private void appendEligibilityToken( + StringBuilder builder, int dataNodeId, TPartitionConsistencyEligibility eligibility) { + builder.append(dataNodeId).append(':'); + if (eligibility == null) { + builder.append("MISSING"); + return; } + builder + .append(eligibility.getPartitionMutationEpoch()) + .append(':') + .append(eligibility.getSnapshotEpoch()) + .append(':') + .append(eligibility.getSnapshotState()) + .append(':') + .append(eligibility.getLiveRootXorHash()) + .append(':') + .append(eligibility.getLiveRootAddHash()) + .append(':') + .append(eligibility.getTombstoneRootXorHash()) + .append(':') + .append(eligibility.getTombstoneRootAddHash()); + } - @Override - public RowRefIndex getRowRefIndex() { + private TPartitionConsistencyEligibility getEligibilityForLocation( + TDataNodeLocation location, long partitionId) { + if (location == null) { return null; } - - @Override - public List decodeDiffs() { - return Collections.emptyList(); + if (location.getDataNodeId() == leaderLocation.getDataNodeId()) { + return leaderEligibilityByPartition.get(partitionId); } + return followerEligibilityByNodeId + .getOrDefault(location.getDataNodeId(), Collections.emptyMap()) + .get(partitionId); + } - @Override - public boolean isDiffDecodeSuccessful() { + private boolean isSnapshotComparable(TPartitionConsistencyEligibility eligibility) { + return eligibility != null + && "READY".equalsIgnoreCase(eligibility.getSnapshotState()) + && eligibility.getSnapshotEpoch() == eligibility.getPartitionMutationEpoch(); + } + + private boolean sameRootDigest( + TPartitionConsistencyEligibility left, TPartitionConsistencyEligibility right) { + return left.getLiveRootXorHash() == right.getLiveRootXorHash() + && left.getLiveRootAddHash() == right.getLiveRootAddHash(); + } + + private boolean sameTombstoneDigest( + TPartitionConsistencyEligibility left, TPartitionConsistencyEligibility right) { + return left.getTombstoneRootXorHash() == right.getTombstoneRootXorHash() + && left.getTombstoneRootAddHash() == right.getTombstoneRootAddHash(); + } + + private boolean sameDigest(TSnapshotSubtreeNode left, TSnapshotSubtreeNode right) { + if (left == null || right == null) { return false; } + return left.getXorHash() == right.getXorHash() && left.getAddHash() == right.getAddHash(); + } - @Override - public long estimateDiffCount() { - return 0L; + private String buildRepairEpoch( + long partitionId, TPartitionConsistencyEligibility leaderEligibility) { + return leaderLocation.getDataNodeId() + + ":" + + partitionId + + ":" + + safeWatermark + + ":" + + leaderEligibility.getSnapshotEpoch() + + ":" + + leaderEligibility.getPartitionMutationEpoch() + + ":" + + routeVersionToken; + } + + private boolean isRouteBindingStillValid() { + int currentLeaderId = + configManager.getLoadManager().getRegionLeaderMap().getOrDefault(consensusGroupId, -1); + return currentLeaderId == leaderLocation.getDataNodeId() + && Objects.equals(routeVersionToken, buildRouteBindingToken(currentLeaderId)); + } + + private String buildRouteBindingToken(int leaderId) { + TRegionReplicaSet routeReplicaSet = + configManager.getLoadManager().getRegionPriorityMap().get(consensusGroupId); + if (routeReplicaSet == null || routeReplicaSet.getDataNodeLocations() == null) { + routeReplicaSet = + configManager + .getPartitionManager() + .getAllReplicaSetsMap(TConsensusGroupType.DataRegion) + .get(consensusGroupId); } + List routeNodeIds = + routeReplicaSet == null || routeReplicaSet.getDataNodeLocations() == null + ? Collections.emptyList() + : routeReplicaSet.getDataNodeLocations().stream() + .map(TDataNodeLocation::getDataNodeId) + .collect(Collectors.toList()); + return leaderId + + "-" + + routeNodeIds.stream().map(String::valueOf).collect(Collectors.joining("_")); + } - @Override - public List getFallbackTsFiles() { - return new ArrayList<>(transferTargets.keySet()); + private String encodeRepairSelector(LogicalMismatchScope.Scope scope) { + if ((scope.getExactKeys() == null || scope.getExactKeys().isEmpty()) + && scope.getKeyRangeStart() == null + && scope.getKeyRangeEnd() == null) { + return scope.getLeafId(); } + List exactKeys = + scope.getExactKeys() == null ? Collections.emptyList() : scope.getExactKeys(); + return scope.getLeafId() + + "@" + + encodeNullable(scope.getKeyRangeStart()) + + "@" + + encodeNullable(scope.getKeyRangeEnd()) + + "@" + + encodeNullable(String.join("\n", exactKeys)); + } - @Override - public long getTsFileSize(String tsFilePath) { - Long tsFileSize = leaderTsFileSizeByPath.get(tsFilePath); - if (tsFileSize == null) { - throw new IllegalArgumentException("Unknown leader TsFile " + tsFilePath); - } - return tsFileSize; + private String encodeNullable(String value) { + if (value == null) { + return ""; } + return java.util.Base64.getUrlEncoder() + .withoutPadding() + .encodeToString(value.getBytes(java.nio.charset.StandardCharsets.UTF_8)); + } - @Override - public int getTotalPointCount(String tsFilePath) { - return 0; + private String chooseKeyRangeStart( + TSnapshotSubtreeNode leaderLeaf, TSnapshotSubtreeNode followerLeaf) { + if (leaderLeaf != null && leaderLeaf.isSetKeyRangeStart()) { + return leaderLeaf.getKeyRangeStart(); } + return followerLeaf != null && followerLeaf.isSetKeyRangeStart() + ? followerLeaf.getKeyRangeStart() + : null; + } - @Override - public boolean shouldForceDirectTsFileTransfer() { - return true; + private String chooseKeyRangeEnd( + TSnapshotSubtreeNode leaderLeaf, TSnapshotSubtreeNode followerLeaf) { + if (leaderLeaf != null && leaderLeaf.isSetKeyRangeEnd()) { + return leaderLeaf.getKeyRangeEnd(); } + return followerLeaf != null && followerLeaf.isSetKeyRangeEnd() + ? followerLeaf.getKeyRangeEnd() + : null; + } - @Override - public boolean verify( - Map - repairPlans, - boolean rootHashMatched) { - return verifyPartition(partitionId); + private Object sendRequest(TEndPoint endPoint, Object req, CnToDnSyncRequestType requestType) { + return SyncDataNodeClientPool.getInstance() + .sendSyncRequestToDataNodeWithRetry(endPoint, req, requestType); + } + + private T sendRequestExpect( + TEndPoint endPoint, Object req, CnToDnSyncRequestType requestType, Class responseType) { + Object response = sendRequest(endPoint, req, requestType); + if (!responseType.isInstance(response)) { + throw new IllegalStateException( + requestType + + " returned unexpected response type " + + response.getClass().getSimpleName()); } + return responseType.cast(response); } - private static final class SnapshotFile { - private final String sourceTsFilePath; - private final long tsFileSize; - private final DigestKey digestKey; + private void requireSuccess(TSStatus status, String action) { + if (status == null || status.getCode() != TSStatusCode.SUCCESS_STATUS.getStatusCode()) { + throw new IllegalStateException( + action + " failed: " + (status == null ? "null status" : status.getMessage())); + } + } - private SnapshotFile(String sourceTsFilePath, long tsFileSize, DigestKey digestKey) { - this.sourceTsFilePath = sourceTsFilePath; - this.tsFileSize = tsFileSize; - this.digestKey = digestKey; + private static TConsensusGroupId requireDataRegion(TConsensusGroupId consensusGroupId) { + if (consensusGroupId == null || consensusGroupId.getType() != TConsensusGroupType.DataRegion) { + throw new IllegalArgumentException("Only DataRegion consistency repair is supported"); } + return consensusGroupId; + } - private String getSourceTsFilePath() { - return sourceTsFilePath; + private static class EligibilitySnapshot { + private final boolean available; + private final long syncLag; + private final long safeWatermark; + private final Map partitionsById = + new LinkedHashMap<>(); + + private EligibilitySnapshot() { + this.available = false; + this.syncLag = Long.MAX_VALUE; + this.safeWatermark = Long.MIN_VALUE; } - private long getTsFileSize() { - return tsFileSize; + private EligibilitySnapshot(TGetConsistencyEligibilityResp resp) { + this.available = true; + this.syncLag = resp.getSyncLag(); + this.safeWatermark = resp.getSafeWatermark(); + if (resp.isSetPartitions()) { + for (TPartitionConsistencyEligibility partition : resp.getPartitions()) { + partitionsById.put(partition.getTimePartitionId(), partition); + } + } } - private DigestKey getDigestKey() { - return digestKey; + private static EligibilitySnapshot unavailable() { + return new EligibilitySnapshot(); } - } - private static final class DigestKey { - private final long fileXorHash; - private final long fileAddHash; + private boolean isAvailable() { + return available; + } - private DigestKey(long fileXorHash, long fileAddHash) { - this.fileXorHash = fileXorHash; - this.fileAddHash = fileAddHash; + private Map getPartitionsMap() { + return partitionsById; } + } - private long getFileXorHash() { - return fileXorHash; + private static final class StaleSnapshotCompareException extends IllegalStateException { + private StaleSnapshotCompareException(String message) { + super(message); } + } - private long getFileAddHash() { - return fileAddHash; + private static class RepairLeafKey { + private final int followerDataNodeId; + private final String treeKind; + private final String leafId; + private final String scopeRef; + + private RepairLeafKey(int followerDataNodeId, String treeKind, String leafId, String scopeRef) { + this.followerDataNodeId = followerDataNodeId; + this.treeKind = treeKind; + this.leafId = leafId; + this.scopeRef = scopeRef; } @Override - public boolean equals(Object o) { - if (this == o) { + public boolean equals(Object obj) { + if (this == obj) { return true; } - if (!(o instanceof DigestKey)) { + if (!(obj instanceof RepairLeafKey)) { return false; } - DigestKey digestKey = (DigestKey) o; - return fileXorHash == digestKey.fileXorHash && fileAddHash == digestKey.fileAddHash; + RepairLeafKey that = (RepairLeafKey) obj; + return followerDataNodeId == that.followerDataNodeId + && Objects.equals(treeKind, that.treeKind) + && Objects.equals(leafId, that.leafId) + && Objects.equals(scopeRef, that.scopeRef); } @Override public int hashCode() { - return Objects.hash(fileXorHash, fileAddHash); + return Objects.hash(followerDataNodeId, treeKind, leafId, scopeRef); + } + } + + private static class RepairOperation { + private final long partitionId; + private final TDataNodeLocation followerLocation; + private final LogicalMismatchScope.Scope scope; + private final String repairEpoch; + + private RepairOperation( + long partitionId, + TDataNodeLocation followerLocation, + LogicalMismatchScope.Scope scope, + String repairEpoch) { + this.partitionId = partitionId; + this.followerLocation = followerLocation; + this.scope = scope; + this.repairEpoch = repairEpoch; + } + } + + private final class LivePartitionRepairContext + implements RepairRegionProcedure.PartitionRepairContext { + private final long partitionId; + private final boolean rootHashMatched; + private final List repairOperationIds; + private final String repairEpoch; + private final String blockingReason; + + private LivePartitionRepairContext( + long partitionId, + boolean rootHashMatched, + List repairOperationIds, + String repairEpoch, + String blockingReason) { + this.partitionId = partitionId; + this.rootHashMatched = rootHashMatched; + this.repairOperationIds = repairOperationIds; + this.repairEpoch = repairEpoch; + this.blockingReason = blockingReason; + } + + @Override + public long getPartitionId() { + return partitionId; + } + + @Override + public boolean isRootHashMatched() { + return rootHashMatched; + } + + @Override + public List getRepairOperationIds() { + return repairOperationIds; + } + + @Override + public String getRepairEpoch() { + return repairEpoch; } @Override - public String toString() { - return "DigestKey{" + "xor=" + fileXorHash + ", add=" + fileAddHash + '}'; + public String getBlockingReason() { + return blockingReason; + } + + @Override + public boolean verify() { + if (!isRouteBindingStillValid()) { + return false; + } + refreshLeaderEligibility(); + TPartitionConsistencyEligibility leaderEligibility = + leaderEligibilityByPartition.get(partitionId); + if (!isSnapshotComparable(leaderEligibility) + || !Objects.equals(repairEpoch, buildRepairEpoch(partitionId, leaderEligibility))) { + return false; + } + for (TDataNodeLocation followerLocation : followerLocations) { + TPartitionConsistencyEligibility followerEligibility = + fetchEligibility(followerLocation, false).getPartitionsMap().get(partitionId); + if (!isSnapshotComparable(followerEligibility) + || !sameRootDigest(leaderEligibility, followerEligibility) + || !sameTombstoneDigest(leaderEligibility, followerEligibility)) { + return false; + } + } + return true; } } } diff --git a/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/procedure/impl/consistency/RepairRegionProcedure.java b/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/procedure/impl/consistency/RepairRegionProcedure.java index 3c0d6008c50f1..019af5dd4c98b 100644 --- a/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/procedure/impl/consistency/RepairRegionProcedure.java +++ b/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/procedure/impl/consistency/RepairRegionProcedure.java @@ -21,18 +21,6 @@ import org.apache.iotdb.common.rpc.thrift.TConsensusGroupId; import org.apache.iotdb.commons.consensus.iotv2.consistency.RepairProgressTable; -import org.apache.iotdb.commons.consensus.iotv2.consistency.RepairProgressTable.RegionRepairStatus; -import org.apache.iotdb.commons.consensus.iotv2.consistency.ibf.DiffEntry; -import org.apache.iotdb.commons.consensus.iotv2.consistency.ibf.RowRefIndex; -import org.apache.iotdb.commons.consensus.iotv2.consistency.merkle.MerkleFileContent; -import org.apache.iotdb.commons.consensus.iotv2.consistency.repair.DiffAttribution; -import org.apache.iotdb.commons.consensus.iotv2.consistency.repair.ModEntrySummary; -import org.apache.iotdb.commons.consensus.iotv2.consistency.repair.RepairConflictResolver; -import org.apache.iotdb.commons.consensus.iotv2.consistency.repair.RepairCostModel; -import org.apache.iotdb.commons.consensus.iotv2.consistency.repair.RepairPlan; -import org.apache.iotdb.commons.consensus.iotv2.consistency.repair.RepairRecord; -import org.apache.iotdb.commons.consensus.iotv2.consistency.repair.RepairSession; -import org.apache.iotdb.commons.consensus.iotv2.consistency.repair.RepairStrategy; import org.apache.iotdb.confignode.procedure.env.ConfigNodeProcedureEnv; import org.apache.iotdb.confignode.procedure.impl.StateMachineProcedure; import org.apache.iotdb.confignode.procedure.state.consistency.RepairState; @@ -48,33 +36,30 @@ import java.util.ArrayList; import java.util.Collections; import java.util.Comparator; -import java.util.HashSet; -import java.util.LinkedHashMap; +import java.util.LinkedHashSet; import java.util.List; -import java.util.Map; import java.util.Objects; import java.util.Set; import java.util.UUID; import java.util.concurrent.ConcurrentHashMap; /** - * RepairRegionProcedure orchestrates the consistency check and repair lifecycle for a single - * consensus group. The procedure is intentionally isolated from transport details: all external - * interactions are supplied by a {@link RepairExecutionContext}, while the procedure itself - * persists the state-machine and applies the repair policy. + * RepairRegionProcedure orchestrates partition-scoped replica repair for a single consensus group. + * + *

The current repair path is logical-snapshot driven: ConfigNode first identifies the exact + * partition/leaf mismatch scope, then the procedure executes the corresponding logical repair + * operations and verifies the partition after repair. */ public class RepairRegionProcedure extends StateMachineProcedure { private static final Logger LOGGER = LoggerFactory.getLogger(RepairRegionProcedure.class); - private static final RepairCostModel REPAIR_COST_MODEL = new RepairCostModel(); private static final ConcurrentHashMap EXECUTION_CONTEXTS = new ConcurrentHashMap<>(); private TConsensusGroupId consensusGroupId; private long tSafe; - private long globalRepairedWatermark; private List pendingPartitions; private int currentPartitionIndex; private boolean hashMatched; @@ -84,15 +69,8 @@ public class RepairRegionProcedure private transient RepairExecutionContext executionContext; private transient RepairProgressTable repairProgressTable; private transient PartitionRepairContext currentPartitionContext; - private transient DiffAttribution diffAttribution; - private transient RowRefIndex currentRowRefIndex; - private transient Map> attributedDiffs; - private transient Map repairPlans; - private transient Set executedTsFileTransfers; - private transient Set executedPointStreamingPlans; - private transient List decodedDiffs; - private transient RepairSession repairSession; - private transient long estimatedDiffCount; + private transient List repairOperationIds; + private transient Set executedRepairOperations; /** Required for deserialization. */ public RepairRegionProcedure() { @@ -130,7 +108,7 @@ protected Flow executeFromState(ConfigNodeProcedureEnv env, RepairState state) case CHECK_SYNC_LAG: LOGGER.info("RepairRegionProcedure: CHECK_SYNC_LAG for group {}", consensusGroupId); - if (!checkSyncLagCompleted(env)) { + if (!requireExecutionContext().isReplicationComplete()) { LOGGER.info( "RepairRegionProcedure: skipping group {} because replication is not complete", consensusGroupId); @@ -142,120 +120,47 @@ protected Flow executeFromState(ConfigNodeProcedureEnv env, RepairState state) case COMPUTE_WATERMARK: LOGGER.info("RepairRegionProcedure: COMPUTE_WATERMARK for group {}", consensusGroupId); - computeWatermarkAndPartitions(env); + computeWatermarkAndPartitions(); if (pendingPartitions.isEmpty()) { - LOGGER.info("No pending partitions to check for group {}", consensusGroupId); + LOGGER.info("No pending partitions to repair for group {}", consensusGroupId); finishWithoutRepair(); return Flow.NO_MORE_STATE; } - setNextState(RepairState.BUILD_MERKLE_VIEW); + setNextState(RepairState.PREPARE_LOGICAL_SNAPSHOT); break; - case BUILD_MERKLE_VIEW: + case PREPARE_LOGICAL_SNAPSHOT: LOGGER.info( - "RepairRegionProcedure: BUILD_MERKLE_VIEW for partition {} of group {}", + "RepairRegionProcedure: PREPARE_LOGICAL_SNAPSHOT for partition {} of group {}", getCurrentPartitionId(), consensusGroupId); - buildMerkleView(env); + buildPartitionContext(); setNextState(RepairState.COMPARE_ROOT_HASH); break; case COMPARE_ROOT_HASH: - hashMatched = compareRootHash(env); + hashMatched = requireCurrentPartitionContext().isRootHashMatched(); if (hashMatched) { - LOGGER.info("Root hash matched for partition {}", getCurrentPartitionId()); + LOGGER.info("Partition {} is already matched", getCurrentPartitionId()); setNextState(RepairState.COMMIT_PARTITION); } else { - LOGGER.info("Root hash mismatched for partition {}", getCurrentPartitionId()); + LOGGER.info("Partition {} is mismatched", getCurrentPartitionId()); setNextState(RepairState.DRILL_DOWN); } break; case DRILL_DOWN: - LOGGER.info( - "RepairRegionProcedure: DRILL_DOWN for partition {}", getCurrentPartitionId()); - drillDown(env); - setNextState(RepairState.SMALL_TSFILE_SHORT_CIRCUIT); - break; - - case SMALL_TSFILE_SHORT_CIRCUIT: - if (prepareDirectTransferOnlyPlans()) { - setNextState(RepairState.EXECUTE_TSFILE_TRANSFER); - } else { - boolean hasSmallFiles = handleSmallTsFileShortCircuit(env); - if (hasLargeFilesNeedingIBF()) { - setNextState(RepairState.NEGOTIATE_KEY_MAPPING); - } else if (hasSmallFiles) { - setNextState(RepairState.EXECUTE_TSFILE_TRANSFER); - } else { - setNextState(RepairState.COMMIT_PARTITION); - } - } - break; - - case NEGOTIATE_KEY_MAPPING: - LOGGER.info("RepairRegionProcedure: NEGOTIATE_KEY_MAPPING"); - negotiateKeyMapping(env); - setNextState(RepairState.ESTIMATE_DIFF); + prepareRepairOperations(); + setNextState(RepairState.EXECUTE_REPAIR_OPERATIONS); break; - case ESTIMATE_DIFF: - estimateDiff(env); - setNextState(RepairState.EXCHANGE_IBF); - break; - - case EXCHANGE_IBF: - exchangeIBF(env); - setNextState(RepairState.DECODE_DIFF); - break; - - case DECODE_DIFF: - boolean decodeSuccess = decodeDiff(env); - if (decodeSuccess) { - setNextState(RepairState.ATTRIBUTE_DIFFS); - } else { - LOGGER.warn( - "IBF decode failed for partition {}, falling back to direct transfer", - getCurrentPartitionId()); - setNextState(RepairState.EXECUTE_TSFILE_TRANSFER); - } - break; - - case ATTRIBUTE_DIFFS: - attributeDiffs(env); - setNextState(RepairState.SELECT_REPAIR_STRATEGY); - break; - - case SELECT_REPAIR_STRATEGY: - selectRepairStrategy(env); - if (hasPendingTsFileTransfers()) { - setNextState(RepairState.EXECUTE_TSFILE_TRANSFER); - } else if (hasPendingPointStreaming()) { - setNextState(RepairState.EXECUTE_POINT_STREAMING); - } else { - setNextState(RepairState.VERIFY_REPAIR); - } - break; - - case EXECUTE_TSFILE_TRANSFER: - LOGGER.info("RepairRegionProcedure: EXECUTE_TSFILE_TRANSFER"); - executeTsFileTransfer(env); - if (hasPendingPointStreaming()) { - setNextState(RepairState.EXECUTE_POINT_STREAMING); - } else { - setNextState(RepairState.VERIFY_REPAIR); - } - break; - - case EXECUTE_POINT_STREAMING: - LOGGER.info("RepairRegionProcedure: EXECUTE_POINT_STREAMING"); - executePointStreaming(env); + case EXECUTE_REPAIR_OPERATIONS: + executeRepairOperations(); setNextState(RepairState.VERIFY_REPAIR); break; case VERIFY_REPAIR: - boolean verified = verifyRepair(env); - if (verified) { + if (hashMatched || requireCurrentPartitionContext().verify()) { setNextState(RepairState.COMMIT_PARTITION); } else { lastFailureReason = @@ -269,22 +174,22 @@ protected Flow executeFromState(ConfigNodeProcedureEnv env, RepairState state) break; case COMMIT_PARTITION: - commitPartition(env); + commitPartition(); currentPartitionIndex++; if (currentPartitionIndex < pendingPartitions.size()) { - setNextState(RepairState.BUILD_MERKLE_VIEW); + setNextState(RepairState.PREPARE_LOGICAL_SNAPSHOT); } else { setNextState(RepairState.ADVANCE_WATERMARK); } break; case ADVANCE_WATERMARK: - advanceWatermark(env); + advanceWatermark(); return Flow.NO_MORE_STATE; case ROLLBACK: LOGGER.warn("RepairRegionProcedure: ROLLBACK for group {}", consensusGroupId); - rollback(env); + rollback(); return Flow.NO_MORE_STATE; case DONE: @@ -309,25 +214,18 @@ protected Flow executeFromState(ConfigNodeProcedureEnv env, RepairState state) return Flow.HAS_MORE_STATE; } - private boolean checkSyncLagCompleted(ConfigNodeProcedureEnv env) { - return requireExecutionContext().isReplicationComplete(); - } - - private void computeWatermarkAndPartitions(ConfigNodeProcedureEnv env) { + private void computeWatermarkAndPartitions() { RepairExecutionContext context = requireExecutionContext(); RepairProgressTable progressTable = getOrCreateRepairProgressTable(); - progressTable.setRegionStatus(RegionRepairStatus.RUNNING); - globalRepairedWatermark = - Math.max(globalRepairedWatermark, progressTable.getGlobalRepairedWatermark()); - tSafe = Math.max(globalRepairedWatermark, context.computeSafeWatermark()); + tSafe = context.computeSafeWatermark(); pendingPartitions.clear(); - if (tSafe <= globalRepairedWatermark) { + if (tSafe == Long.MIN_VALUE) { return; } List candidatePartitions = - safeList(context.collectPendingPartitions(globalRepairedWatermark, tSafe, progressTable)); + safeList(context.collectPendingPartitions(tSafe, progressTable)); candidatePartitions.stream() .filter(Objects::nonNull) .filter(partitionId -> partitionId <= tSafe) @@ -341,7 +239,7 @@ private void computeWatermarkAndPartitions(ConfigNodeProcedureEnv env) { persistRepairProgressTable(progressTable); } - private void buildMerkleView(ConfigNodeProcedureEnv env) { + private void buildPartitionContext() { resetCurrentPartitionState(); long partitionId = getCurrentPartitionId(); currentPartitionContext = requireExecutionContext().getPartitionContext(partitionId); @@ -361,249 +259,101 @@ private void buildMerkleView(ConfigNodeProcedureEnv env) { } } - private boolean compareRootHash(ConfigNodeProcedureEnv env) { - return requireCurrentPartitionContext().isRootHashMatched(); - } - - private void drillDown(ConfigNodeProcedureEnv env) { + private void prepareRepairOperations() { PartitionRepairContext partitionContext = requireCurrentPartitionContext(); - List mismatchedFiles = - safeList(partitionContext.getMismatchedLeaderMerkleFiles()); LOGGER.info( - "Partition {} has {} mismatched TsFiles after Merkle drill-down", + "Partition {} has {} logical repair operations in the cached mismatch scope", partitionContext.getPartitionId(), - mismatchedFiles.size()); - if (mismatchedFiles.isEmpty()) { - LOGGER.warn( - "Partition {} reported root mismatch but returned no mismatched TsFiles", - partitionContext.getPartitionId()); - } - } - - private boolean prepareDirectTransferOnlyPlans() { - PartitionRepairContext partitionContext = requireCurrentPartitionContext(); - if (!partitionContext.shouldForceDirectTsFileTransfer()) { - return false; - } + safeList(partitionContext.getRepairOperationIds()).size()); - List fallbackTsFiles = safeList(partitionContext.getFallbackTsFiles()); - if (fallbackTsFiles.isEmpty()) { + List operationIds = safeList(partitionContext.getRepairOperationIds()); + if (operationIds.isEmpty()) { + String blockingReason = partitionContext.getBlockingReason(); throw new IllegalStateException( - "Partition " - + partitionContext.getPartitionId() - + " requested direct TsFile transfer but provided no fallback files"); + blockingReason != null + ? blockingReason + : ("Partition " + + partitionContext.getPartitionId() + + " is mismatched but no repair operations were provided")); } - for (String tsFilePath : fallbackTsFiles) { - repairPlans.putIfAbsent( - tsFilePath, - RepairPlan.directTransfer(tsFilePath, partitionContext.getTsFileSize(tsFilePath))); + for (String operationId : operationIds) { + repairOperationIds.add(operationId); } - return true; - } - private boolean handleSmallTsFileShortCircuit(ConfigNodeProcedureEnv env) { - boolean hasSmallFiles = false; - for (MerkleFileContent content : - safeList(requireCurrentPartitionContext().getMismatchedLeaderMerkleFiles())) { - String tsFilePath = content.getSourceTsFilePath(); - long tsFileSize = requireCurrentPartitionContext().getTsFileSize(tsFilePath); - if (REPAIR_COST_MODEL.shouldBypassIBF(tsFileSize)) { - hasSmallFiles = true; - repairPlans.put(tsFilePath, RepairPlan.directTransfer(tsFilePath, tsFileSize)); - } - } - return hasSmallFiles; - } - - private boolean hasLargeFilesNeedingIBF() { - for (MerkleFileContent content : - safeList(requireCurrentPartitionContext().getMismatchedLeaderMerkleFiles())) { - String tsFilePath = content.getSourceTsFilePath(); - if (repairPlans.containsKey(tsFilePath)) { - continue; - } - long tsFileSize = requireCurrentPartitionContext().getTsFileSize(tsFilePath); - if (!REPAIR_COST_MODEL.shouldBypassIBF(tsFileSize)) { - return true; - } - } - return false; - } - - private void negotiateKeyMapping(ConfigNodeProcedureEnv env) { - currentRowRefIndex = requireCurrentPartitionContext().getRowRefIndex(); - if (currentRowRefIndex == null) { - throw new IllegalStateException( - "Large-file diff localization requires a RowRefIndex for partition " - + getCurrentPartitionId()); - } - } - - private void estimateDiff(ConfigNodeProcedureEnv env) { - estimatedDiffCount = requireCurrentPartitionContext().estimateDiffCount(); - if (estimatedDiffCount <= 0 && requireCurrentPartitionContext().isDiffDecodeSuccessful()) { - estimatedDiffCount = safeList(requireCurrentPartitionContext().decodeDiffs()).size(); - } - LOGGER.info( - "Estimated diff count for partition {} is {}", getCurrentPartitionId(), estimatedDiffCount); - } - - private void exchangeIBF(ConfigNodeProcedureEnv env) { - decodedDiffs.clear(); - decodedDiffs.addAll(safeList(requireCurrentPartitionContext().decodeDiffs())); - } - - private boolean decodeDiff(ConfigNodeProcedureEnv env) { - PartitionRepairContext partitionContext = requireCurrentPartitionContext(); - if (!partitionContext.isDiffDecodeSuccessful()) { - for (String tsFilePath : safeList(partitionContext.getFallbackTsFiles())) { - repairPlans.put( - tsFilePath, - RepairPlan.directTransfer(tsFilePath, partitionContext.getTsFileSize(tsFilePath))); - } - return false; - } - if (decodedDiffs.isEmpty()) { - decodedDiffs.addAll(safeList(partitionContext.decodeDiffs())); - } - return true; - } - - private void attributeDiffs(ConfigNodeProcedureEnv env) { - attributedDiffs = - diffAttribution.attributeToSourceTsFiles( - decodedDiffs, - currentRowRefIndex, - safeList(requireCurrentPartitionContext().getLeaderMerkleFiles())); - } - - private void selectRepairStrategy(ConfigNodeProcedureEnv env) { - PartitionRepairContext partitionContext = requireCurrentPartitionContext(); - for (Map.Entry> entry : attributedDiffs.entrySet()) { - String tsFilePath = entry.getKey(); - if (repairPlans.containsKey(tsFilePath)) { - continue; - } - List diffs = entry.getValue(); - long tsFileSize = partitionContext.getTsFileSize(tsFilePath); - int totalPointCount = partitionContext.getTotalPointCount(tsFilePath); - RepairStrategy strategy = - REPAIR_COST_MODEL.selectStrategy(tsFileSize, totalPointCount, diffs.size()); - if (strategy == RepairStrategy.DIRECT_TSFILE_TRANSFER) { - repairPlans.put(tsFilePath, RepairPlan.directTransfer(tsFilePath, tsFileSize)); - } else { - repairPlans.put(tsFilePath, RepairPlan.pointStreaming(tsFilePath, tsFileSize, diffs)); - } - } - } - - private boolean hasPendingTsFileTransfers() { - return repairPlans.values().stream() - .anyMatch( - plan -> - plan.getStrategy() == RepairStrategy.DIRECT_TSFILE_TRANSFER - && !executedTsFileTransfers.contains(plan.getTsFilePath())); - } - - private boolean hasPendingPointStreaming() { - return repairPlans.values().stream() - .anyMatch( - plan -> - plan.getStrategy() == RepairStrategy.POINT_STREAMING - && !executedPointStreamingPlans.contains(plan.getTsFilePath())); + RepairProgressTable progressTable = getOrCreateRepairProgressTable(); + String repairEpoch = + partitionContext.getRepairEpoch() != null + ? partitionContext.getRepairEpoch() + : progressTable + .getOrCreatePartition(partitionContext.getPartitionId()) + .getRepairEpoch(); + progressTable.markRepairRunning(partitionContext.getPartitionId(), repairEpoch); + persistRepairProgressTable(progressTable); } - private void executeTsFileTransfer(ConfigNodeProcedureEnv env) { - for (RepairPlan plan : repairPlans.values()) { - if (plan.getStrategy() != RepairStrategy.DIRECT_TSFILE_TRANSFER - || executedTsFileTransfers.contains(plan.getTsFilePath())) { + private void executeRepairOperations() throws Exception { + for (String operationId : repairOperationIds) { + if (!executedRepairOperations.add(operationId)) { continue; } - try { - requireExecutionContext().transferTsFile(plan.getTsFilePath()); - executedTsFileTransfers.add(plan.getTsFilePath()); - } catch (Exception e) { - throw new IllegalStateException( - "Failed to transfer TsFile " + plan.getTsFilePath() + ": " + e.getMessage(), e); - } + requireExecutionContext().executeRepairOperation(operationId); } } - private void executePointStreaming(ConfigNodeProcedureEnv env) { - RepairConflictResolver conflictResolver = - new RepairConflictResolver( - safeList(requireCurrentPartitionContext().getLeaderDeletions()), - safeList(requireCurrentPartitionContext().getFollowerDeletions())); - repairSession = requireExecutionContext().createRepairSession(getCurrentPartitionId()); - - for (RepairPlan plan : repairPlans.values()) { - if (plan.getStrategy() != RepairStrategy.POINT_STREAMING - || executedPointStreamingPlans.contains(plan.getTsFilePath())) { - continue; - } - for (DiffEntry diff : plan.getDiffs()) { - RepairRecord record = - requireExecutionContext() - .buildRepairRecord( - requireCurrentPartitionContext(), diff, currentRowRefIndex, conflictResolver); - if (record != null) { - repairSession.stage(record); - } - } - executedPointStreamingPlans.add(plan.getTsFilePath()); - } - - if (repairSession.getStagedCount() > 0 && !repairSession.promoteAtomically()) { - throw new IllegalStateException( - "Failed to atomically promote repair session " + repairSession.getSessionId()); - } - } - - private boolean verifyRepair(ConfigNodeProcedureEnv env) { - return hashMatched || requireCurrentPartitionContext().verify(repairPlans, hashMatched); - } - - private void commitPartition(ConfigNodeProcedureEnv env) { + private void commitPartition() { long partitionId = getCurrentPartitionId(); RepairProgressTable progressTable = getOrCreateRepairProgressTable(); - long repairedTo = - Math.max(tSafe, progressTable.getOrCreatePartition(partitionId).getRepairedTo()); - progressTable.commitPartition(partitionId, repairedTo); - requireExecutionContext().onPartitionCommitted(partitionId, repairedTo, progressTable); - if (repairSession != null) { - repairSession.cleanup(); + long checkedAt = System.currentTimeMillis(); + requireExecutionContext().onPartitionCommitted(partitionId, checkedAt, progressTable); + RepairProgressTable.PartitionProgress progress = + progressTable.getOrCreatePartition(partitionId); + if (progress.getCheckState() != RepairProgressTable.CheckState.VERIFIED) { + progressTable.markRepairSucceeded( + partitionId, + checkedAt, + tSafe, + progress.getPartitionMutationEpoch(), + progress.getSnapshotEpoch(), + progress.getSnapshotState(), + progress.getRepairEpoch()); } persistRepairProgressTable(progressTable); LOGGER.info("Committed partition {} for group {}", partitionId, consensusGroupId); resetCurrentPartitionState(); } - private void advanceWatermark(ConfigNodeProcedureEnv env) { - RepairProgressTable progressTable = getOrCreateRepairProgressTable(); - globalRepairedWatermark = progressTable.advanceGlobalWatermark(); - progressTable.setRegionStatus(RegionRepairStatus.IDLE); - requireExecutionContext().onWatermarkAdvanced(globalRepairedWatermark, progressTable); - persistRepairProgressTable(progressTable); + private void advanceWatermark() { + persistRepairProgressTable(getOrCreateRepairProgressTable()); LOGGER.info( - "Advanced repair watermark for group {} to {}", consensusGroupId, globalRepairedWatermark); + "Finished repair procedure for group {} at safe watermark {}", consensusGroupId, tSafe); cleanupExecutionContext(); } - protected void rollback(ConfigNodeProcedureEnv env) { + protected void rollback() { long partitionId = getCurrentPartitionId(); RepairProgressTable progressTable = getOrCreateRepairProgressTable(); if (partitionId >= 0) { - progressTable.failPartition( - partitionId, lastFailureReason == null ? "Unknown repair failure" : lastFailureReason); - } - progressTable.setRegionStatus(RegionRepairStatus.FAILED); - if (repairSession != null) { - repairSession.abort(); + RepairProgressTable.PartitionProgress partitionProgress = + progressTable.getOrCreatePartition(partitionId); + progressTable.markRepairFailed( + partitionId, + partitionProgress.getRepairEpoch(), + "REPAIR_FAILED", + lastFailureReason == null ? "Unknown repair failure" : lastFailureReason); + progressTable.markCheckFailed( + partitionId, + System.currentTimeMillis(), + tSafe, + partitionProgress.getPartitionMutationEpoch(), + partitionProgress.getSnapshotEpoch(), + partitionProgress.getSnapshotState(), + "REPAIR_FAILED", + lastFailureReason == null ? "Unknown repair failure" : lastFailureReason); } RepairExecutionContext context = getExecutionContextIfPresent(); if (context != null && partitionId >= 0) { - context.rollbackPartition(partitionId, repairSession, progressTable); + context.rollbackPartition(partitionId, progressTable); } persistRepairProgressTable(progressTable); LOGGER.warn("Rolled back repair for group {}", consensusGroupId); @@ -648,7 +398,6 @@ public void serialize(DataOutputStream stream) throws IOException { stream.writeInt(consensusGroupId.getType().getValue()); } stream.writeLong(tSafe); - stream.writeLong(globalRepairedWatermark); stream.writeInt(currentPartitionIndex); stream.writeBoolean(hashMatched); writeString(stream, executionContextId); @@ -676,7 +425,6 @@ public void deserialize(ByteBuffer byteBuffer) { } this.tSafe = byteBuffer.getLong(); - this.globalRepairedWatermark = byteBuffer.getLong(); this.currentPartitionIndex = byteBuffer.getInt(); this.hashMatched = byteBuffer.get() != 0; this.executionContextId = readString(byteBuffer); @@ -694,10 +442,6 @@ public TConsensusGroupId getConsensusGroupId() { return consensusGroupId; } - public long getGlobalRepairedWatermark() { - return globalRepairedWatermark; - } - public String getExecutionContextId() { return executionContextId; } @@ -728,7 +472,6 @@ public boolean equals(Object o) { } RepairRegionProcedure that = (RepairRegionProcedure) o; return tSafe == that.tSafe - && globalRepairedWatermark == that.globalRepairedWatermark && currentPartitionIndex == that.currentPartitionIndex && hashMatched == that.hashMatched && Objects.equals(consensusGroupId, that.consensusGroupId) @@ -742,7 +485,6 @@ public int hashCode() { return Objects.hash( consensusGroupId, tSafe, - globalRepairedWatermark, pendingPartitions, currentPartitionIndex, hashMatched, @@ -754,28 +496,15 @@ private void initializeTransientState() { this.executionContext = getExecutionContextIfPresent(); this.currentPartitionContext = null; this.repairProgressTable = null; - this.diffAttribution = new DiffAttribution(); - this.currentRowRefIndex = null; - this.attributedDiffs = new LinkedHashMap<>(); - this.repairPlans = new LinkedHashMap<>(); - this.executedTsFileTransfers = new HashSet<>(); - this.executedPointStreamingPlans = new HashSet<>(); - this.decodedDiffs = new ArrayList<>(); - this.repairSession = null; - this.estimatedDiffCount = 0L; + this.repairOperationIds = new ArrayList<>(); + this.executedRepairOperations = new LinkedHashSet<>(); } private void resetCurrentPartitionState() { this.currentPartitionContext = null; - this.currentRowRefIndex = null; - this.attributedDiffs.clear(); - this.repairPlans.clear(); - this.executedTsFileTransfers.clear(); - this.executedPointStreamingPlans.clear(); - this.decodedDiffs.clear(); - this.repairSession = null; + this.repairOperationIds.clear(); + this.executedRepairOperations.clear(); this.hashMatched = false; - this.estimatedDiffCount = 0L; this.lastFailureReason = null; } @@ -816,23 +545,12 @@ private RepairProgressTable getOrCreateRepairProgressTable() { if (repairProgressTable == null) { repairProgressTable = new RepairProgressTable(toConsensusGroupKey(consensusGroupId)); } - repairProgressTable.setRegionStatus(RegionRepairStatus.RUNNING); - for (int i = 0; i < pendingPartitions.size(); i++) { - long partitionId = pendingPartitions.get(i); - RepairProgressTable.PartitionProgress progress = - repairProgressTable.getOrCreatePartition(partitionId); - if (i < currentPartitionIndex) { - progress.markVerified(tSafe); - } - } - repairProgressTable.advanceGlobalWatermark(); } return repairProgressTable; } private void finishWithoutRepair() { if (repairProgressTable != null) { - repairProgressTable.setRegionStatus(RegionRepairStatus.IDLE); persistRepairProgressTable(repairProgressTable); } cleanupExecutionContext(); @@ -895,7 +613,7 @@ private static String readString(ByteBuffer byteBuffer) { /** * Bridge between the state machine and the transport/runtime-specific implementation that - * provides Merkle snapshots, decoded diffs and repair primitives. + * provides logical-snapshot mismatch repair operations. */ public interface RepairExecutionContext extends AutoCloseable { @@ -904,21 +622,11 @@ public interface RepairExecutionContext extends AutoCloseable { long computeSafeWatermark(); List collectPendingPartitions( - long globalRepairedWatermark, long safeWatermark, RepairProgressTable repairProgressTable); + long safeWatermark, RepairProgressTable repairProgressTable); PartitionRepairContext getPartitionContext(long partitionId); - RepairRecord buildRepairRecord( - PartitionRepairContext partitionContext, - DiffEntry diffEntry, - RowRefIndex rowRefIndex, - RepairConflictResolver conflictResolver); - - void transferTsFile(String tsFilePath) throws Exception; - - default RepairSession createRepairSession(long partitionId) { - return new RepairSession(partitionId); - } + void executeRepairOperation(String operationId) throws Exception; default RepairProgressTable loadRepairProgressTable(String consensusGroupKey) { return null; @@ -927,13 +635,9 @@ default RepairProgressTable loadRepairProgressTable(String consensusGroupKey) { default void persistRepairProgressTable(RepairProgressTable repairProgressTable) {} default void onPartitionCommitted( - long partitionId, long repairedTo, RepairProgressTable repairProgressTable) {} + long partitionId, long committedAt, RepairProgressTable repairProgressTable) {} - default void onWatermarkAdvanced( - long globalWatermark, RepairProgressTable repairProgressTable) {} - - default void rollbackPartition( - long partitionId, RepairSession repairSession, RepairProgressTable repairProgressTable) {} + default void rollbackPartition(long partitionId, RepairProgressTable repairProgressTable) {} @Override default void close() {} @@ -946,37 +650,17 @@ public interface PartitionRepairContext { boolean isRootHashMatched(); - List getLeaderMerkleFiles(); - - List getMismatchedLeaderMerkleFiles(); - - RowRefIndex getRowRefIndex(); - - List decodeDiffs(); - - boolean isDiffDecodeSuccessful(); - - long estimateDiffCount(); - - List getFallbackTsFiles(); + List getRepairOperationIds(); - long getTsFileSize(String tsFilePath); - - int getTotalPointCount(String tsFilePath); - - default List getLeaderDeletions() { - return Collections.emptyList(); - } - - default List getFollowerDeletions() { - return Collections.emptyList(); + default String getRepairEpoch() { + return null; } - default boolean shouldForceDirectTsFileTransfer() { - return false; + default String getBlockingReason() { + return null; } - default boolean verify(Map repairPlans, boolean rootHashMatched) { + default boolean verify() { return true; } } diff --git a/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/procedure/state/consistency/RepairState.java b/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/procedure/state/consistency/RepairState.java index 32062e55a6099..f0d263380de51 100644 --- a/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/procedure/state/consistency/RepairState.java +++ b/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/procedure/state/consistency/RepairState.java @@ -24,18 +24,10 @@ public enum RepairState { INIT, CHECK_SYNC_LAG, COMPUTE_WATERMARK, - BUILD_MERKLE_VIEW, + PREPARE_LOGICAL_SNAPSHOT, COMPARE_ROOT_HASH, DRILL_DOWN, - SMALL_TSFILE_SHORT_CIRCUIT, - NEGOTIATE_KEY_MAPPING, - ESTIMATE_DIFF, - EXCHANGE_IBF, - DECODE_DIFF, - ATTRIBUTE_DIFFS, - SELECT_REPAIR_STRATEGY, - EXECUTE_TSFILE_TRANSFER, - EXECUTE_POINT_STREAMING, + EXECUTE_REPAIR_OPERATIONS, VERIFY_REPAIR, COMMIT_PARTITION, ADVANCE_WATERMARK, diff --git a/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/service/thrift/ConfigNodeRPCServiceProcessor.java b/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/service/thrift/ConfigNodeRPCServiceProcessor.java index 962a2e6caa8fe..c9a08f0b331c5 100644 --- a/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/service/thrift/ConfigNodeRPCServiceProcessor.java +++ b/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/service/thrift/ConfigNodeRPCServiceProcessor.java @@ -207,6 +207,7 @@ import org.apache.iotdb.confignode.rpc.thrift.TShowPipeResp; import org.apache.iotdb.confignode.rpc.thrift.TShowRegionReq; import org.apache.iotdb.confignode.rpc.thrift.TShowRegionResp; +import org.apache.iotdb.confignode.rpc.thrift.TShowRepairProgressResp; import org.apache.iotdb.confignode.rpc.thrift.TShowSubscriptionReq; import org.apache.iotdb.confignode.rpc.thrift.TShowSubscriptionResp; import org.apache.iotdb.confignode.rpc.thrift.TShowTTLResp; @@ -1098,6 +1099,11 @@ public TShowConfigNodes4InformationSchemaResp showConfigNodes4InformationSchema( return configManager.showConfigNodes4InformationSchema(); } + @Override + public TShowRepairProgressResp showRepairProgress() { + return configManager.showRepairProgress(); + } + @Override public TShowDatabaseResp showDatabase(TGetDatabaseReq req) { return configManager.showDatabase(req); diff --git a/iotdb-core/confignode/src/test/java/org/apache/iotdb/confignode/manager/consistency/ConsistencyCheckSchedulerTest.java b/iotdb-core/confignode/src/test/java/org/apache/iotdb/confignode/manager/consistency/ConsistencyCheckSchedulerTest.java new file mode 100644 index 0000000000000..6baf304af99cc --- /dev/null +++ b/iotdb-core/confignode/src/test/java/org/apache/iotdb/confignode/manager/consistency/ConsistencyCheckSchedulerTest.java @@ -0,0 +1,86 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.iotdb.confignode.manager.consistency; + +import org.apache.iotdb.common.rpc.thrift.TConsensusGroupId; +import org.apache.iotdb.common.rpc.thrift.TConsensusGroupType; +import org.apache.iotdb.common.rpc.thrift.TRegionReplicaSet; +import org.apache.iotdb.commons.consensus.iotv2.consistency.RepairProgressTable; +import org.apache.iotdb.confignode.manager.ConfigManager; +import org.apache.iotdb.confignode.manager.ProcedureManager; +import org.apache.iotdb.confignode.manager.partition.PartitionManager; +import org.apache.iotdb.confignode.persistence.ConsistencyProgressInfo; + +import org.junit.Assert; +import org.junit.Test; + +import java.util.ArrayList; +import java.util.LinkedHashMap; +import java.util.List; +import java.util.Map; + +import static org.mockito.Mockito.mock; +import static org.mockito.Mockito.when; + +public class ConsistencyCheckSchedulerTest { + + @Test + public void schedulerShouldSkipRegionsWithRunningRepairProcedure() { + ConfigManager configManager = mock(ConfigManager.class); + ProcedureManager procedureManager = mock(ProcedureManager.class); + PartitionManager partitionManager = mock(PartitionManager.class); + + TConsensusGroupId region1 = new TConsensusGroupId(TConsensusGroupType.DataRegion, 1); + TConsensusGroupId region2 = new TConsensusGroupId(TConsensusGroupType.DataRegion, 2); + Map replicaSets = new LinkedHashMap<>(); + replicaSets.put(region1, new TRegionReplicaSet()); + replicaSets.put(region2, new TRegionReplicaSet()); + + ConsistencyProgressManager progressManager = + new ConsistencyProgressManager(new ConsistencyProgressInfo()); + List checkedRegions = new ArrayList<>(); + + when(configManager.getPartitionManager()).thenReturn(partitionManager); + when(configManager.getConsistencyProgressManager()).thenReturn(progressManager); + when(partitionManager.getAllReplicaSetsMap(TConsensusGroupType.DataRegion)) + .thenReturn(replicaSets); + when(procedureManager.hasRunningRepairProcedure(region1)).thenReturn(false); + when(procedureManager.hasRunningRepairProcedure(region2)).thenReturn(true); + + ConsistencyCheckScheduler scheduler = + new ConsistencyCheckScheduler( + configManager, + procedureManager, + 0L, + 1L, + (manager, consensusGroupId, progressTable) -> { + checkedRegions.add(consensusGroupId); + progressTable.markVerified( + 100L, 1000L, 2000L, 3000L, 3000L, RepairProgressTable.SnapshotState.READY); + }); + + scheduler.runOneRound(); + + Assert.assertEquals(1, checkedRegions.size()); + Assert.assertEquals(region1, checkedRegions.get(0)); + Assert.assertNotNull(progressManager.loadRepairProgressTable(region1).getPartition(100L)); + Assert.assertNull(progressManager.loadRepairProgressTable(region2).getPartition(100L)); + } +} diff --git a/iotdb-core/confignode/src/test/java/org/apache/iotdb/confignode/persistence/ConsistencyProgressInfoTest.java b/iotdb-core/confignode/src/test/java/org/apache/iotdb/confignode/persistence/ConsistencyProgressInfoTest.java new file mode 100644 index 0000000000000..c0890c0f5a1b7 --- /dev/null +++ b/iotdb-core/confignode/src/test/java/org/apache/iotdb/confignode/persistence/ConsistencyProgressInfoTest.java @@ -0,0 +1,60 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.iotdb.confignode.persistence; + +import org.apache.iotdb.commons.consensus.iotv2.consistency.RepairProgressTable; + +import org.junit.Assert; +import org.junit.Rule; +import org.junit.Test; +import org.junit.rules.TemporaryFolder; + +import java.io.File; + +public class ConsistencyProgressInfoTest { + + @Rule public TemporaryFolder temporaryFolder = new TemporaryFolder(); + + @Test + public void snapshotShouldRestorePersistedProgressTables() throws Exception { + ConsistencyProgressInfo info = new ConsistencyProgressInfo(); + RepairProgressTable table = new RepairProgressTable("DataRegion-5"); + table.markMismatch( + 100L, + 1000L, + 2000L, + 3000L, + 3000L, + RepairProgressTable.SnapshotState.READY, + "LIVE@leaf:5:0", + 1, + "leader:5:2000:3000:3000"); + table.markRepairRunning(100L, "leader:5:2000:3000:3000"); + info.updateTable(table); + + File snapshotDir = temporaryFolder.newFolder("consistency-progress-snapshot"); + Assert.assertTrue(info.processTakeSnapshot(snapshotDir)); + + ConsistencyProgressInfo recovered = new ConsistencyProgressInfo(); + recovered.processLoadSnapshot(snapshotDir); + + Assert.assertEquals(info.getAllTables(), recovered.getAllTables()); + } +} diff --git a/iotdb-core/confignode/src/test/java/org/apache/iotdb/confignode/procedure/impl/consistency/ConsistencyPartitionSelectorTest.java b/iotdb-core/confignode/src/test/java/org/apache/iotdb/confignode/procedure/impl/consistency/ConsistencyPartitionSelectorTest.java new file mode 100644 index 0000000000000..833ce319f2f87 --- /dev/null +++ b/iotdb-core/confignode/src/test/java/org/apache/iotdb/confignode/procedure/impl/consistency/ConsistencyPartitionSelectorTest.java @@ -0,0 +1,141 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.iotdb.confignode.procedure.impl.consistency; + +import org.apache.iotdb.commons.consensus.iotv2.consistency.RepairProgressTable; +import org.apache.iotdb.mpp.rpc.thrift.TPartitionConsistencyEligibility; + +import org.junit.Assert; +import org.junit.Test; + +import java.util.Arrays; +import java.util.Collections; +import java.util.LinkedHashMap; +import java.util.List; +import java.util.Map; + +public class ConsistencyPartitionSelectorTest { + + @Test + public void verifiedPartitionWithSameSnapshotShouldNotBeRequeued() { + RepairProgressTable progressTable = new RepairProgressTable("DataRegion-1"); + progressTable.markVerified(1L, 100L, 200L, 300L, 300L, RepairProgressTable.SnapshotState.READY); + + Map eligibilityByPartition = new LinkedHashMap<>(); + eligibilityByPartition.put(1L, eligibility(1L, 300L, 300L, "READY")); + eligibilityByPartition.put(2L, eligibility(2L, 301L, 301L, "READY")); + + List selected = + ConsistencyPartitionSelector.selectCandidatePartitions( + eligibilityByPartition, Collections.emptySet(), false, null, progressTable); + + Assert.assertEquals(Collections.singletonList(2L), selected); + } + + @Test + public void dirtyAndBuildingPartitionsShouldBeRechecked() { + RepairProgressTable progressTable = new RepairProgressTable("DataRegion-1"); + progressTable.markVerified(1L, 100L, 200L, 300L, 300L, RepairProgressTable.SnapshotState.READY); + progressTable.markVerified(2L, 100L, 200L, 400L, 400L, RepairProgressTable.SnapshotState.READY); + + Map eligibilityByPartition = new LinkedHashMap<>(); + eligibilityByPartition.put(1L, eligibility(1L, 301L, 301L, "READY")); + eligibilityByPartition.put(2L, eligibility(2L, 400L, 400L, "BUILDING")); + + List selected = + ConsistencyPartitionSelector.selectCandidatePartitions( + eligibilityByPartition, Collections.emptySet(), false, null, progressTable); + + Assert.assertEquals(Arrays.asList(1L, 2L), selected); + } + + @Test + public void verifiedPartitionShouldBeRequeuedWhenReplicaObservationChanges() { + RepairProgressTable progressTable = new RepairProgressTable("DataRegion-1"); + progressTable.markVerified( + 1L, + 100L, + 200L, + 300L, + 300L, + RepairProgressTable.SnapshotState.READY, + "4:300:300:READY|3:300:300:READY"); + + Map eligibilityByPartition = new LinkedHashMap<>(); + eligibilityByPartition.put(1L, eligibility(1L, 300L, 300L, "READY")); + + Map replicaObservationTokens = new LinkedHashMap<>(); + replicaObservationTokens.put(1L, "4:300:300:READY|3:0:0:READY"); + + List selected = + ConsistencyPartitionSelector.selectCandidatePartitions( + eligibilityByPartition, + Collections.emptySet(), + false, + null, + progressTable, + replicaObservationTokens); + + Assert.assertEquals(Collections.singletonList(1L), selected); + } + + @Test + public void repairModeShouldPreferLatestMismatchScope() { + RepairProgressTable progressTable = new RepairProgressTable("DataRegion-1"); + progressTable.markMismatch( + 1L, + 100L, + 200L, + 300L, + 300L, + RepairProgressTable.SnapshotState.READY, + "LIVE@leaf:1:0", + 1, + "leader:1:200:300:300"); + progressTable.markVerified(2L, 100L, 200L, 400L, 400L, RepairProgressTable.SnapshotState.READY); + + Map eligibilityByPartition = new LinkedHashMap<>(); + eligibilityByPartition.put(1L, eligibility(1L, 300L, 300L, "READY")); + eligibilityByPartition.put(2L, eligibility(2L, 401L, 401L, "READY")); + + List selected = + ConsistencyPartitionSelector.selectCandidatePartitions( + eligibilityByPartition, + Collections.emptySet(), + true, + "leader:1:200:300:300", + progressTable); + + Assert.assertEquals(Collections.singletonList(1L), selected); + } + + private static TPartitionConsistencyEligibility eligibility( + long partitionId, long partitionMutationEpoch, long snapshotEpoch, String snapshotState) { + return new TPartitionConsistencyEligibility( + partitionId, + partitionMutationEpoch, + snapshotEpoch, + snapshotState, + partitionId, + partitionId, + partitionId, + partitionId); + } +} diff --git a/iotdb-core/confignode/src/test/java/org/apache/iotdb/confignode/procedure/impl/consistency/RepairRegionProcedureTest.java b/iotdb-core/confignode/src/test/java/org/apache/iotdb/confignode/procedure/impl/consistency/RepairRegionProcedureTest.java index 54533991a9cd8..a882c36ba4857 100644 --- a/iotdb-core/confignode/src/test/java/org/apache/iotdb/confignode/procedure/impl/consistency/RepairRegionProcedureTest.java +++ b/iotdb-core/confignode/src/test/java/org/apache/iotdb/confignode/procedure/impl/consistency/RepairRegionProcedureTest.java @@ -22,19 +22,6 @@ import org.apache.iotdb.common.rpc.thrift.TConsensusGroupId; import org.apache.iotdb.common.rpc.thrift.TConsensusGroupType; import org.apache.iotdb.commons.consensus.iotv2.consistency.RepairProgressTable; -import org.apache.iotdb.commons.consensus.iotv2.consistency.ibf.CompositeKeyCodec; -import org.apache.iotdb.commons.consensus.iotv2.consistency.ibf.DataPointLocator; -import org.apache.iotdb.commons.consensus.iotv2.consistency.ibf.DiffEntry; -import org.apache.iotdb.commons.consensus.iotv2.consistency.ibf.RowRefIndex; -import org.apache.iotdb.commons.consensus.iotv2.consistency.merkle.MerkleEntry; -import org.apache.iotdb.commons.consensus.iotv2.consistency.merkle.MerkleFileContent; -import org.apache.iotdb.commons.consensus.iotv2.consistency.repair.ModEntrySummary; -import org.apache.iotdb.commons.consensus.iotv2.consistency.repair.RepairAction; -import org.apache.iotdb.commons.consensus.iotv2.consistency.repair.RepairConflictResolver; -import org.apache.iotdb.commons.consensus.iotv2.consistency.repair.RepairPlan; -import org.apache.iotdb.commons.consensus.iotv2.consistency.repair.RepairRecord; -import org.apache.iotdb.commons.consensus.iotv2.consistency.repair.RepairSession; -import org.apache.iotdb.commons.consensus.iotv2.consistency.repair.RepairStrategy; import org.apache.iotdb.confignode.procedure.Procedure; import org.apache.iotdb.confignode.procedure.store.ProcedureFactory; @@ -47,13 +34,10 @@ import java.util.ArrayList; import java.util.Arrays; import java.util.Collections; -import java.util.HashMap; import java.util.LinkedHashMap; -import java.util.LinkedHashSet; import java.util.List; import java.util.Map; -import java.util.Objects; -import java.util.Set; +import java.util.function.BooleanSupplier; public class RepairRegionProcedureTest { @@ -61,8 +45,7 @@ public class RepairRegionProcedureTest { public void serDeTest() throws Exception { TConsensusGroupId groupId = new TConsensusGroupId(TConsensusGroupType.DataRegion, 10); RepairRegionProcedure procedure = - new RepairRegionProcedure( - groupId, new TestExecutionContext(Collections.emptyMap(), new SimulatedReplicaState())); + new RepairRegionProcedure(groupId, new TestExecutionContext(Collections.emptyMap())); try (PublicBAOS byteArrayOutputStream = new PublicBAOS(); DataOutputStream outputStream = new DataOutputStream(byteArrayOutputStream)) { procedure.serialize(outputStream); @@ -75,327 +58,170 @@ public void serDeTest() throws Exception { } @Test - public void executeRepairFlowTest() throws Exception { + public void executeRepairOperationFlowTest() throws Exception { TConsensusGroupId groupId = new TConsensusGroupId(TConsensusGroupType.DataRegion, 1); - RowRefIndex rowRefIndex = - new RowRefIndex.Builder() - .addDevice("root.sg.d1", Arrays.asList("s1", "s2")) - .setTimeBucketStart(0L) - .build(); - long compositeKey = CompositeKeyCodec.encode(0, 0, 100L, 0L, 1L); - DiffEntry diffEntry = new DiffEntry(compositeKey, 42L, DiffEntry.DiffType.LEADER_HAS); - DataPointLocator largeLocator = new DataPointLocator("root.sg.d1", "s1", 100L); - DataPointLocator smallLocator = new DataPointLocator("root.sg.d1", "s2", 200L); - - MerkleFileContent smallFile = - new MerkleFileContent( - 1L, - 1L, - Collections.singletonList(new MerkleEntry("root.sg.d1", "s2", 0L, 3_600_000L, 10, 1L)), - "small.tsfile"); - MerkleFileContent largeFile = - new MerkleFileContent( - 2L, - 2L, - Collections.singletonList( - new MerkleEntry("root.sg.d1", "s1", 0L, 3_600_000L, 1000, 2L)), - "large.tsfile"); - - Map tsFileSizes = new HashMap<>(); - tsFileSizes.put("small.tsfile", 8L * 1024 * 1024); - tsFileSizes.put("large.tsfile", 128L * 1024 * 1024); - - Map totalPointCounts = new HashMap<>(); - totalPointCounts.put("small.tsfile", 10); - totalPointCounts.put("large.tsfile", 1000); - - SimulatedReplicaState replicaState = new SimulatedReplicaState(); - replicaState.addLeaderPoint("small.tsfile", smallLocator, 1L, "small-value"); - replicaState.addLeaderPoint("large.tsfile", largeLocator, 10L, "value"); - replicaState.addFollowerTsFile("large.tsfile"); - - TestPartitionRepairContext partitionContext = + String repairEpoch = "leader:1:1000:2000:2000"; + List operationIds = Arrays.asList("live@leaf:1:0", "tombstone@leaf:1:0"); + + TestExecutionContext executionContext = new TestExecutionContext(new LinkedHashMap<>()); + executionContext.lastPersistedTable.markMismatch( + 0L, + 100L, + 1000L, + 2000L, + 2000L, + RepairProgressTable.SnapshotState.READY, + "LIVE@leaf:1:0,TOMBSTONE@leaf:1:0", + 2, + repairEpoch); + executionContext.partitionContexts.put( + 0L, new TestPartitionRepairContext( 0L, false, - Arrays.asList(smallFile, largeFile), - Arrays.asList(smallFile, largeFile), - rowRefIndex, - Collections.singletonList(diffEntry), - true, - 1L, - Collections.emptyList(), - tsFileSizes, - totalPointCounts, - Collections.emptyList(), - Collections.emptyList(), - (repairPlans, rootHashMatched) -> { - RepairPlan smallPlan = repairPlans.get("small.tsfile"); - RepairPlan largePlan = repairPlans.get("large.tsfile"); - return !rootHashMatched - && smallPlan != null - && largePlan != null - && smallPlan.getStrategy() == RepairStrategy.DIRECT_TSFILE_TRANSFER - && largePlan.getStrategy() == RepairStrategy.POINT_STREAMING - && replicaState.followerHasTsFile("small.tsfile") - && replicaState.isConsistentWithLeader(); - }); - - TestExecutionContext executionContext = - new TestExecutionContext(Collections.singletonMap(0L, partitionContext), replicaState); + operationIds, + repairEpoch, + () -> executionContext.executedOperationIds.equals(operationIds))); + ExposedRepairRegionProcedure procedure = new ExposedRepairRegionProcedure(groupId, executionContext); - executeProcedureToCompletion(procedure, 32); - Assert.assertEquals( - Collections.singletonList("small.tsfile"), executionContext.transferredTsFiles); - Assert.assertEquals(1, executionContext.generatedRecords.size()); - Assert.assertEquals(1, executionContext.appliedInsertRecords.size()); - Assert.assertTrue(executionContext.appliedDeleteRecords.isEmpty()); - Assert.assertEquals( - Arrays.asList("append:INSERT", "commit", "delete"), executionContext.repairJournalEvents); - Assert.assertEquals(1, executionContext.committedPartitions.size()); - Assert.assertEquals(Long.valueOf(0L), executionContext.committedPartitions.get(0)); - Assert.assertEquals(1000L, executionContext.advancedWatermark); + executeProcedureToCompletion(procedure, 16); + + Assert.assertEquals(operationIds, executionContext.executedOperationIds); + Assert.assertEquals(Collections.singletonList(0L), executionContext.committedPartitions); Assert.assertTrue(executionContext.closed); Assert.assertFalse(executionContext.rolledBack); - Assert.assertEquals(1000L, procedure.getGlobalRepairedWatermark()); - Assert.assertTrue(replicaState.isConsistentWithLeader()); + + RepairProgressTable.PartitionProgress progress = + executionContext.lastPersistedTable.getPartition(0L); + Assert.assertNotNull(progress); + Assert.assertEquals(RepairProgressTable.CheckState.VERIFIED, progress.getCheckState()); + Assert.assertEquals(RepairProgressTable.RepairState.SUCCEEDED, progress.getRepairState()); + Assert.assertEquals(repairEpoch, progress.getRepairEpoch()); + Assert.assertNull(progress.getLastErrorCode()); } @Test - public void executeRepairFlowDeletesFollowerOnlyDataWhenLeaderDeletionWinsTest() - throws Exception { + public void executeRepairOperationFailureRollsBackTest() throws Exception { TConsensusGroupId groupId = new TConsensusGroupId(TConsensusGroupType.DataRegion, 2); - RowRefIndex rowRefIndex = - new RowRefIndex.Builder() - .addDevice("root.sg.d1", Collections.singletonList("s1")) - .setTimeBucketStart(0L) - .build(); - long extraCompositeKey = CompositeKeyCodec.encode(0, 0, 100L, 0L, 1L); - DiffEntry extraFollowerPoint = - new DiffEntry(extraCompositeKey, 84L, DiffEntry.DiffType.FOLLOWER_HAS); - DataPointLocator stableLocator = new DataPointLocator("root.sg.d1", "s1", 200L); - DataPointLocator extraLocator = new DataPointLocator("root.sg.d1", "s1", 100L); - - MerkleFileContent largeFile = - new MerkleFileContent( - 2L, - 2L, - Collections.singletonList( - new MerkleEntry("root.sg.d1", "s1", 0L, 3_600_000L, 1000, 2L)), - "large.tsfile"); - - Map tsFileSizes = Collections.singletonMap("large.tsfile", 128L * 1024 * 1024); - Map totalPointCounts = Collections.singletonMap("large.tsfile", 1000); - - SimulatedReplicaState replicaState = new SimulatedReplicaState(); - replicaState.addLeaderPoint("large.tsfile", stableLocator, 20L, "stable-value"); - replicaState.addFollowerPoint("large.tsfile", stableLocator, 20L, "stable-value"); - replicaState.addFollowerPoint("large.tsfile", extraLocator, 5L, "stale-value"); - - ModEntrySummary leaderDeletion = new ModEntrySummary("root.sg.d1", "s1", 0L, 150L, 10L); - TestPartitionRepairContext partitionContext = + String repairEpoch = "leader:2:1000:3000:3000"; + + TestExecutionContext executionContext = new TestExecutionContext(new LinkedHashMap<>()); + executionContext.lastPersistedTable.markMismatch( + 0L, + 100L, + 1000L, + 3000L, + 3000L, + RepairProgressTable.SnapshotState.READY, + "LIVE@leaf:2:0", + 1, + repairEpoch); + executionContext.operationToFail = "live@leaf:2:0"; + executionContext.partitionContexts.put( + 0L, new TestPartitionRepairContext( - 0L, - false, - Collections.singletonList(largeFile), - Collections.singletonList(largeFile), - rowRefIndex, - Collections.singletonList(extraFollowerPoint), - true, - 1L, - Collections.emptyList(), - tsFileSizes, - totalPointCounts, - Collections.singletonList(leaderDeletion), - Collections.emptyList(), - (repairPlans, rootHashMatched) -> { - RepairPlan largePlan = repairPlans.get("large.tsfile"); - return !rootHashMatched - && largePlan != null - && largePlan.getStrategy() == RepairStrategy.POINT_STREAMING - && replicaState.isConsistentWithLeader(); - }); - - TestExecutionContext executionContext = - new TestExecutionContext(Collections.singletonMap(0L, partitionContext), replicaState); + 0L, false, Collections.singletonList("live@leaf:2:0"), repairEpoch, () -> false)); + ExposedRepairRegionProcedure procedure = new ExposedRepairRegionProcedure(groupId, executionContext); - executeProcedureToCompletion(procedure, 32); - Assert.assertTrue(executionContext.transferredTsFiles.isEmpty()); - Assert.assertEquals(1, executionContext.generatedRecords.size()); - Assert.assertTrue(executionContext.appliedInsertRecords.isEmpty()); - Assert.assertEquals(1, executionContext.appliedDeleteRecords.size()); + executeProcedureToCompletion(procedure, 16); + Assert.assertEquals( - Arrays.asList("append:DELETE", "commit", "delete"), executionContext.repairJournalEvents); - Assert.assertTrue(replicaState.isConsistentWithLeader()); - Assert.assertFalse(replicaState.hasFollowerPoint(extraLocator)); - Assert.assertFalse(executionContext.rolledBack); + Collections.singletonList("live@leaf:2:0"), executionContext.executedOperationIds); + Assert.assertTrue(executionContext.closed); + Assert.assertTrue(executionContext.rolledBack); + Assert.assertTrue(executionContext.committedPartitions.isEmpty()); + + RepairProgressTable.PartitionProgress progress = + executionContext.lastPersistedTable.getPartition(0L); + Assert.assertNotNull(progress); + Assert.assertEquals(RepairProgressTable.CheckState.FAILED, progress.getCheckState()); + Assert.assertEquals(RepairProgressTable.RepairState.FAILED, progress.getRepairState()); + Assert.assertEquals("REPAIR_FAILED", progress.getLastErrorCode()); + Assert.assertEquals(repairEpoch, progress.getRepairEpoch()); } @Test - public void executeRepairFlowDeletesLeaderDataWhenFollowerTombstoneWinsTest() throws Exception { + public void blockingReasonShouldFailFastWithoutRepairOperations() throws Exception { TConsensusGroupId groupId = new TConsensusGroupId(TConsensusGroupType.DataRegion, 3); - RowRefIndex rowRefIndex = - new RowRefIndex.Builder() - .addDevice("root.sg.d1", Collections.singletonList("s1")) - .setTimeBucketStart(0L) - .build(); - long staleCompositeKey = CompositeKeyCodec.encode(0, 0, 100L, 0L, 1L); - DiffEntry staleLeaderPoint = - new DiffEntry(staleCompositeKey, 126L, DiffEntry.DiffType.LEADER_HAS); - DataPointLocator stableLocator = new DataPointLocator("root.sg.d1", "s1", 200L); - DataPointLocator staleLocator = new DataPointLocator("root.sg.d1", "s1", 100L); - - MerkleFileContent largeFile = - new MerkleFileContent( - 2L, - 2L, - Collections.singletonList( - new MerkleEntry("root.sg.d1", "s1", 0L, 3_600_000L, 1000, 2L)), - "large.tsfile"); - - Map tsFileSizes = Collections.singletonMap("large.tsfile", 128L * 1024 * 1024); - Map totalPointCounts = Collections.singletonMap("large.tsfile", 1000); - - SimulatedReplicaState replicaState = new SimulatedReplicaState(); - replicaState.addLeaderPoint("large.tsfile", stableLocator, 20L, "stable-value"); - replicaState.addFollowerPoint("large.tsfile", stableLocator, 20L, "stable-value"); - replicaState.addLeaderPoint("large.tsfile", staleLocator, 5L, "stale-leader-value"); - - ModEntrySummary followerDeletion = new ModEntrySummary("root.sg.d1", "s1", 0L, 150L, 10L); - TestPartitionRepairContext partitionContext = + String repairEpoch = "leader:3:1000:4000:4000"; + String blockingReason = + "Partition 0 contains follower-only tombstone mismatches that cannot be rolled back safely yet"; + + TestExecutionContext executionContext = new TestExecutionContext(new LinkedHashMap<>()); + executionContext.lastPersistedTable.markMismatch( + 0L, + 100L, + 1000L, + 4000L, + 4000L, + RepairProgressTable.SnapshotState.READY, + "TOMBSTONE@leaf:3:0", + 1, + repairEpoch); + executionContext.partitionContexts.put( + 0L, new TestPartitionRepairContext( - 0L, - false, - Collections.singletonList(largeFile), - Collections.singletonList(largeFile), - rowRefIndex, - Collections.singletonList(staleLeaderPoint), - true, - 1L, - Collections.emptyList(), - tsFileSizes, - totalPointCounts, - Collections.emptyList(), - Collections.singletonList(followerDeletion), - (repairPlans, rootHashMatched) -> { - RepairPlan largePlan = repairPlans.get("large.tsfile"); - return !rootHashMatched - && largePlan != null - && largePlan.getStrategy() == RepairStrategy.POINT_STREAMING - && replicaState.isConsistentWithLeader(); - }); - - TestExecutionContext executionContext = - new TestExecutionContext(Collections.singletonMap(0L, partitionContext), replicaState); + 0L, false, Collections.emptyList(), repairEpoch, blockingReason, () -> false)); + ExposedRepairRegionProcedure procedure = new ExposedRepairRegionProcedure(groupId, executionContext); - executeProcedureToCompletion(procedure, 32); - Assert.assertTrue(executionContext.transferredTsFiles.isEmpty()); - Assert.assertEquals(1, executionContext.generatedRecords.size()); - Assert.assertTrue(executionContext.appliedInsertRecords.isEmpty()); - Assert.assertEquals(1, executionContext.appliedDeleteRecords.size()); - Assert.assertEquals( - Arrays.asList("append:DELETE", "commit", "delete"), executionContext.repairJournalEvents); - Assert.assertTrue(replicaState.isConsistentWithLeader()); - Assert.assertFalse(replicaState.hasLeaderPoint(staleLocator)); - Assert.assertFalse(executionContext.rolledBack); + executeProcedureToCompletion(procedure, 16); + + Assert.assertTrue(executionContext.executedOperationIds.isEmpty()); + Assert.assertTrue(executionContext.closed); + Assert.assertTrue(executionContext.rolledBack); + + RepairProgressTable.PartitionProgress progress = + executionContext.lastPersistedTable.getPartition(0L); + Assert.assertNotNull(progress); + Assert.assertEquals(RepairProgressTable.CheckState.FAILED, progress.getCheckState()); + Assert.assertEquals(RepairProgressTable.RepairState.FAILED, progress.getRepairState()); + Assert.assertEquals("REPAIR_FAILED", progress.getLastErrorCode()); + Assert.assertTrue(progress.getLastErrorMessage().contains(blockingReason)); } - @Test - public void executeRepairFlowStreamsFollowerDataBackToLeaderTest() throws Exception { - TConsensusGroupId groupId = new TConsensusGroupId(TConsensusGroupType.DataRegion, 4); - RowRefIndex rowRefIndex = - new RowRefIndex.Builder() - .addDevice("root.sg.d1", Collections.singletonList("s1")) - .setTimeBucketStart(0L) - .build(); - long missingCompositeKey = CompositeKeyCodec.encode(0, 0, 100L, 0L, 1L); - DiffEntry followerOwnedPoint = - new DiffEntry(missingCompositeKey, 168L, DiffEntry.DiffType.FOLLOWER_HAS); - DataPointLocator stableLocator = new DataPointLocator("root.sg.d1", "s1", 200L); - DataPointLocator followerLocator = new DataPointLocator("root.sg.d1", "s1", 100L); - - MerkleFileContent largeFile = - new MerkleFileContent( - 2L, - 2L, - Collections.singletonList( - new MerkleEntry("root.sg.d1", "s1", 0L, 3_600_000L, 1000, 2L)), - "large.tsfile"); - - Map tsFileSizes = Collections.singletonMap("large.tsfile", 128L * 1024 * 1024); - Map totalPointCounts = Collections.singletonMap("large.tsfile", 1000); - - SimulatedReplicaState replicaState = new SimulatedReplicaState(); - replicaState.addLeaderPoint("large.tsfile", stableLocator, 20L, "stable-value"); - replicaState.addFollowerPoint("large.tsfile", stableLocator, 20L, "stable-value"); - replicaState.addFollowerPoint("large.tsfile", followerLocator, 25L, "follower-owned-value"); - - TestPartitionRepairContext partitionContext = - new TestPartitionRepairContext( - 0L, - false, - Collections.singletonList(largeFile), - Collections.singletonList(largeFile), - rowRefIndex, - Collections.singletonList(followerOwnedPoint), - true, - 1L, - Collections.emptyList(), - tsFileSizes, - totalPointCounts, - Collections.emptyList(), - Collections.emptyList(), - (repairPlans, rootHashMatched) -> { - RepairPlan largePlan = repairPlans.get("large.tsfile"); - return !rootHashMatched - && largePlan != null - && largePlan.getStrategy() == RepairStrategy.POINT_STREAMING - && replicaState.isConsistentWithLeader(); - }); - - TestExecutionContext executionContext = - new TestExecutionContext(Collections.singletonMap(0L, partitionContext), replicaState); - ExposedRepairRegionProcedure procedure = - new ExposedRepairRegionProcedure(groupId, executionContext); + private static void executeProcedureToCompletion( + ExposedRepairRegionProcedure procedure, int maxSteps) throws Exception { + int steps = 0; + Procedure[] next; + do { + next = procedure.executeOnce(); + steps++; + } while (next != null && steps < maxSteps); + Assert.assertTrue("procedure should finish within " + maxSteps + " steps", steps < maxSteps); + } - executeProcedureToCompletion(procedure, 32); - Assert.assertTrue(executionContext.transferredTsFiles.isEmpty()); - Assert.assertEquals(1, executionContext.generatedRecords.size()); - Assert.assertEquals(1, executionContext.appliedInsertRecords.size()); - Assert.assertTrue(executionContext.appliedDeleteRecords.isEmpty()); - Assert.assertEquals( - Arrays.asList("append:INSERT", "commit", "delete"), executionContext.repairJournalEvents); - Assert.assertTrue(replicaState.isConsistentWithLeader()); - Assert.assertTrue(replicaState.hasLeaderPoint(followerLocator)); - Assert.assertFalse(executionContext.rolledBack); + private static final class ExposedRepairRegionProcedure extends RepairRegionProcedure { + + private ExposedRepairRegionProcedure( + TConsensusGroupId consensusGroupId, RepairExecutionContext executionContext) { + super(consensusGroupId, executionContext); + } + + private Procedure[] executeOnce() throws InterruptedException { + return doExecute(null); + } } - private static class TestExecutionContext + private static final class TestExecutionContext implements RepairRegionProcedure.RepairExecutionContext { private final Map partitionContexts; - private final SimulatedReplicaState replicaState; - private final List transferredTsFiles = new ArrayList<>(); - private final List generatedRecords = new ArrayList<>(); - private final List appliedInsertRecords = new ArrayList<>(); - private final List appliedDeleteRecords = new ArrayList<>(); - private final List repairJournalEvents = new ArrayList<>(); + private final List executedOperationIds = new ArrayList<>(); private final List committedPartitions = new ArrayList<>(); - private long advancedWatermark = -1L; + private RepairProgressTable lastPersistedTable = new RepairProgressTable("DataRegion-unknown"); + private String operationToFail; private boolean closed; private boolean rolledBack; private TestExecutionContext( - Map partitionContexts, - SimulatedReplicaState replicaState) { + Map partitionContexts) { this.partitionContexts = partitionContexts; - this.replicaState = replicaState; } @Override @@ -410,7 +236,7 @@ public long computeSafeWatermark() { @Override public List collectPendingPartitions( - long globalRepairedWatermark, long safeWatermark, RepairProgressTable repairProgressTable) { + long safeWatermark, RepairProgressTable repairProgressTable) { return new ArrayList<>(partitionContexts.keySet()); } @@ -420,106 +246,31 @@ public RepairRegionProcedure.PartitionRepairContext getPartitionContext(long par } @Override - public RepairRecord buildRepairRecord( - RepairRegionProcedure.PartitionRepairContext partitionContext, - DiffEntry diffEntry, - RowRefIndex rowRefIndex, - RepairConflictResolver conflictResolver) { - DataPointLocator locator = rowRefIndex.resolve(diffEntry.getCompositeKey()); - SimulatedPoint leaderPoint = replicaState.getLeaderPoint(locator); - SimulatedPoint followerPoint = replicaState.getFollowerPoint(locator); - RepairRecord record = null; - if (diffEntry.getType() == DiffEntry.DiffType.LEADER_HAS) { - if (leaderPoint == null) { - return null; - } - RepairAction action = - conflictResolver.resolveLeaderHas(locator, leaderPoint.getProgressIndex()); - if (action == RepairAction.SEND_TO_FOLLOWER) { - record = - RepairRecord.insertToFollower( - locator, - leaderPoint.getProgressIndex(), - leaderPoint.getValue(), - locator.getTimestamp()); - } else if (action == RepairAction.DELETE_ON_LEADER) { - record = - RepairRecord.deleteOnLeader( - locator, leaderPoint.getProgressIndex(), locator.getTimestamp()); - } - } else if (diffEntry.getType() == DiffEntry.DiffType.FOLLOWER_HAS) { - if (followerPoint == null) { - return null; - } - RepairAction action = - conflictResolver.resolveFollowerHas(locator, followerPoint.getProgressIndex()); - if (action == RepairAction.DELETE_ON_FOLLOWER) { - record = - RepairRecord.deleteOnFollower( - locator, followerPoint.getProgressIndex(), locator.getTimestamp()); - } else if (action == RepairAction.SEND_TO_LEADER) { - record = - RepairRecord.insertToLeader( - locator, - followerPoint.getProgressIndex(), - followerPoint.getValue(), - locator.getTimestamp()); - } - } - if (record != null) { - generatedRecords.add(record); + public void executeRepairOperation(String operationId) { + executedOperationIds.add(operationId); + if (operationId.equals(operationToFail)) { + throw new IllegalStateException("Injected repair failure for " + operationId); } - return record; } @Override - public void transferTsFile(String tsFilePath) { - transferredTsFiles.add(tsFilePath); - replicaState.transferTsFile(tsFilePath); + public RepairProgressTable loadRepairProgressTable(String consensusGroupKey) { + return lastPersistedTable.copy(); } @Override - public RepairSession createRepairSession(long partitionId) { - return new RepairSession( - partitionId, - (sessionId, ignoredPartitionId, inserts, deletes) -> { - appliedInsertRecords.addAll(inserts); - appliedDeleteRecords.addAll(deletes); - replicaState.applyInserts(inserts); - replicaState.applyDeletes(deletes); - }, - new RepairSession.RepairSessionJournal() { - @Override - public void append(String sessionId, RepairRecord record) { - repairJournalEvents.add("append:" + record.getType()); - } - - @Override - public void markCommitted(String sessionId) { - repairJournalEvents.add("commit"); - } - - @Override - public void delete(String sessionId) { - repairJournalEvents.add("delete"); - } - }); + public void persistRepairProgressTable(RepairProgressTable repairProgressTable) { + this.lastPersistedTable = repairProgressTable.copy(); } @Override public void onPartitionCommitted( - long partitionId, long repairedTo, RepairProgressTable repairProgressTable) { + long partitionId, long committedAt, RepairProgressTable repairProgressTable) { committedPartitions.add(partitionId); } @Override - public void onWatermarkAdvanced(long globalWatermark, RepairProgressTable repairProgressTable) { - this.advancedWatermark = globalWatermark; - } - - @Override - public void rollbackPartition( - long partitionId, RepairSession repairSession, RepairProgressTable repairProgressTable) { + public void rollbackPartition(long partitionId, RepairProgressTable repairProgressTable) { rolledBack = true; } @@ -529,67 +280,40 @@ public void close() { } } - private static class ExposedRepairRegionProcedure extends RepairRegionProcedure { - - private ExposedRepairRegionProcedure( - TConsensusGroupId consensusGroupId, RepairExecutionContext executionContext) { - super(consensusGroupId, executionContext); - } - - private Procedure[] executeOnce() throws InterruptedException { - return doExecute(null); - } - } - - private static class TestPartitionRepairContext + private static final class TestPartitionRepairContext implements RepairRegionProcedure.PartitionRepairContext { private final long partitionId; private final boolean rootHashMatched; - private final List leaderMerkleFiles; - private final List mismatchedLeaderMerkleFiles; - private final RowRefIndex rowRefIndex; - private final List decodedDiffs; - private final boolean diffDecodeSuccessful; - private final long estimatedDiffCount; - private final List fallbackTsFiles; - private final Map tsFileSizes; - private final Map totalPointCounts; - private final List leaderDeletions; - private final List followerDeletions; - private final RepairVerifier verifier; + private final List repairOperationIds; + private final String repairEpoch; + private final String blockingReason; + private final BooleanSupplier verifier; private TestPartitionRepairContext( long partitionId, boolean rootHashMatched, - List leaderMerkleFiles, - List mismatchedLeaderMerkleFiles, - RowRefIndex rowRefIndex, - List decodedDiffs, - boolean diffDecodeSuccessful, - long estimatedDiffCount, - List fallbackTsFiles, - Map tsFileSizes, - Map totalPointCounts, - List leaderDeletions, - List followerDeletions, - RepairVerifier verifier) { + List repairOperationIds, + String repairEpoch, + String blockingReason, + BooleanSupplier verifier) { this.partitionId = partitionId; this.rootHashMatched = rootHashMatched; - this.leaderMerkleFiles = leaderMerkleFiles; - this.mismatchedLeaderMerkleFiles = mismatchedLeaderMerkleFiles; - this.rowRefIndex = rowRefIndex; - this.decodedDiffs = decodedDiffs; - this.diffDecodeSuccessful = diffDecodeSuccessful; - this.estimatedDiffCount = estimatedDiffCount; - this.fallbackTsFiles = fallbackTsFiles; - this.tsFileSizes = tsFileSizes; - this.totalPointCounts = totalPointCounts; - this.leaderDeletions = leaderDeletions; - this.followerDeletions = followerDeletions; + this.repairOperationIds = repairOperationIds; + this.repairEpoch = repairEpoch; + this.blockingReason = blockingReason; this.verifier = verifier; } + private TestPartitionRepairContext( + long partitionId, + boolean rootHashMatched, + List repairOperationIds, + String repairEpoch, + BooleanSupplier verifier) { + this(partitionId, rootHashMatched, repairOperationIds, repairEpoch, null, verifier); + } + @Override public long getPartitionId() { return partitionId; @@ -601,223 +325,23 @@ public boolean isRootHashMatched() { } @Override - public List getLeaderMerkleFiles() { - return leaderMerkleFiles; - } - - @Override - public List getMismatchedLeaderMerkleFiles() { - return mismatchedLeaderMerkleFiles; - } - - @Override - public RowRefIndex getRowRefIndex() { - return rowRefIndex; - } - - @Override - public List decodeDiffs() { - return decodedDiffs; - } - - @Override - public boolean isDiffDecodeSuccessful() { - return diffDecodeSuccessful; + public List getRepairOperationIds() { + return repairOperationIds; } @Override - public long estimateDiffCount() { - return estimatedDiffCount; + public String getRepairEpoch() { + return repairEpoch; } @Override - public List getFallbackTsFiles() { - return fallbackTsFiles; - } - - @Override - public long getTsFileSize(String tsFilePath) { - return tsFileSizes.getOrDefault(tsFilePath, 0L); - } - - @Override - public int getTotalPointCount(String tsFilePath) { - return totalPointCounts.getOrDefault(tsFilePath, 0); - } - - @Override - public List getLeaderDeletions() { - return leaderDeletions; - } - - @Override - public List getFollowerDeletions() { - return followerDeletions; - } - - @Override - public boolean verify(Map repairPlans, boolean rootHashMatched) { - return verifier.verify(repairPlans, rootHashMatched); - } - } - - private static void executeProcedureToCompletion( - ExposedRepairRegionProcedure procedure, int maxSteps) throws Exception { - int steps = 0; - Procedure[] next; - do { - next = procedure.executeOnce(); - steps++; - } while (next != null && steps < maxSteps); - Assert.assertTrue("procedure should finish within " + maxSteps + " steps", steps < maxSteps); - } - - @FunctionalInterface - private interface RepairVerifier { - boolean verify(Map repairPlans, boolean rootHashMatched); - } - - private static class SimulatedReplicaState { - - private final Map> leaderPointsByTsFile = new LinkedHashMap<>(); - private final Map leaderPoints = new LinkedHashMap<>(); - private final Map followerPoints = new LinkedHashMap<>(); - private final Set leaderTsFiles = new LinkedHashSet<>(); - private final Set followerTsFiles = new LinkedHashSet<>(); - - private void addLeaderPoint( - String tsFilePath, DataPointLocator locator, long progressIndex, Object value) { - SimulatedPoint point = new SimulatedPoint(tsFilePath, locator, progressIndex, value); - leaderTsFiles.add(tsFilePath); - leaderPoints.put(locator, point); - leaderPointsByTsFile.computeIfAbsent(tsFilePath, ignored -> new ArrayList<>()).add(point); - } - - private void addFollowerTsFile(String tsFilePath) { - followerTsFiles.add(tsFilePath); - } - - private void addFollowerPoint( - String tsFilePath, DataPointLocator locator, long progressIndex, Object value) { - followerTsFiles.add(tsFilePath); - followerPoints.put(locator, new SimulatedPoint(tsFilePath, locator, progressIndex, value)); - } - - private SimulatedPoint getLeaderPoint(DataPointLocator locator) { - return leaderPoints.get(locator); - } - - private SimulatedPoint getFollowerPoint(DataPointLocator locator) { - return followerPoints.get(locator); - } - - private boolean hasFollowerPoint(DataPointLocator locator) { - return followerPoints.containsKey(locator); - } - - private boolean hasLeaderPoint(DataPointLocator locator) { - return leaderPoints.containsKey(locator); - } - - private boolean followerHasTsFile(String tsFilePath) { - return followerTsFiles.contains(tsFilePath); - } - - private void transferTsFile(String tsFilePath) { - followerTsFiles.add(tsFilePath); - for (SimulatedPoint leaderPoint : - leaderPointsByTsFile.getOrDefault(tsFilePath, Collections.emptyList())) { - followerPoints.put(leaderPoint.getLocator(), leaderPoint); - } - } - - private void applyInserts(List inserts) { - for (RepairRecord record : inserts) { - if (record.getTargetReplica() == RepairRecord.TargetReplica.LEADER) { - SimulatedPoint followerPoint = followerPoints.get(record.getLocator()); - String tsFilePath = - followerPoint == null ? "stream.tsfile" : followerPoint.getTsFilePath(); - leaderTsFiles.add(tsFilePath); - leaderPoints.put( - record.getLocator(), - new SimulatedPoint( - tsFilePath, record.getLocator(), record.getProgressIndex(), record.getValue())); - } else { - SimulatedPoint leaderPoint = leaderPoints.get(record.getLocator()); - String tsFilePath = leaderPoint == null ? "stream.tsfile" : leaderPoint.getTsFilePath(); - followerTsFiles.add(tsFilePath); - followerPoints.put( - record.getLocator(), - new SimulatedPoint( - tsFilePath, record.getLocator(), record.getProgressIndex(), record.getValue())); - } - } - } - - private void applyDeletes(List deletes) { - for (RepairRecord record : deletes) { - if (record.getTargetReplica() == RepairRecord.TargetReplica.LEADER) { - leaderPoints.remove(record.getLocator()); - } else { - followerPoints.remove(record.getLocator()); - } - } - } - - private boolean isConsistentWithLeader() { - return leaderTsFiles.equals(followerTsFiles) && Objects.equals(leaderPoints, followerPoints); - } - } - - private static class SimulatedPoint { - - private final String tsFilePath; - private final DataPointLocator locator; - private final long progressIndex; - private final Object value; - - private SimulatedPoint( - String tsFilePath, DataPointLocator locator, long progressIndex, Object value) { - this.tsFilePath = tsFilePath; - this.locator = locator; - this.progressIndex = progressIndex; - this.value = value; - } - - private String getTsFilePath() { - return tsFilePath; - } - - private DataPointLocator getLocator() { - return locator; - } - - private long getProgressIndex() { - return progressIndex; - } - - private Object getValue() { - return value; - } - - @Override - public boolean equals(Object o) { - if (this == o) { - return true; - } - if (!(o instanceof SimulatedPoint)) { - return false; - } - SimulatedPoint that = (SimulatedPoint) o; - return progressIndex == that.progressIndex - && Objects.equals(tsFilePath, that.tsFilePath) - && Objects.equals(locator, that.locator) - && Objects.equals(value, that.value); + public String getBlockingReason() { + return blockingReason; } @Override - public int hashCode() { - return Objects.hash(tsFilePath, locator, progressIndex, value); + public boolean verify() { + return verifier.getAsBoolean(); } } } diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/sink/protocol/iotconsensusv2/IoTConsensusV2AsyncSink.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/sink/protocol/iotconsensusv2/IoTConsensusV2AsyncSink.java index 3001c40b16fbf..1b41e2cda56ae 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/sink/protocol/iotconsensusv2/IoTConsensusV2AsyncSink.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/sink/protocol/iotconsensusv2/IoTConsensusV2AsyncSink.java @@ -38,7 +38,6 @@ import org.apache.iotdb.consensus.pipe.metric.IoTConsensusV2SyncLagManager; import org.apache.iotdb.db.conf.IoTDBConfig; import org.apache.iotdb.db.conf.IoTDBDescriptor; -import org.apache.iotdb.db.pipe.consensus.ReplicateProgressDataNodeManager; import org.apache.iotdb.db.pipe.consensus.metric.IoTConsensusV2SinkMetrics; import org.apache.iotdb.db.pipe.event.common.PipeInsertionEvent; import org.apache.iotdb.db.pipe.event.common.deletion.PipeDeleteDataNodeEvent; @@ -115,6 +114,10 @@ public class IoTConsensusV2AsyncSink extends IoTDBSink implements ConsensusPipeS private IoTConsensusV2SyncSink retryConnector; private IClientManager asyncTransferClientManager; private IoTConsensusV2AsyncBatchReqBuilder tabletBatchBuilder; + // Track the highest replicate index that actually enters this connector instead of the + // source-side pre-assigned index, otherwise discarded realtime TsFile events can create + // phantom sync lag. + private volatile long leaderReplicateProgress = 0; private volatile long currentReplicateProgress = 0; @Override @@ -196,6 +199,8 @@ private boolean addEvent2Buffer(EnrichedEvent event) { iotConsensusV2SinkMetrics.recordConnectorEnqueueTimer(duration); // add reference if (result) { + leaderReplicateProgress = + Math.max(leaderReplicateProgress, event.getReplicateIndexForIoTV2()); event.increaseReferenceCount(IoTConsensusV2AsyncSink.class.getName()); } // if connector is closed when executing this method, need to clear this event's reference @@ -717,7 +722,7 @@ public int getRetryBufferSize() { @Override public long getLeaderReplicateProgress() { - return ReplicateProgressDataNodeManager.getReplicateIndexForIoTV2(consensusPipeName); + return leaderReplicateProgress; } @Override diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/protocol/client/ConfigNodeClient.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/protocol/client/ConfigNodeClient.java index 6ac2da44c7ef3..70a09529063e4 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/protocol/client/ConfigNodeClient.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/protocol/client/ConfigNodeClient.java @@ -165,6 +165,7 @@ import org.apache.iotdb.confignode.rpc.thrift.TShowPipeResp; import org.apache.iotdb.confignode.rpc.thrift.TShowRegionReq; import org.apache.iotdb.confignode.rpc.thrift.TShowRegionResp; +import org.apache.iotdb.confignode.rpc.thrift.TShowRepairProgressResp; import org.apache.iotdb.confignode.rpc.thrift.TShowSubscriptionReq; import org.apache.iotdb.confignode.rpc.thrift.TShowSubscriptionResp; import org.apache.iotdb.confignode.rpc.thrift.TShowTTLResp; @@ -917,6 +918,12 @@ public TShowConfigNodes4InformationSchemaResp showConfigNodes4InformationSchema( resp -> !updateConfigNodeLeader(resp.status)); } + @Override + public TShowRepairProgressResp showRepairProgress() throws TException { + return executeRemoteCallWithRetry( + () -> client.showRepairProgress(), resp -> !updateConfigNodeLeader(resp.status)); + } + @Override public TShowDatabaseResp showDatabase(TGetDatabaseReq req) throws TException { return executeRemoteCallWithRetry( diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/protocol/thrift/impl/DataNodeInternalRPCServiceImpl.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/protocol/thrift/impl/DataNodeInternalRPCServiceImpl.java index 7edf9fde0d3d8..6104664c39c28 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/protocol/thrift/impl/DataNodeInternalRPCServiceImpl.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/protocol/thrift/impl/DataNodeInternalRPCServiceImpl.java @@ -217,6 +217,7 @@ import org.apache.iotdb.mpp.rpc.thrift.TAlterEncodingCompressorReq; import org.apache.iotdb.mpp.rpc.thrift.TAlterTimeSeriesReq; import org.apache.iotdb.mpp.rpc.thrift.TAlterViewReq; +import org.apache.iotdb.mpp.rpc.thrift.TApplyLogicalRepairBatchReq; import org.apache.iotdb.mpp.rpc.thrift.TAttributeUpdateReq; import org.apache.iotdb.mpp.rpc.thrift.TAuditLogReq; import org.apache.iotdb.mpp.rpc.thrift.TCancelFragmentInstanceReq; @@ -241,9 +242,9 @@ import org.apache.iotdb.mpp.rpc.thrift.TCreateTriggerInstanceReq; import org.apache.iotdb.mpp.rpc.thrift.TDataNodeHeartbeatReq; import org.apache.iotdb.mpp.rpc.thrift.TDataNodeHeartbeatResp; -import org.apache.iotdb.mpp.rpc.thrift.TDataRegionConsistencySnapshotReq; -import org.apache.iotdb.mpp.rpc.thrift.TDataRegionConsistencySnapshotResp; import org.apache.iotdb.mpp.rpc.thrift.TDeactivateTemplateReq; +import org.apache.iotdb.mpp.rpc.thrift.TDecodeLeafDiffReq; +import org.apache.iotdb.mpp.rpc.thrift.TDecodeLeafDiffResp; import org.apache.iotdb.mpp.rpc.thrift.TDeleteColumnDataReq; import org.apache.iotdb.mpp.rpc.thrift.TDeleteDataForDeleteSchemaReq; import org.apache.iotdb.mpp.rpc.thrift.TDeleteDataOrDevicesForDropTableReq; @@ -254,15 +255,22 @@ import org.apache.iotdb.mpp.rpc.thrift.TDropFunctionInstanceReq; import org.apache.iotdb.mpp.rpc.thrift.TDropPipePluginInstanceReq; import org.apache.iotdb.mpp.rpc.thrift.TDropTriggerInstanceReq; +import org.apache.iotdb.mpp.rpc.thrift.TEstimateLeafDiffReq; +import org.apache.iotdb.mpp.rpc.thrift.TEstimateLeafDiffResp; import org.apache.iotdb.mpp.rpc.thrift.TExecuteCQ; import org.apache.iotdb.mpp.rpc.thrift.TFetchFragmentInstanceInfoReq; import org.apache.iotdb.mpp.rpc.thrift.TFetchFragmentInstanceStatisticsReq; import org.apache.iotdb.mpp.rpc.thrift.TFetchFragmentInstanceStatisticsResp; import org.apache.iotdb.mpp.rpc.thrift.TFetchSchemaBlackListReq; import org.apache.iotdb.mpp.rpc.thrift.TFetchSchemaBlackListResp; +import org.apache.iotdb.mpp.rpc.thrift.TFinishLogicalRepairSessionReq; import org.apache.iotdb.mpp.rpc.thrift.TFireTriggerReq; import org.apache.iotdb.mpp.rpc.thrift.TFireTriggerResp; import org.apache.iotdb.mpp.rpc.thrift.TFragmentInstanceInfoResp; +import org.apache.iotdb.mpp.rpc.thrift.TGetConsistencyEligibilityReq; +import org.apache.iotdb.mpp.rpc.thrift.TGetConsistencyEligibilityResp; +import org.apache.iotdb.mpp.rpc.thrift.TGetSnapshotSubtreeReq; +import org.apache.iotdb.mpp.rpc.thrift.TGetSnapshotSubtreeResp; import org.apache.iotdb.mpp.rpc.thrift.TInactiveTriggerInstanceReq; import org.apache.iotdb.mpp.rpc.thrift.TInvalidateCacheReq; import org.apache.iotdb.mpp.rpc.thrift.TInvalidateColumnCacheReq; @@ -293,7 +301,6 @@ import org.apache.iotdb.mpp.rpc.thrift.TRegionLeaderChangeResp; import org.apache.iotdb.mpp.rpc.thrift.TRegionMigrateResult; import org.apache.iotdb.mpp.rpc.thrift.TRegionRouteReq; -import org.apache.iotdb.mpp.rpc.thrift.TRepairTransferTsFileReq; import org.apache.iotdb.mpp.rpc.thrift.TResetPeerListReq; import org.apache.iotdb.mpp.rpc.thrift.TRollbackSchemaBlackListReq; import org.apache.iotdb.mpp.rpc.thrift.TRollbackSchemaBlackListWithTemplateReq; @@ -305,6 +312,8 @@ import org.apache.iotdb.mpp.rpc.thrift.TSendFragmentInstanceReq; import org.apache.iotdb.mpp.rpc.thrift.TSendFragmentInstanceResp; import org.apache.iotdb.mpp.rpc.thrift.TSendSinglePlanNodeResp; +import org.apache.iotdb.mpp.rpc.thrift.TStreamLogicalRepairReq; +import org.apache.iotdb.mpp.rpc.thrift.TStreamLogicalRepairResp; import org.apache.iotdb.mpp.rpc.thrift.TTableDeviceDeletionWithPatternAndFilterReq; import org.apache.iotdb.mpp.rpc.thrift.TTableDeviceDeletionWithPatternOrModReq; import org.apache.iotdb.mpp.rpc.thrift.TTableDeviceInvalidateCacheReq; @@ -597,14 +606,39 @@ public TLoadResp sendLoadCommand(TLoadCommandReq req) { } @Override - public TDataRegionConsistencySnapshotResp getDataRegionConsistencySnapshot( - TDataRegionConsistencySnapshotReq req) { - return consistencyRepairService.getSnapshot(req); + public TGetConsistencyEligibilityResp getConsistencyEligibility( + TGetConsistencyEligibilityReq req) { + return consistencyRepairService.getConsistencyEligibility(req); } @Override - public TSStatus repairTransferTsFile(TRepairTransferTsFileReq req) { - return consistencyRepairService.repairTransferTsFile(req); + public TGetSnapshotSubtreeResp getSnapshotSubtree(TGetSnapshotSubtreeReq req) { + return consistencyRepairService.getSnapshotSubtree(req); + } + + @Override + public TEstimateLeafDiffResp estimateLeafDiff(TEstimateLeafDiffReq req) { + return consistencyRepairService.estimateLeafDiff(req); + } + + @Override + public TDecodeLeafDiffResp decodeLeafDiff(TDecodeLeafDiffReq req) { + return consistencyRepairService.decodeLeafDiff(req); + } + + @Override + public TStreamLogicalRepairResp streamLogicalRepair(TStreamLogicalRepairReq req) { + return consistencyRepairService.streamLogicalRepair(req); + } + + @Override + public TSStatus applyLogicalRepairBatch(TApplyLogicalRepairBatchReq req) { + return consistencyRepairService.applyLogicalRepairBatch(req); + } + + @Override + public TSStatus finishLogicalRepairSession(TFinishLogicalRepairSessionReq req) { + return consistencyRepairService.finishLogicalRepairSession(req); } @Override diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/execution/operator/source/relational/InformationSchemaContentSupplierFactory.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/execution/operator/source/relational/InformationSchemaContentSupplierFactory.java index 8774b670e7a31..dff7027a9cfcc 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/execution/operator/source/relational/InformationSchemaContentSupplierFactory.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/execution/operator/source/relational/InformationSchemaContentSupplierFactory.java @@ -53,6 +53,7 @@ import org.apache.iotdb.confignode.rpc.thrift.TGetUdfTableReq; import org.apache.iotdb.confignode.rpc.thrift.TNodeVersionInfo; import org.apache.iotdb.confignode.rpc.thrift.TRegionInfo; +import org.apache.iotdb.confignode.rpc.thrift.TRepairProgressInfo; import org.apache.iotdb.confignode.rpc.thrift.TShowClusterResp; import org.apache.iotdb.confignode.rpc.thrift.TShowPipeInfo; import org.apache.iotdb.confignode.rpc.thrift.TShowPipeReq; @@ -153,6 +154,8 @@ public static Iterator getSupplier( return new ColumnSupplier(dataTypes, userEntity); case InformationSchema.REGIONS: return new RegionSupplier(dataTypes, userEntity); + case InformationSchema.REPAIR_PROGRESS: + return new RepairProgressSupplier(dataTypes, userEntity); case InformationSchema.PIPES: return new PipeSupplier(dataTypes, userEntity.getUsername()); case InformationSchema.PIPE_PLUGINS: @@ -561,6 +564,67 @@ public boolean hasNext() { } } + private static class RepairProgressSupplier extends TsBlockSupplier { + private final Iterator iterator; + + private RepairProgressSupplier(final List dataTypes, final UserEntity userEntity) + throws Exception { + super(dataTypes); + accessControl.checkUserGlobalSysPrivilege(userEntity); + try (final ConfigNodeClient client = + ConfigNodeClientManager.getInstance().borrowClient(ConfigNodeInfo.CONFIG_REGION_ID)) { + final org.apache.iotdb.confignode.rpc.thrift.TShowRepairProgressResp resp = + client.showRepairProgress(); + iterator = + resp == null || !resp.isSetRepairProgressInfoList() + ? Collections.emptyList().iterator() + : resp.getRepairProgressInfoListIterator(); + } + } + + @Override + protected void constructLine() { + final TRepairProgressInfo progressInfo = iterator.next(); + columnBuilders[0].writeInt(progressInfo.getRegionId()); + columnBuilders[1].writeLong(progressInfo.getTimePartition()); + columnBuilders[2].writeBinary(BytesUtils.valueOf(progressInfo.getCheckState())); + columnBuilders[3].writeBinary(BytesUtils.valueOf(progressInfo.getRepairState())); + columnBuilders[4].writeLong(progressInfo.getLastCheckedAt()); + columnBuilders[5].writeLong(progressInfo.getLastSafeWatermark()); + columnBuilders[6].writeLong(progressInfo.getPartitionMutationEpoch()); + columnBuilders[7].writeLong(progressInfo.getSnapshotEpoch()); + columnBuilders[8].writeBinary(BytesUtils.valueOf(progressInfo.getSnapshotState())); + columnBuilders[9].writeLong(progressInfo.getLastMismatchAt()); + if (progressInfo.isSetMismatchScopeRef()) { + columnBuilders[10].writeBinary(BytesUtils.valueOf(progressInfo.getMismatchScopeRef())); + } else { + columnBuilders[10].appendNull(); + } + columnBuilders[11].writeInt(progressInfo.getMismatchLeafCount()); + if (progressInfo.isSetRepairEpoch()) { + columnBuilders[12].writeBinary(BytesUtils.valueOf(progressInfo.getRepairEpoch())); + } else { + columnBuilders[12].appendNull(); + } + if (progressInfo.isSetLastErrorCode()) { + columnBuilders[13].writeBinary(BytesUtils.valueOf(progressInfo.getLastErrorCode())); + } else { + columnBuilders[13].appendNull(); + } + if (progressInfo.isSetLastErrorMessage()) { + columnBuilders[14].writeBinary(BytesUtils.valueOf(progressInfo.getLastErrorMessage())); + } else { + columnBuilders[14].appendNull(); + } + resultBuilder.declarePosition(); + } + + @Override + public boolean hasNext() { + return iterator.hasNext(); + } + } + private static class PipeSupplier extends TsBlockSupplier { private final Iterator iterator; diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/planner/optimizations/DataNodeLocationSupplierFactory.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/planner/optimizations/DataNodeLocationSupplierFactory.java index 8e73872f5ba8f..93e3be62357d4 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/planner/optimizations/DataNodeLocationSupplierFactory.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/planner/optimizations/DataNodeLocationSupplierFactory.java @@ -93,6 +93,7 @@ public List getDataNodeLocations(final String tableName) { case InformationSchema.TABLES: case InformationSchema.COLUMNS: case InformationSchema.REGIONS: + case InformationSchema.REPAIR_PROGRESS: case InformationSchema.PIPES: case InformationSchema.PIPE_PLUGINS: case InformationSchema.TOPICS: diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/DataRegion.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/DataRegion.java index e00a17ad854cb..3aac7d0bee388 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/DataRegion.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/DataRegion.java @@ -109,6 +109,7 @@ import org.apache.iotdb.db.storageengine.dataregion.compaction.schedule.CompactionScheduleTaskManager; import org.apache.iotdb.db.storageengine.dataregion.compaction.schedule.CompactionScheduler; import org.apache.iotdb.db.storageengine.dataregion.compaction.schedule.CompactionTaskManager; +import org.apache.iotdb.db.storageengine.dataregion.consistency.DataRegionConsistencyManager; import org.apache.iotdb.db.storageengine.dataregion.flush.CloseFileListener; import org.apache.iotdb.db.storageengine.dataregion.flush.FlushListener; import org.apache.iotdb.db.storageengine.dataregion.flush.FlushStatus; @@ -1232,6 +1233,7 @@ public void insert(InsertRowNode insertRowNode) throws WriteProcessException { PERFORMANCE_OVERVIEW_METRICS.recordScheduleUpdateLastCacheCost( System.nanoTime() - startTime); } + markConsistencyDirtyForInsert(insertRowNode.getTime()); } finally { writeUnlock(); } @@ -1389,6 +1391,7 @@ public void insertTablet(InsertTabletNode insertTabletNode) // infoForMetrics[4]: InsertedPointsNumber boolean noFailure = executeInsertTablet(insertTabletNode, results, infoForMetrics); updateTsFileProcessorMetric(insertTabletNode, infoForMetrics); + markConsistencyDirtyForTablet(insertTabletNode.getTimes()); if (!noFailure) { throw new BatchProcessException(results); @@ -2873,6 +2876,10 @@ public void deleteByDevice(final MeasurementPath pattern, final DeleteDataNode n if (deletionResource != null && deletionResource.waitForResult() == Status.FAILURE) { throw deletionResource.getCause(); } + DataRegionConsistencyManager.getInstance() + .onDeletion(dataRegionId.convertToTConsensusGroupId(), sealedTsFileResource); + DataRegionConsistencyManager.getInstance() + .onDeletion(dataRegionId.convertToTConsensusGroupId(), startTime, endTime); writeUnlock(); hasReleasedLock = true; @@ -2977,6 +2984,17 @@ public void deleteByTable(RelationalDeleteDataNode node) throws IOException { if (deletionResource != null && deletionResource.waitForResult() == Status.FAILURE) { throw deletionResource.getCause(); } + for (TableDeletionEntry modEntry : modEntries) { + DataRegionConsistencyManager.getInstance() + .onDeletion( + dataRegionId.convertToTConsensusGroupId(), + modEntry.getStartTime(), + modEntry.getEndTime()); + } + for (List sealedTsFileResources : sealedTsFileResourceLists) { + DataRegionConsistencyManager.getInstance() + .onDeletion(dataRegionId.convertToTConsensusGroupId(), sealedTsFileResources); + } writeUnlock(); hasReleasedLock = true; @@ -3035,6 +3053,10 @@ public void deleteDataDirectly(MeasurementPath pathToDelete, DeleteDataNode node if (deletionResource != null && deletionResource.waitForResult() == Status.FAILURE) { throw deletionResource.getCause(); } + DataRegionConsistencyManager.getInstance() + .onDeletion(dataRegionId.convertToTConsensusGroupId(), sealedTsFileResource); + DataRegionConsistencyManager.getInstance() + .onDeletion(dataRegionId.convertToTConsensusGroupId(), startTime, endTime); writeUnlock(); releasedLock = true; deleteDataDirectlyInFile(sealedTsFileResource, deletion); @@ -4622,6 +4644,7 @@ public void insert(InsertRowsNode insertRowsNode) if (!insertRowsNode.getResults().isEmpty()) { throw new BatchProcessException("Partial failed inserting rows"); } + markConsistencyDirtyForRows(timePartitionIds); } finally { writeUnlock(); } @@ -4686,6 +4709,7 @@ public void insertTablets(InsertMultiTabletsNode insertMultiTabletsNode) } insertMultiTabletsNode.getResults().put(i, firstStatus); } + markConsistencyDirtyForTablet(insertTabletNode.getTimes()); } updateTsFileProcessorMetric(insertMultiTabletsNode, infoForMetrics); @@ -4981,4 +5005,38 @@ private long getTTL(InsertNode insertNode) { return DataNodeTTLCache.getInstance().getTTLForTable(databaseName, insertNode.getTableName()); } } + + private void markConsistencyDirtyForInsert(long time) { + DataRegionConsistencyManager.getInstance() + .onPartitionMutation( + dataRegionId.convertToTConsensusGroupId(), TimePartitionUtils.getTimePartitionId(time)); + } + + private void markConsistencyDirtyForTablet(long[] times) { + if (times == null || times.length == 0) { + return; + } + Set partitions = new HashSet<>(); + for (long time : times) { + partitions.add(TimePartitionUtils.getTimePartitionId(time)); + } + for (Long partitionId : partitions) { + DataRegionConsistencyManager.getInstance() + .onPartitionMutation(dataRegionId.convertToTConsensusGroupId(), partitionId); + } + } + + private void markConsistencyDirtyForRows(long[] timePartitionIds) { + if (timePartitionIds == null || timePartitionIds.length == 0) { + return; + } + Set partitions = new HashSet<>(); + for (long partitionId : timePartitionIds) { + partitions.add(partitionId); + } + for (Long partitionId : partitions) { + DataRegionConsistencyManager.getInstance() + .onPartitionMutation(dataRegionId.convertToTConsensusGroupId(), partitionId); + } + } } diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/consistency/DataRegionConsistencyManager.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/consistency/DataRegionConsistencyManager.java new file mode 100644 index 0000000000000..13e90ecfb885c --- /dev/null +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/consistency/DataRegionConsistencyManager.java @@ -0,0 +1,1781 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.iotdb.db.storageengine.dataregion.consistency; + +import org.apache.iotdb.common.rpc.thrift.TConsensusGroupId; +import org.apache.iotdb.commons.consensus.iotv2.consistency.DualDigest; +import org.apache.iotdb.commons.consensus.iotv2.consistency.RepairProgressTable; +import org.apache.iotdb.commons.path.AlignedFullPath; +import org.apache.iotdb.commons.path.IFullPath; +import org.apache.iotdb.commons.path.MeasurementPath; +import org.apache.iotdb.commons.path.NonAlignedFullPath; +import org.apache.iotdb.commons.path.PartialPath; +import org.apache.iotdb.db.queryengine.execution.fragment.FragmentInstanceContext; +import org.apache.iotdb.db.queryengine.plan.planner.plan.node.PlanNodeId; +import org.apache.iotdb.db.queryengine.plan.planner.plan.node.write.DeleteDataNode; +import org.apache.iotdb.db.queryengine.plan.planner.plan.node.write.InsertRowNode; +import org.apache.iotdb.db.queryengine.plan.planner.plan.node.write.InsertRowsNode; +import org.apache.iotdb.db.storageengine.dataregion.DataRegion; +import org.apache.iotdb.db.storageengine.dataregion.compaction.execute.performer.impl.ReadPointCompactionPerformer; +import org.apache.iotdb.db.storageengine.dataregion.compaction.execute.utils.MultiTsFileDeviceIterator; +import org.apache.iotdb.db.storageengine.dataregion.compaction.execute.utils.reader.IDataBlockReader; +import org.apache.iotdb.db.storageengine.dataregion.memtable.AlignedWritableMemChunkGroup; +import org.apache.iotdb.db.storageengine.dataregion.memtable.IMemTable; +import org.apache.iotdb.db.storageengine.dataregion.memtable.IWritableMemChunk; +import org.apache.iotdb.db.storageengine.dataregion.memtable.IWritableMemChunkGroup; +import org.apache.iotdb.db.storageengine.dataregion.memtable.TsFileProcessor; +import org.apache.iotdb.db.storageengine.dataregion.read.QueryDataSource; +import org.apache.iotdb.db.storageengine.dataregion.read.control.FileReaderManager; +import org.apache.iotdb.db.storageengine.dataregion.read.control.QueryResourceManager; +import org.apache.iotdb.db.storageengine.dataregion.tsfile.TsFileResource; +import org.apache.iotdb.mpp.rpc.thrift.TConsistencyDeletionSummary; + +import org.apache.tsfile.block.column.Column; +import org.apache.tsfile.enums.TSDataType; +import org.apache.tsfile.file.metadata.IDeviceID; +import org.apache.tsfile.read.common.block.TsBlock; +import org.apache.tsfile.utils.Binary; +import org.apache.tsfile.write.schema.IMeasurementSchema; +import org.apache.tsfile.write.schema.MeasurementSchema; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.io.IOException; +import java.nio.ByteBuffer; +import java.nio.channels.ClosedChannelException; +import java.nio.charset.StandardCharsets; +import java.util.ArrayList; +import java.util.Base64; +import java.util.Collections; +import java.util.Comparator; +import java.util.HashMap; +import java.util.LinkedHashMap; +import java.util.LinkedHashSet; +import java.util.List; +import java.util.Map; +import java.util.Set; +import java.util.TreeMap; +import java.util.concurrent.Callable; +import java.util.concurrent.ConcurrentHashMap; +import java.util.stream.Collectors; + +/** + * DataNode-side logical snapshot manager. The snapshot is built from the merged logical visible + * view instead of TsFile correspondence, so physical file layout heterogeneity does not affect + * consistency results. + */ +public class DataRegionConsistencyManager { + + private static final Logger LOGGER = LoggerFactory.getLogger(DataRegionConsistencyManager.class); + private static final DataRegionConsistencyManager INSTANCE = new DataRegionConsistencyManager(); + + private static final int DEVICE_SHARD_COUNT = 256; + private static final long LEAF_TIME_BUCKET_MS = 3_600_000L; + private static final String TREE_KIND_LIVE = "LIVE"; + private static final String TREE_KIND_TOMBSTONE = "TOMBSTONE"; + private static final String BATCH_KIND_RESET_LEAF = "RESET_LEAF"; + private static final String BATCH_KIND_RESET_SCOPE = "RESET_SCOPE"; + private static final String BATCH_KIND_INSERT_ROWS = "INSERT_ROWS"; + private static final String BATCH_KIND_DELETE_DATA = "DELETE_DATA"; + private static final int SNAPSHOT_REBUILD_MAX_ATTEMPTS = 2; + + private final ConcurrentHashMap regionStates = new ConcurrentHashMap<>(); + private final ThreadLocal repairMutationContext = new ThreadLocal<>(); + + public static DataRegionConsistencyManager getInstance() { + return INSTANCE; + } + + public PartitionInspection inspectPartition( + TConsensusGroupId consensusGroupId, + DataRegion dataRegion, + long partitionId, + List partitionDeletionSummaries) { + RegionState regionState = getOrCreateRegionState(consensusGroupId); + PartitionState partitionState = + regionState.partitions.computeIfAbsent(partitionId, PartitionState::new); + synchronized (partitionState) { + if (partitionState.snapshotState != RepairProgressTable.SnapshotState.READY + || partitionState.snapshotEpoch != partitionState.partitionMutationEpoch) { + try { + rebuildSnapshot(partitionState, dataRegion, partitionId, partitionDeletionSummaries); + } catch (Exception e) { + partitionState.snapshotState = RepairProgressTable.SnapshotState.FAILED; + partitionState.lastError = e.getMessage(); + LOGGER.warn( + "Failed to build logical consistency snapshot for region {} partition {}", + consensusGroupId, + partitionId, + e); + } + } + return partitionState.toInspection(partitionId); + } + } + + public SnapshotSubtreeResult getSnapshotSubtree( + TConsensusGroupId consensusGroupId, + DataRegion dataRegion, + long partitionId, + long snapshotEpoch, + String treeKind, + List nodeHandles, + List partitionDeletionSummaries) { + PartitionState partitionState = + getOrCreateRegionState(consensusGroupId) + .partitions + .computeIfAbsent(partitionId, PartitionState::new); + synchronized (partitionState) { + if (!ensureReadySnapshot( + partitionState, dataRegion, partitionId, snapshotEpoch, partitionDeletionSummaries)) { + return SnapshotSubtreeResult.stale(snapshotEpoch); + } + SnapshotTree tree = partitionState.getTree(treeKind); + List resultNodes = new ArrayList<>(); + List requestedHandles = + nodeHandles == null || nodeHandles.isEmpty() + ? Collections.singletonList(SnapshotTree.ROOT_HANDLE) + : nodeHandles; + for (String handle : requestedHandles) { + SnapshotNode node = tree.nodesByHandle.get(handle); + if (node == null) { + continue; + } + if (node.leaf) { + resultNodes.add(node); + continue; + } + for (String childHandle : node.childrenHandles) { + SnapshotNode child = tree.nodesByHandle.get(childHandle); + if (child != null) { + resultNodes.add(child); + } + } + } + resultNodes.sort(Comparator.comparing(SnapshotNode::getNodeHandle)); + return SnapshotSubtreeResult.ready(snapshotEpoch, resultNodes); + } + } + + public LeafEstimate estimateLeaf( + TConsensusGroupId consensusGroupId, + DataRegion dataRegion, + long partitionId, + long snapshotEpoch, + String treeKind, + String leafId, + List partitionDeletionSummaries) { + PartitionState partitionState = + getOrCreateRegionState(consensusGroupId) + .partitions + .computeIfAbsent(partitionId, PartitionState::new); + synchronized (partitionState) { + if (!ensureReadySnapshot( + partitionState, dataRegion, partitionId, snapshotEpoch, partitionDeletionSummaries)) { + return null; + } + SnapshotNode leaf = partitionState.getTree(treeKind).nodesByHandle.get(leafId); + if (leaf == null || !leaf.leaf) { + return null; + } + return new LeafEstimate( + partitionId, + snapshotEpoch, + treeKind, + leafId, + leaf.itemCount, + TREE_KIND_TOMBSTONE.equalsIgnoreCase(treeKind) ? leaf.itemCount : 0L, + leaf.itemCount, + leaf.keyRangeStart, + leaf.keyRangeEnd); + } + } + + public List decodeLeaf( + TConsensusGroupId consensusGroupId, + DataRegion dataRegion, + long partitionId, + long snapshotEpoch, + String treeKind, + String leafId, + List partitionDeletionSummaries) + throws Exception { + PartitionState partitionState = + getOrCreateRegionState(consensusGroupId) + .partitions + .computeIfAbsent(partitionId, PartitionState::new); + synchronized (partitionState) { + if (!ensureReadySnapshot( + partitionState, dataRegion, partitionId, snapshotEpoch, partitionDeletionSummaries)) { + return null; + } + } + + LogicalLeafSelector selector = LogicalLeafSelector.parse(leafId); + List entries = new ArrayList<>(); + if (TREE_KIND_TOMBSTONE.equalsIgnoreCase(treeKind)) { + for (TConsistencyDeletionSummary summary : partitionDeletionSummaries) { + if (selector.matches(summary)) { + entries.add(new LeafDiffEntry(encodeDeletionKey(summary), "LOCAL")); + } + } + } else { + scanLiveCells( + dataRegion, + partitionId, + (deviceId, measurement, type, time, value, aligned) -> { + if (selector.matches(deviceId, time)) { + entries.add( + new LeafDiffEntry( + encodeLogicalCell(deviceId, measurement, time, type, value), "LOCAL")); + } + }); + } + entries.sort(Comparator.comparing(LeafDiffEntry::getLogicalKey)); + return entries; + } + + public List streamLogicalRepair( + TConsensusGroupId consensusGroupId, + DataRegion dataRegion, + long partitionId, + String repairEpoch, + List leafSelectors, + List partitionDeletionSummaries) + throws Exception { + PartitionState partitionState = + getOrCreateRegionState(consensusGroupId) + .partitions + .computeIfAbsent(partitionId, PartitionState::new); + synchronized (partitionState) { + ensureRepairEpochReadySnapshot( + partitionState, dataRegion, partitionId, repairEpoch, partitionDeletionSummaries); + } + + String sessionId = buildRepairSessionId(repairEpoch, leafSelectors); + List batches = new ArrayList<>(); + final int[] seqNo = {0}; + for (LeafSelector selector : leafSelectors) { + if (TREE_KIND_TOMBSTONE.equalsIgnoreCase(selector.treeKind)) { + for (TConsistencyDeletionSummary summary : partitionDeletionSummaries) { + if (!selector.selector.matches(summary)) { + continue; + } + DeleteDataNode deleteDataNode = buildDeleteDataNode(summary); + batches.add( + new LogicalRepairBatch( + sessionId, + selector.treeKind, + selector.selector.leafId, + seqNo[0]++, + BATCH_KIND_DELETE_DATA, + deleteDataNode.serializeToByteBuffer())); + } + continue; + } + + batches.add( + new LogicalRepairBatch( + sessionId, + selector.treeKind, + selector.selector.leafId, + seqNo[0]++, + selector.selector.requiresScopedReset() + ? BATCH_KIND_RESET_SCOPE + : BATCH_KIND_RESET_LEAF, + selector.selector.requiresScopedReset() + ? selector.selector.serialize() + : ByteBuffer.allocate(0))); + + List bufferedRows = new ArrayList<>(); + scanLiveCells( + dataRegion, + partitionId, + (deviceId, measurement, type, time, value, aligned) -> { + if (!selector.selector.matchesLiveCell(deviceId, measurement, type, time, value)) { + return; + } + bufferedRows.add(buildInsertRow(deviceId, measurement, type, time, value, aligned)); + if (bufferedRows.size() >= 256) { + batches.add( + new LogicalRepairBatch( + sessionId, + selector.treeKind, + selector.selector.leafId, + seqNo[0]++, + BATCH_KIND_INSERT_ROWS, + toInsertRowsNode(bufferedRows).serializeToByteBuffer())); + bufferedRows.clear(); + } + }); + if (!bufferedRows.isEmpty()) { + batches.add( + new LogicalRepairBatch( + sessionId, + selector.treeKind, + selector.selector.leafId, + seqNo[0]++, + BATCH_KIND_INSERT_ROWS, + toInsertRowsNode(bufferedRows).serializeToByteBuffer())); + } + } + return batches; + } + + public T runWithLogicalRepairMutation( + TConsensusGroupId consensusGroupId, long partitionId, String repairEpoch, Callable action) + throws Exception { + RepairMutationContext previousContext = repairMutationContext.get(); + repairMutationContext.set( + new RepairMutationContext(consensusGroupId.toString(), partitionId, repairEpoch)); + try { + return action.call(); + } finally { + if (previousContext == null) { + repairMutationContext.remove(); + } else { + repairMutationContext.set(previousContext); + } + } + } + + public void resetLiveLeaf(DataRegion dataRegion, long partitionId, String leafId) + throws Exception { + LogicalLeafSelector selector = LogicalLeafSelector.parse(leafId); + resetLiveBySelector(dataRegion, partitionId, selector); + } + + public void resetLiveScope(DataRegion dataRegion, long partitionId, ByteBuffer selectorPayload) + throws Exception { + LogicalLeafSelector selector = LogicalLeafSelector.deserialize(selectorPayload); + resetLiveBySelector(dataRegion, partitionId, selector); + } + + private void resetLiveBySelector( + DataRegion dataRegion, long partitionId, LogicalLeafSelector selector) throws Exception { + if (selector.requiresScopedReset()) { + resetLiveScopedCells(dataRegion, partitionId, selector); + return; + } + Set fullPaths = new LinkedHashSet<>(); + scanLiveCells( + dataRegion, + partitionId, + (deviceId, measurement, type, time, value, aligned) -> { + if (selector.matches(deviceId, time)) { + fullPaths.add(deviceId + "." + measurement); + } + }); + if (fullPaths.isEmpty()) { + return; + } + List paths = new ArrayList<>(fullPaths.size()); + for (String fullPath : fullPaths) { + paths.add(new MeasurementPath(fullPath)); + } + long bucketStart = selector.bucket * LEAF_TIME_BUCKET_MS; + long bucketEnd = bucketStart + LEAF_TIME_BUCKET_MS - 1; + DeleteDataNode deleteDataNode = + new DeleteDataNode(new PlanNodeId("logical-reset-leaf"), paths, bucketStart, bucketEnd); + for (MeasurementPath path : paths) { + dataRegion.deleteByDevice(path, deleteDataNode); + } + } + + private void resetLiveScopedCells( + DataRegion dataRegion, long partitionId, LogicalLeafSelector selector) throws Exception { + Map> timesByPath = new HashMap<>(); + scanLiveCells( + dataRegion, + partitionId, + (deviceId, measurement, type, time, value, aligned) -> { + if (!selector.matchesLiveCell(deviceId, measurement, type, time, value)) { + return; + } + timesByPath + .computeIfAbsent(deviceId + "." + measurement, ignored -> new LinkedHashSet<>()) + .add(time); + }); + for (Map.Entry> entry : timesByPath.entrySet()) { + MeasurementPath path = new MeasurementPath(entry.getKey()); + for (Long time : entry.getValue()) { + DeleteDataNode deleteDataNode = + new DeleteDataNode( + new PlanNodeId("logical-reset-scope"), Collections.singletonList(path), time, time); + dataRegion.deleteByDevice(path, deleteDataNode); + } + } + } + + public void onTsFileClosed(TConsensusGroupId consensusGroupId, TsFileResource tsFileResource) { + // Logical snapshots are driven by mutation epoch, not close-file events. + } + + public void onCompaction( + TConsensusGroupId consensusGroupId, + List seqSourceFiles, + List unseqSourceFiles, + List targetFiles, + long timePartition) { + // Compaction only changes physical layout and must not dirty logical snapshots. + } + + public void onDeletion(TConsensusGroupId consensusGroupId, List affectedTsFiles) { + if (affectedTsFiles == null) { + return; + } + for (TsFileResource resource : affectedTsFiles) { + onPartitionMutation(consensusGroupId, resource.getTimePartition()); + } + } + + public void onDeletion(TConsensusGroupId consensusGroupId, long startTime, long endTime) { + long partitionInterval = + org.apache.iotdb.commons.conf.CommonDescriptor.getInstance() + .getConfig() + .getTimePartitionInterval(); + if (partitionInterval <= 0) { + onPartitionMutation(consensusGroupId, 0L); + return; + } + long startPartition = Math.floorDiv(startTime, partitionInterval); + long endPartition = Math.floorDiv(endTime, partitionInterval); + for (long partitionId = startPartition; partitionId <= endPartition; partitionId++) { + onPartitionMutation(consensusGroupId, partitionId); + } + } + + public void onPartitionMutation(TConsensusGroupId consensusGroupId, long partitionId) { + PartitionState state = + getOrCreateRegionState(consensusGroupId) + .partitions + .computeIfAbsent(partitionId, PartitionState::new); + synchronized (state) { + if (isRepairMutation(consensusGroupId, partitionId)) { + state.snapshotState = RepairProgressTable.SnapshotState.DIRTY; + state.liveTree = SnapshotTree.empty(); + state.tombstoneTree = SnapshotTree.empty(); + state.lastError = null; + return; + } + state.partitionMutationEpoch++; + state.snapshotState = RepairProgressTable.SnapshotState.DIRTY; + state.liveTree = SnapshotTree.empty(); + state.tombstoneTree = SnapshotTree.empty(); + state.lastError = null; + } + } + + private RegionState getOrCreateRegionState(TConsensusGroupId consensusGroupId) { + return regionStates.computeIfAbsent(consensusGroupId.toString(), ignored -> new RegionState()); + } + + private boolean ensureReadySnapshot( + PartitionState partitionState, + DataRegion dataRegion, + long partitionId, + long snapshotEpoch, + List partitionDeletionSummaries) { + if (partitionState.snapshotEpoch == snapshotEpoch + && partitionState.snapshotState == RepairProgressTable.SnapshotState.READY) { + return true; + } + if (partitionState.partitionMutationEpoch != snapshotEpoch) { + return false; + } + try { + rebuildSnapshot(partitionState, dataRegion, partitionId, partitionDeletionSummaries); + return partitionState.snapshotEpoch == snapshotEpoch + && partitionState.snapshotState == RepairProgressTable.SnapshotState.READY; + } catch (Exception e) { + partitionState.snapshotState = RepairProgressTable.SnapshotState.FAILED; + partitionState.lastError = e.getMessage(); + return false; + } + } + + private void rebuildSnapshot( + PartitionState partitionState, + DataRegion dataRegion, + long partitionId, + List partitionDeletionSummaries) + throws Exception { + long expectedMutationEpoch = partitionState.partitionMutationEpoch; + partitionState.snapshotState = RepairProgressTable.SnapshotState.BUILDING; + + for (int attempt = 1; attempt <= SNAPSHOT_REBUILD_MAX_ATTEMPTS; attempt++) { + SnapshotTree liveTree = SnapshotTree.empty(); + SnapshotTree tombstoneTree = SnapshotTree.empty(); + + try { + scanLiveCells( + dataRegion, + partitionId, + (deviceId, measurement, type, time, value, aligned) -> { + int shard = computeDeviceShard(deviceId); + long bucket = Math.floorDiv(time, LEAF_TIME_BUCKET_MS); + String leafId = LogicalLeafSelector.leafId(shard, bucket); + String logicalKey = encodeLogicalCell(deviceId, measurement, time, type, value); + long hash = hashLogicalCell(deviceId, measurement, time, type, value); + liveTree.addLeafEntry(leafId, shard, logicalKey, hash, 1L); + }); + } catch (Exception e) { + if (attempt >= SNAPSHOT_REBUILD_MAX_ATTEMPTS || !isRetryableSnapshotReadFailure(e)) { + throw e; + } + invalidateSnapshotReaders(dataRegion, partitionId); + LOGGER.info( + "Retrying logical consistency snapshot build for partition {} after refreshing stale readers", + partitionId); + continue; + } + + for (TConsistencyDeletionSummary summary : partitionDeletionSummaries) { + int shard = computeDeviceShard(summary.getPathPattern()); + long startBucket = Math.floorDiv(summary.getTimeRangeStart(), LEAF_TIME_BUCKET_MS); + long endBucket = Math.floorDiv(summary.getTimeRangeEnd(), LEAF_TIME_BUCKET_MS); + String deletionKey = encodeDeletionKey(summary); + for (long bucket = startBucket; bucket <= endBucket; bucket++) { + String leafId = LogicalLeafSelector.leafId(shard, bucket); + tombstoneTree.addLeafEntry(leafId, shard, deletionKey, hashDeletion(summary), 1L); + } + } + + liveTree.finalizeTree(); + tombstoneTree.finalizeTree(); + + if (expectedMutationEpoch != partitionState.partitionMutationEpoch) { + partitionState.snapshotState = RepairProgressTable.SnapshotState.DIRTY; + return; + } + partitionState.snapshotEpoch = expectedMutationEpoch; + partitionState.snapshotState = RepairProgressTable.SnapshotState.READY; + partitionState.liveTree = liveTree; + partitionState.tombstoneTree = tombstoneTree; + partitionState.lastError = null; + return; + } + + throw new IllegalStateException( + "Logical consistency snapshot rebuild finished without a terminal result"); + } + + private void ensureRepairEpochReadySnapshot( + PartitionState partitionState, + DataRegion dataRegion, + long partitionId, + String repairEpoch, + List partitionDeletionSummaries) + throws Exception { + RepairEpochRef repairEpochRef = RepairEpochRef.parse(repairEpoch); + if (repairEpochRef.partitionId != partitionId) { + throw new IllegalStateException( + "Repair epoch partition " + + repairEpochRef.partitionId + + " does not match requested partition " + + partitionId); + } + if (partitionState.partitionMutationEpoch != repairEpochRef.partitionMutationEpoch) { + throw new IllegalStateException( + "Repair epoch drift detected for partition " + + partitionId + + ": expected mutation epoch " + + repairEpochRef.partitionMutationEpoch + + ", actual " + + partitionState.partitionMutationEpoch); + } + if (partitionState.snapshotState != RepairProgressTable.SnapshotState.READY + || partitionState.snapshotEpoch != repairEpochRef.snapshotEpoch) { + rebuildSnapshot(partitionState, dataRegion, partitionId, partitionDeletionSummaries); + } + if (partitionState.snapshotState != RepairProgressTable.SnapshotState.READY + || partitionState.snapshotEpoch != repairEpochRef.snapshotEpoch + || partitionState.partitionMutationEpoch != repairEpochRef.partitionMutationEpoch) { + throw new IllegalStateException( + "Repair epoch drift detected after snapshot rebuild for partition " + partitionId); + } + } + + private boolean isRepairMutation(TConsensusGroupId consensusGroupId, long partitionId) { + RepairMutationContext mutationContext = repairMutationContext.get(); + return mutationContext != null + && mutationContext.partitionId == partitionId + && mutationContext.consensusGroupKey.equals(consensusGroupId.toString()); + } + + private void scanLiveCells(DataRegion dataRegion, long partitionId, LiveCellConsumer consumer) + throws Exception { + Map deviceSeriesContexts = + collectLogicalSeriesContexts(dataRegion, partitionId); + if (deviceSeriesContexts.isEmpty()) { + return; + } + + long queryId = QueryResourceManager.getInstance().assignInternalQueryId(); + FragmentInstanceContext context = + FragmentInstanceContext.createFragmentInstanceContextForCompaction(queryId); + + try { + for (DeviceSeriesContext deviceSeriesContext : deviceSeriesContexts.values()) { + List paths = deviceSeriesContext.buildPaths(); + if (paths.isEmpty()) { + continue; + } + + QueryDataSource dataSource = + dataRegion.query( + paths, + deviceSeriesContext.deviceId, + context, + null, + Collections.singletonList(partitionId)); + if (dataSource == null || dataSource.isEmpty()) { + continue; + } + + QueryResourceManager.getInstance() + .getQueryFileManager() + .addUsedFilesForQuery(queryId, dataSource); + dataSource.fillOrderIndexes(deviceSeriesContext.deviceId, true); + String deviceId = String.valueOf(deviceSeriesContext.deviceId); + if (deviceSeriesContext.aligned) { + QueryDataSource deviceDataSource = copyQueryDataSourceForDevice(dataSource); + IDataBlockReader reader = + ReadPointCompactionPerformer.constructReader( + deviceSeriesContext.deviceId, + deviceSeriesContext.getMeasurementNames(), + deviceSeriesContext.getMeasurementSchemas(), + deviceSeriesContext.getMeasurementNames(), + context, + deviceDataSource, + true); + consumeAligned( + reader, + deviceId, + deviceSeriesContext.getMeasurementNames(), + deviceSeriesContext.measurementSchemas, + consumer); + } else { + for (String measurementName : deviceSeriesContext.getMeasurementNames()) { + IMeasurementSchema schema = deviceSeriesContext.measurementSchemas.get(measurementName); + QueryDataSource deviceDataSource = copyQueryDataSourceForDevice(dataSource); + IDataBlockReader reader = + ReadPointCompactionPerformer.constructReader( + deviceSeriesContext.deviceId, + Collections.singletonList(measurementName), + Collections.singletonList(schema), + deviceSeriesContext.getMeasurementNames(), + context, + deviceDataSource, + false); + consumeNonAligned(reader, deviceId, measurementName, schema, consumer); + } + } + } + } finally { + QueryResourceManager.getInstance().endQuery(queryId); + } + } + + private void consumeAligned( + IDataBlockReader reader, + String deviceId, + List measurementNames, + Map schemaMap, + LiveCellConsumer consumer) + throws Exception { + while (reader.hasNextBatch()) { + TsBlock tsBlock = reader.nextBatch(); + for (int row = 0; row < tsBlock.getPositionCount(); row++) { + long time = tsBlock.getTimeByIndex(row); + for (int columnIndex = 0; columnIndex < tsBlock.getValueColumnCount(); columnIndex++) { + Column column = tsBlock.getColumn(columnIndex); + if (column.isNull(row)) { + continue; + } + String measurement = measurementNames.get(columnIndex); + IMeasurementSchema schema = schemaMap.get(measurement); + consumer.accept( + deviceId, + measurement, + schema.getType(), + time, + extractValue(column, row, schema.getType()), + true); + } + } + } + } + + private void consumeNonAligned( + IDataBlockReader reader, + String deviceId, + String measurement, + IMeasurementSchema schema, + LiveCellConsumer consumer) + throws Exception { + while (reader.hasNextBatch()) { + TsBlock tsBlock = reader.nextBatch(); + Column column = tsBlock.getColumn(0); + for (int row = 0; row < tsBlock.getPositionCount(); row++) { + if (column.isNull(row)) { + continue; + } + consumer.accept( + deviceId, + measurement, + schema.getType(), + tsBlock.getTimeByIndex(row), + extractValue(column, row, schema.getType()), + false); + } + } + } + + private Object extractValue(Column column, int rowIndex, TSDataType dataType) { + switch (dataType) { + case BOOLEAN: + return column.getBoolean(rowIndex); + case INT32: + case DATE: + return column.getInt(rowIndex); + case INT64: + case TIMESTAMP: + return column.getLong(rowIndex); + case FLOAT: + return column.getFloat(rowIndex); + case DOUBLE: + return column.getDouble(rowIndex); + case TEXT: + case STRING: + case BLOB: + return column.getBinary(rowIndex); + default: + return String.valueOf(column.getObject(rowIndex)); + } + } + + private Map collectLogicalSeriesContexts( + DataRegion dataRegion, long partitionId) throws Exception { + Map deviceSeriesContexts = new TreeMap<>(); + + List seqResources = + new ArrayList<>(dataRegion.getTsFileManager().getTsFileListSnapshot(partitionId, true)); + List unseqResources = + new ArrayList<>(dataRegion.getTsFileManager().getTsFileListSnapshot(partitionId, false)); + pruneClosedResources(seqResources); + pruneClosedResources(unseqResources); + + if (!seqResources.isEmpty() || !unseqResources.isEmpty()) { + // Use dedicated metadata readers here instead of the shared FileReaderManager-backed read + // point iterator. The snapshot rebuild later performs a real logical scan through the query + // engine, so closing the schema-discovery iterator must never poison the shared reader cache. + try (MultiTsFileDeviceIterator deviceIterator = + new MultiTsFileDeviceIterator(seqResources, unseqResources, new HashMap<>())) { + while (deviceIterator.hasNextDevice()) { + org.apache.tsfile.utils.Pair deviceInfo = deviceIterator.nextDevice(); + DeviceSeriesContext deviceSeriesContext = + deviceSeriesContexts.computeIfAbsent( + String.valueOf(deviceInfo.left), + ignored -> new DeviceSeriesContext(deviceInfo.left, deviceInfo.right)); + deviceSeriesContext.mergeAlignment(deviceInfo.right); + + Map rawSchemaMap = + deviceIterator.getAllSchemasOfCurrentDevice(); + rawSchemaMap.remove("time"); + for (MeasurementSchema schema : rawSchemaMap.values()) { + deviceSeriesContext.mergeMeasurement(schema); + } + } + } + } + + collectOpenProcessorSeriesContexts( + dataRegion.getWorkSequenceTsFileProcessors(), partitionId, deviceSeriesContexts); + collectOpenProcessorSeriesContexts( + dataRegion.getWorkUnsequenceTsFileProcessors(), partitionId, deviceSeriesContexts); + return deviceSeriesContexts; + } + + private void collectOpenProcessorSeriesContexts( + Iterable processors, + long partitionId, + Map deviceSeriesContexts) + throws IOException, InterruptedException { + for (TsFileProcessor processor : processors) { + if (processor == null || processor.getTimeRangeId() != partitionId) { + continue; + } + + if (!processor.tryReadLock(1_000L)) { + throw new IOException( + "Failed to acquire logical snapshot read lock for time partition " + partitionId); + } + try { + collectMemTableSeriesContexts(processor.getWorkMemTable(), deviceSeriesContexts); + for (IMemTable flushingMemTable : processor.getFlushingMemTable()) { + collectMemTableSeriesContexts(flushingMemTable, deviceSeriesContexts); + } + } finally { + processor.readUnLock(); + } + } + } + + private void collectMemTableSeriesContexts( + IMemTable memTable, Map deviceSeriesContexts) { + if (memTable == null || memTable.getMemTableMap().isEmpty()) { + return; + } + + for (Map.Entry entry : + memTable.getMemTableMap().entrySet()) { + IWritableMemChunkGroup memChunkGroup = entry.getValue(); + if (memChunkGroup == null || memChunkGroup.isEmpty()) { + continue; + } + + boolean aligned = memChunkGroup instanceof AlignedWritableMemChunkGroup; + DeviceSeriesContext deviceSeriesContext = + deviceSeriesContexts.computeIfAbsent( + String.valueOf(entry.getKey()), + ignored -> new DeviceSeriesContext(entry.getKey(), aligned)); + deviceSeriesContext.mergeAlignment(aligned); + + if (aligned) { + for (IMeasurementSchema schema : + ((AlignedWritableMemChunkGroup) memChunkGroup).getAlignedMemChunk().getSchemaList()) { + deviceSeriesContext.mergeMeasurement(schema); + } + continue; + } + + for (IWritableMemChunk memChunk : memChunkGroup.getMemChunkMap().values()) { + if (memChunk == null) { + continue; + } + deviceSeriesContext.mergeMeasurement(memChunk.getSchema()); + } + } + } + + private void pruneClosedResources(List resources) { + resources.removeIf( + resource -> !resource.isClosed() || resource.isDeleted() || !resource.getTsFile().exists()); + resources.sort(TsFileResource::compareFileName); + } + + private QueryDataSource copyQueryDataSourceForDevice(QueryDataSource dataSource) { + QueryDataSource deviceDataSource = new QueryDataSource(dataSource); + deviceDataSource.setSingleDevice(true); + return deviceDataSource; + } + + private boolean isRetryableSnapshotReadFailure(Throwable throwable) { + Throwable current = throwable; + while (current != null) { + if (current instanceof ClosedChannelException) { + return true; + } + current = current.getCause(); + } + return false; + } + + private void invalidateSnapshotReaders(DataRegion dataRegion, long partitionId) { + DataRegion.operateClearCache(); + closeClosedSnapshotReaders( + dataRegion.getTsFileManager().getTsFileListSnapshot(partitionId, true)); + closeClosedSnapshotReaders( + dataRegion.getTsFileManager().getTsFileListSnapshot(partitionId, false)); + } + + private void closeClosedSnapshotReaders(List resources) { + if (resources == null) { + return; + } + for (TsFileResource resource : resources) { + if (resource == null || !resource.isClosed()) { + continue; + } + try { + FileReaderManager.getInstance().closeFileAndRemoveReader(resource.getTsFileID()); + } catch (IOException e) { + LOGGER.debug( + "Failed to invalidate cached reader for logical snapshot file {}", + resource.getTsFilePath(), + e); + } + } + } + + private static final class DeviceSeriesContext { + private final IDeviceID deviceId; + private boolean aligned; + private final Map measurementSchemas = new TreeMap<>(); + + private DeviceSeriesContext(IDeviceID deviceId, boolean aligned) { + this.deviceId = deviceId; + this.aligned = aligned; + } + + private void mergeAlignment(boolean newAligned) { + aligned = aligned || newAligned; + } + + private void mergeMeasurement(IMeasurementSchema schema) { + if (schema == null + || schema.getMeasurementName() == null + || schema.getMeasurementName().isEmpty()) { + return; + } + measurementSchemas.putIfAbsent(schema.getMeasurementName(), schema); + } + + private List getMeasurementNames() { + return new ArrayList<>(measurementSchemas.keySet()); + } + + private List getMeasurementSchemas() { + return new ArrayList<>(measurementSchemas.values()); + } + + private List buildPaths() { + if (measurementSchemas.isEmpty()) { + return Collections.emptyList(); + } + if (aligned) { + return Collections.singletonList( + new AlignedFullPath(deviceId, getMeasurementNames(), getMeasurementSchemas())); + } + + List paths = new ArrayList<>(measurementSchemas.size()); + for (IMeasurementSchema schema : measurementSchemas.values()) { + paths.add(new NonAlignedFullPath(deviceId, schema)); + } + return paths; + } + } + + private InsertRowNode buildInsertRow( + String deviceId, + String measurement, + TSDataType type, + long time, + Object value, + boolean aligned) { + try { + MeasurementSchema[] schemas = {new MeasurementSchema(measurement, type)}; + return new InsertRowNode( + new PlanNodeId("logical-repair-row"), + new PartialPath(deviceId), + aligned, + new String[] {measurement}, + new TSDataType[] {type}, + schemas, + time, + new Object[] {value}, + false); + } catch (Exception e) { + throw new IllegalStateException( + "Failed to create logical repair row for " + deviceId + "." + measurement, e); + } + } + + private InsertRowsNode toInsertRowsNode(List rows) { + List clonedRows = new ArrayList<>(rows); + List indexes = new ArrayList<>(rows.size()); + for (int i = 0; i < rows.size(); i++) { + indexes.add(i); + } + return new InsertRowsNode(new PlanNodeId("logical-repair-rows"), indexes, clonedRows); + } + + private DeleteDataNode buildDeleteDataNode(TConsistencyDeletionSummary summary) { + try { + return new DeleteDataNode( + new PlanNodeId("logical-repair-delete"), + Collections.singletonList(new MeasurementPath(summary.getPathPattern())), + summary.getTimeRangeStart(), + summary.getTimeRangeEnd()); + } catch (Exception e) { + throw new IllegalStateException( + "Failed to build logical repair delete for " + summary.getPathPattern(), e); + } + } + + private static int computeDeviceShard(String deviceId) { + return Math.floorMod(deviceId.hashCode(), DEVICE_SHARD_COUNT); + } + + private String buildRepairSessionId(String repairEpoch, List leafSelectors) { + String selectorFingerprint = + leafSelectors.stream() + .map(selector -> selector.treeKind + '@' + selector.selector.toSelectorToken()) + .sorted() + .collect(Collectors.joining(",")); + return java.util + .UUID + .nameUUIDFromBytes( + (repairEpoch + "|" + selectorFingerprint).getBytes(StandardCharsets.UTF_8)) + .toString(); + } + + private long hashLogicalCell( + String deviceId, String measurement, long time, TSDataType dataType, Object value) { + long hash = 17L; + hash = 31L * hash + deviceId.hashCode(); + hash = 31L * hash + measurement.hashCode(); + hash = 31L * hash + Long.hashCode(time); + hash = 31L * hash + dataType.ordinal(); + hash = 31L * hash + valueHash(value); + return hash; + } + + private int valueHash(Object value) { + if (value == null) { + return 0; + } + if (value instanceof Binary) { + return value.toString().hashCode(); + } + return value.hashCode(); + } + + private String encodeLogicalCell( + String deviceId, String measurement, long time, TSDataType dataType, Object value) { + return deviceId + + '|' + + measurement + + '|' + + time + + '|' + + dataType.name() + + '|' + + valueHash(value); + } + + private long hashDeletion(TConsistencyDeletionSummary summary) { + long hash = 19L; + hash = 31L * hash + summary.getPathPattern().hashCode(); + hash = 31L * hash + Long.hashCode(summary.getTimeRangeStart()); + hash = 31L * hash + Long.hashCode(summary.getTimeRangeEnd()); + return hash; + } + + private String encodeDeletionKey(TConsistencyDeletionSummary summary) { + return encodeDeletionKeyStatic(summary); + } + + public static class PartitionInspection { + private final long partitionId; + private final long partitionMutationEpoch; + private final long snapshotEpoch; + private final RepairProgressTable.SnapshotState snapshotState; + private final DualDigest liveRootDigest; + private final DualDigest tombstoneRootDigest; + private final String lastError; + + private PartitionInspection( + long partitionId, + long partitionMutationEpoch, + long snapshotEpoch, + RepairProgressTable.SnapshotState snapshotState, + DualDigest liveRootDigest, + DualDigest tombstoneRootDigest, + String lastError) { + this.partitionId = partitionId; + this.partitionMutationEpoch = partitionMutationEpoch; + this.snapshotEpoch = snapshotEpoch; + this.snapshotState = snapshotState; + this.liveRootDigest = liveRootDigest; + this.tombstoneRootDigest = tombstoneRootDigest; + this.lastError = lastError; + } + + public long getPartitionId() { + return partitionId; + } + + public long getPartitionMutationEpoch() { + return partitionMutationEpoch; + } + + public long getSnapshotEpoch() { + return snapshotEpoch; + } + + public RepairProgressTable.SnapshotState getSnapshotState() { + return snapshotState; + } + + public DualDigest getLiveRootDigest() { + return liveRootDigest; + } + + public DualDigest getTombstoneRootDigest() { + return tombstoneRootDigest; + } + + public String getLastError() { + return lastError; + } + } + + public static class SnapshotSubtreeResult { + private final long snapshotEpoch; + private final boolean stale; + private final List nodes; + + private SnapshotSubtreeResult(long snapshotEpoch, boolean stale, List nodes) { + this.snapshotEpoch = snapshotEpoch; + this.stale = stale; + this.nodes = nodes; + } + + public static SnapshotSubtreeResult ready(long snapshotEpoch, List nodes) { + return new SnapshotSubtreeResult(snapshotEpoch, false, nodes); + } + + public static SnapshotSubtreeResult stale(long snapshotEpoch) { + return new SnapshotSubtreeResult(snapshotEpoch, true, Collections.emptyList()); + } + + public long getSnapshotEpoch() { + return snapshotEpoch; + } + + public boolean isStale() { + return stale; + } + + public List getNodes() { + return nodes; + } + } + + public static class LeafEstimate { + private final long partitionId; + private final long snapshotEpoch; + private final String treeKind; + private final String leafId; + private final long rowCount; + private final long tombstoneCount; + private final long strataEstimate; + private final String keyRangeStart; + private final String keyRangeEnd; + + private LeafEstimate( + long partitionId, + long snapshotEpoch, + String treeKind, + String leafId, + long rowCount, + long tombstoneCount, + long strataEstimate, + String keyRangeStart, + String keyRangeEnd) { + this.partitionId = partitionId; + this.snapshotEpoch = snapshotEpoch; + this.treeKind = treeKind; + this.leafId = leafId; + this.rowCount = rowCount; + this.tombstoneCount = tombstoneCount; + this.strataEstimate = strataEstimate; + this.keyRangeStart = keyRangeStart; + this.keyRangeEnd = keyRangeEnd; + } + + public long getPartitionId() { + return partitionId; + } + + public long getSnapshotEpoch() { + return snapshotEpoch; + } + + public String getTreeKind() { + return treeKind; + } + + public String getLeafId() { + return leafId; + } + + public long getRowCount() { + return rowCount; + } + + public long getTombstoneCount() { + return tombstoneCount; + } + + public long getStrataEstimate() { + return strataEstimate; + } + + public String getKeyRangeStart() { + return keyRangeStart; + } + + public String getKeyRangeEnd() { + return keyRangeEnd; + } + } + + public static class LeafDiffEntry { + private final String logicalKey; + private final String diffType; + + private LeafDiffEntry(String logicalKey, String diffType) { + this.logicalKey = logicalKey; + this.diffType = diffType; + } + + public String getLogicalKey() { + return logicalKey; + } + + public String getDiffType() { + return diffType; + } + } + + public static class LogicalRepairBatch { + private final String sessionId; + private final String treeKind; + private final String leafId; + private final int seqNo; + private final String batchKind; + private final ByteBuffer payload; + + private LogicalRepairBatch( + String sessionId, + String treeKind, + String leafId, + int seqNo, + String batchKind, + ByteBuffer payload) { + this.sessionId = sessionId; + this.treeKind = treeKind; + this.leafId = leafId; + this.seqNo = seqNo; + this.batchKind = batchKind; + this.payload = payload; + } + + public String getSessionId() { + return sessionId; + } + + public String getTreeKind() { + return treeKind; + } + + public String getLeafId() { + return leafId; + } + + public int getSeqNo() { + return seqNo; + } + + public String getBatchKind() { + return batchKind; + } + + public ByteBuffer getPayload() { + return payload; + } + } + + public static class LeafSelector { + private final String treeKind; + private final LogicalLeafSelector selector; + + public LeafSelector(String treeKind, String leafId) { + this.treeKind = treeKind; + this.selector = LogicalLeafSelector.parse(leafId); + } + } + + public static class SnapshotNode { + private final String parentNodeHandle; + private final String nodeHandle; + private final int depth; + private final boolean leaf; + private final DualDigest digest; + private final long itemCount; + private final String leafId; + private final String keyRangeStart; + private final String keyRangeEnd; + private final List childrenHandles = new ArrayList<>(); + + private SnapshotNode( + String parentNodeHandle, + String nodeHandle, + int depth, + boolean leaf, + DualDigest digest, + long itemCount, + String leafId, + String keyRangeStart, + String keyRangeEnd) { + this.parentNodeHandle = parentNodeHandle; + this.nodeHandle = nodeHandle; + this.depth = depth; + this.leaf = leaf; + this.digest = digest; + this.itemCount = itemCount; + this.leafId = leafId; + this.keyRangeStart = keyRangeStart; + this.keyRangeEnd = keyRangeEnd; + } + + public String getParentNodeHandle() { + return parentNodeHandle; + } + + public String getNodeHandle() { + return nodeHandle; + } + + public int getDepth() { + return depth; + } + + public boolean isLeaf() { + return leaf; + } + + public DualDigest getDigest() { + return digest; + } + + public long getItemCount() { + return itemCount; + } + + public String getLeafId() { + return leafId; + } + + public String getKeyRangeStart() { + return keyRangeStart; + } + + public String getKeyRangeEnd() { + return keyRangeEnd; + } + } + + private interface LiveCellConsumer { + void accept( + String deviceId, + String measurement, + TSDataType type, + long time, + Object value, + boolean aligned) + throws Exception; + } + + private static class RegionState { + private final ConcurrentHashMap partitions = new ConcurrentHashMap<>(); + } + + private static class RepairMutationContext { + private final String consensusGroupKey; + private final long partitionId; + private final String repairEpoch; + + private RepairMutationContext(String consensusGroupKey, long partitionId, String repairEpoch) { + this.consensusGroupKey = consensusGroupKey; + this.partitionId = partitionId; + this.repairEpoch = repairEpoch; + } + } + + private static class RepairEpochRef { + private final long partitionId; + private final long snapshotEpoch; + private final long partitionMutationEpoch; + + private RepairEpochRef(long partitionId, long snapshotEpoch, long partitionMutationEpoch) { + this.partitionId = partitionId; + this.snapshotEpoch = snapshotEpoch; + this.partitionMutationEpoch = partitionMutationEpoch; + } + + private static RepairEpochRef parse(String repairEpoch) { + if (repairEpoch == null || repairEpoch.isEmpty()) { + throw new IllegalStateException("Repair epoch is missing"); + } + String[] parts = repairEpoch.split(":"); + if (parts.length < 5) { + throw new IllegalStateException("Invalid repair epoch: " + repairEpoch); + } + try { + return new RepairEpochRef( + Long.parseLong(parts[1]), Long.parseLong(parts[3]), Long.parseLong(parts[4])); + } catch (NumberFormatException e) { + throw new IllegalStateException("Invalid repair epoch: " + repairEpoch, e); + } + } + } + + private static class PartitionState { + private final long partitionId; + private long partitionMutationEpoch; + private long snapshotEpoch = Long.MIN_VALUE; + private RepairProgressTable.SnapshotState snapshotState = + RepairProgressTable.SnapshotState.DIRTY; + private SnapshotTree liveTree = SnapshotTree.empty(); + private SnapshotTree tombstoneTree = SnapshotTree.empty(); + private String lastError; + + private PartitionState(long partitionId) { + this.partitionId = partitionId; + } + + private PartitionInspection toInspection(long partitionId) { + return new PartitionInspection( + partitionId, + partitionMutationEpoch, + snapshotEpoch, + snapshotState, + liveTree.rootDigest, + tombstoneTree.rootDigest, + lastError); + } + + private SnapshotTree getTree(String treeKind) { + return TREE_KIND_TOMBSTONE.equalsIgnoreCase(treeKind) ? tombstoneTree : liveTree; + } + } + + private static class SnapshotTree { + private static final String ROOT_HANDLE = "root"; + + private final Map nodesByHandle = new LinkedHashMap<>(); + private DualDigest rootDigest = DualDigest.ZERO; + + private static SnapshotTree empty() { + SnapshotTree tree = new SnapshotTree(); + tree.nodesByHandle.put( + ROOT_HANDLE, + new SnapshotNode("", ROOT_HANDLE, 0, false, DualDigest.ZERO, 0L, null, null, null)); + return tree; + } + + private void addLeafEntry( + String leafId, int shard, String logicalKey, long hash, long itemCount) { + String shardHandle = "shard:" + shard; + nodesByHandle.computeIfAbsent( + ROOT_HANDLE, + ignored -> + new SnapshotNode("", ROOT_HANDLE, 0, false, DualDigest.ZERO, 0L, null, null, null)); + SnapshotNode shardNode = + nodesByHandle.computeIfAbsent( + shardHandle, + ignored -> + new SnapshotNode( + ROOT_HANDLE, shardHandle, 1, false, DualDigest.ZERO, 0L, null, null, null)); + if (!nodesByHandle.get(ROOT_HANDLE).childrenHandles.contains(shardHandle)) { + nodesByHandle.get(ROOT_HANDLE).childrenHandles.add(shardHandle); + } + SnapshotNode leafNode = + nodesByHandle.computeIfAbsent( + leafId, + ignored -> + new SnapshotNode( + shardHandle, + leafId, + 2, + true, + DualDigest.ZERO, + 0L, + leafId, + logicalKey, + logicalKey)); + if (!shardNode.childrenHandles.contains(leafId)) { + shardNode.childrenHandles.add(leafId); + } + mergeDigest(leafNode, logicalKey, hash, itemCount); + } + + private void finalizeTree() { + Map rebuilt = new LinkedHashMap<>(); + SnapshotNode root = nodesByHandle.get(ROOT_HANDLE); + if (root == null) { + root = new SnapshotNode("", ROOT_HANDLE, 0, false, DualDigest.ZERO, 0L, null, null, null); + } + DualDigest aggregatedRoot = DualDigest.ZERO; + long rootItemCount = 0L; + rebuilt.put(ROOT_HANDLE, root); + List shardHandles = new ArrayList<>(root.childrenHandles); + shardHandles.sort(String::compareTo); + root.childrenHandles.clear(); + for (String shardHandle : shardHandles) { + SnapshotNode rawShard = nodesByHandle.get(shardHandle); + if (rawShard == null) { + continue; + } + DualDigest shardDigest = DualDigest.ZERO; + long shardItemCount = 0L; + SnapshotNode shard = + new SnapshotNode( + ROOT_HANDLE, shardHandle, 1, false, DualDigest.ZERO, 0L, null, null, null); + List leafHandles = new ArrayList<>(rawShard.childrenHandles); + leafHandles.sort(String::compareTo); + for (String leafHandle : leafHandles) { + SnapshotNode leaf = nodesByHandle.get(leafHandle); + if (leaf == null) { + continue; + } + shard.childrenHandles.add(leafHandle); + shardDigest = shardDigest.merge(leaf.digest); + shardItemCount += leaf.itemCount; + rebuilt.put(leafHandle, leaf); + } + SnapshotNode finalizedShard = + new SnapshotNode( + ROOT_HANDLE, shardHandle, 1, false, shardDigest, shardItemCount, null, null, null); + finalizedShard.childrenHandles.addAll(shard.childrenHandles); + rebuilt.put(shardHandle, finalizedShard); + root.childrenHandles.add(shardHandle); + aggregatedRoot = aggregatedRoot.merge(shardDigest); + rootItemCount += shardItemCount; + } + SnapshotNode finalizedRoot = + new SnapshotNode( + "", ROOT_HANDLE, 0, false, aggregatedRoot, rootItemCount, null, null, null); + finalizedRoot.childrenHandles.addAll(root.childrenHandles); + rebuilt.put(ROOT_HANDLE, finalizedRoot); + nodesByHandle.clear(); + nodesByHandle.putAll(rebuilt); + rootDigest = aggregatedRoot; + } + + private void mergeDigest(SnapshotNode leafNode, String logicalKey, long hash, long itemCount) { + SnapshotNode merged = + new SnapshotNode( + leafNode.parentNodeHandle, + leafNode.nodeHandle, + leafNode.depth, + leafNode.leaf, + leafNode.digest.merge(DualDigest.fromSingleHash(hash)), + leafNode.itemCount + itemCount, + leafNode.leafId, + minComparable(leafNode.keyRangeStart, logicalKey), + maxComparable(leafNode.keyRangeEnd, logicalKey)); + merged.childrenHandles.addAll(leafNode.childrenHandles); + nodesByHandle.put(leafNode.nodeHandle, merged); + } + + private static String minComparable(String left, String right) { + if (left == null) { + return right; + } + if (right == null) { + return left; + } + return left.compareTo(right) <= 0 ? left : right; + } + + private static String maxComparable(String left, String right) { + if (left == null) { + return right; + } + if (right == null) { + return left; + } + return left.compareTo(right) >= 0 ? left : right; + } + } + + private static class LogicalLeafSelector { + private final String leafId; + private final int shard; + private final long bucket; + private final String keyRangeStart; + private final String keyRangeEnd; + private final Set exactKeys; + + private LogicalLeafSelector( + String leafId, + int shard, + long bucket, + String keyRangeStart, + String keyRangeEnd, + Set exactKeys) { + this.leafId = leafId; + this.shard = shard; + this.bucket = bucket; + this.keyRangeStart = keyRangeStart; + this.keyRangeEnd = keyRangeEnd; + this.exactKeys = exactKeys == null ? Collections.emptySet() : new LinkedHashSet<>(exactKeys); + } + + private static LogicalLeafSelector parse(String selectorToken) { + String[] selectorParts = selectorToken.split("@", -1); + String rawLeafId = selectorParts[0]; + String[] parts = rawLeafId.split(":"); + if (parts.length != 3 || !"leaf".equals(parts[0])) { + throw new IllegalArgumentException("Unsupported leaf id: " + selectorToken); + } + return new LogicalLeafSelector( + rawLeafId, + Integer.parseInt(parts[1]), + Long.parseLong(parts[2]), + selectorParts.length >= 2 ? decodeNullable(selectorParts[1]) : null, + selectorParts.length >= 3 ? decodeNullable(selectorParts[2]) : null, + selectorParts.length >= 4 ? decodeStringSet(selectorParts[3]) : Collections.emptySet()); + } + + private static String leafId(int shard, long bucket) { + return "leaf:" + shard + ":" + bucket; + } + + private boolean matches(String deviceId, long time) { + return computeDeviceShard(deviceId) == shard + && Math.floorDiv(time, LEAF_TIME_BUCKET_MS) == bucket; + } + + private boolean matchesLiveCell( + String deviceId, String measurement, TSDataType dataType, long time, Object value) { + if (!matches(deviceId, time)) { + return false; + } + String logicalKey = encodeLogicalCellStatic(deviceId, measurement, time, dataType, value); + if (!isWithinRange(logicalKey)) { + return false; + } + return exactKeys.isEmpty() || exactKeys.contains(logicalKey); + } + + private boolean matches(TConsistencyDeletionSummary summary) { + return computeDeviceShard(summary.getPathPattern()) == shard + && Math.floorDiv(summary.getTimeRangeStart(), LEAF_TIME_BUCKET_MS) <= bucket + && Math.floorDiv(summary.getTimeRangeEnd(), LEAF_TIME_BUCKET_MS) >= bucket + && isWithinRange(encodeDeletionKeyStatic(summary)); + } + + private boolean requiresScopedReset() { + return !exactKeys.isEmpty() || keyRangeStart != null || keyRangeEnd != null; + } + + private ByteBuffer serialize() { + return ByteBuffer.wrap(toSelectorToken().getBytes(StandardCharsets.UTF_8)); + } + + private String toSelectorToken() { + return leafId + + "@" + + encodeNullable(keyRangeStart) + + "@" + + encodeNullable(keyRangeEnd) + + "@" + + encodeStringSet(exactKeys); + } + + private static LogicalLeafSelector deserialize(ByteBuffer buffer) { + ByteBuffer duplicate = buffer.duplicate(); + byte[] bytes = new byte[duplicate.remaining()]; + duplicate.get(bytes); + return parse(new String(bytes, StandardCharsets.UTF_8)); + } + + private static String encodeNullable(String value) { + if (value == null) { + return ""; + } + return Base64.getUrlEncoder() + .withoutPadding() + .encodeToString(value.getBytes(StandardCharsets.UTF_8)); + } + + private static String decodeNullable(String value) { + if (value == null || value.isEmpty()) { + return null; + } + return new String(Base64.getUrlDecoder().decode(value), StandardCharsets.UTF_8); + } + + private static String encodeStringSet(Set values) { + if (values == null || values.isEmpty()) { + return ""; + } + return encodeNullable(String.join("\n", values)); + } + + private static Set decodeStringSet(String encoded) { + String decoded = decodeNullable(encoded); + if (decoded == null || decoded.isEmpty()) { + return Collections.emptySet(); + } + Set values = new LinkedHashSet<>(); + Collections.addAll(values, decoded.split("\n")); + values.remove(""); + return values; + } + + private boolean isWithinRange(String logicalKey) { + if (logicalKey == null) { + return false; + } + if (keyRangeStart != null && logicalKey.compareTo(keyRangeStart) < 0) { + return false; + } + return keyRangeEnd == null || logicalKey.compareTo(keyRangeEnd) <= 0; + } + } + + private static String encodeLogicalCellStatic( + String deviceId, String measurement, long time, TSDataType dataType, Object value) { + return deviceId + + '|' + + measurement + + '|' + + time + + '|' + + dataType.name() + + '|' + + valueHashStatic(value); + } + + private static String encodeDeletionKeyStatic(TConsistencyDeletionSummary summary) { + return summary.getPathPattern() + + '|' + + summary.getTimeRangeStart() + + '|' + + summary.getTimeRangeEnd(); + } + + private static int valueHashStatic(Object value) { + if (value == null) { + return 0; + } + if (value instanceof Binary) { + return value.toString().hashCode(); + } + return value.hashCode(); + } +} diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/consistency/DataRegionConsistencyRepairService.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/consistency/DataRegionConsistencyRepairService.java index 3964d46486161..03754c787eda9 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/consistency/DataRegionConsistencyRepairService.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/consistency/DataRegionConsistencyRepairService.java @@ -20,33 +20,40 @@ package org.apache.iotdb.db.storageengine.dataregion.consistency; import org.apache.iotdb.common.rpc.thrift.TConsensusGroupId; -import org.apache.iotdb.common.rpc.thrift.TDataNodeLocation; -import org.apache.iotdb.common.rpc.thrift.TEndPoint; import org.apache.iotdb.common.rpc.thrift.TSStatus; -import org.apache.iotdb.common.rpc.thrift.TTimePartitionSlot; -import org.apache.iotdb.commons.client.ClientPoolFactory; -import org.apache.iotdb.commons.client.IClientManager; -import org.apache.iotdb.commons.client.sync.SyncDataNodeInternalServiceClient; import org.apache.iotdb.commons.consensus.ConsensusGroupId; import org.apache.iotdb.commons.consensus.DataRegionId; import org.apache.iotdb.commons.consensus.index.ProgressIndex; -import org.apache.iotdb.db.conf.IoTDBDescriptor; -import org.apache.iotdb.db.exception.load.LoadFileException; -import org.apache.iotdb.db.queryengine.plan.planner.plan.node.PlanNodeId; -import org.apache.iotdb.db.queryengine.plan.planner.plan.node.load.LoadTsFilePieceNode; -import org.apache.iotdb.db.queryengine.plan.scheduler.load.LoadTsFileScheduler; +import org.apache.iotdb.commons.path.MeasurementPath; +import org.apache.iotdb.consensus.pipe.metric.IoTConsensusV2SyncLagManager; +import org.apache.iotdb.db.pipe.consensus.deletion.DeletionResource; +import org.apache.iotdb.db.pipe.consensus.deletion.DeletionResourceManager; +import org.apache.iotdb.db.queryengine.plan.planner.plan.node.PlanNode; +import org.apache.iotdb.db.queryengine.plan.planner.plan.node.PlanNodeType; +import org.apache.iotdb.db.queryengine.plan.planner.plan.node.write.AbstractDeleteDataNode; +import org.apache.iotdb.db.queryengine.plan.planner.plan.node.write.DeleteDataNode; +import org.apache.iotdb.db.queryengine.plan.planner.plan.node.write.InsertRowsNode; import org.apache.iotdb.db.storageengine.StorageEngine; import org.apache.iotdb.db.storageengine.dataregion.DataRegion; -import org.apache.iotdb.db.storageengine.dataregion.tsfile.TsFileResource; -import org.apache.iotdb.db.storageengine.load.splitter.TsFileSplitter; -import org.apache.iotdb.mpp.rpc.thrift.TConsistencyMerkleFile; -import org.apache.iotdb.mpp.rpc.thrift.TDataRegionConsistencySnapshotReq; -import org.apache.iotdb.mpp.rpc.thrift.TDataRegionConsistencySnapshotResp; -import org.apache.iotdb.mpp.rpc.thrift.TLoadCommandReq; -import org.apache.iotdb.mpp.rpc.thrift.TLoadResp; -import org.apache.iotdb.mpp.rpc.thrift.TRepairTransferTsFileReq; -import org.apache.iotdb.mpp.rpc.thrift.TTimePartitionConsistencyView; -import org.apache.iotdb.mpp.rpc.thrift.TTsFilePieceReq; +import org.apache.iotdb.mpp.rpc.thrift.TApplyLogicalRepairBatchReq; +import org.apache.iotdb.mpp.rpc.thrift.TConsistencyDeletionSummary; +import org.apache.iotdb.mpp.rpc.thrift.TDecodeLeafDiffReq; +import org.apache.iotdb.mpp.rpc.thrift.TDecodeLeafDiffResp; +import org.apache.iotdb.mpp.rpc.thrift.TEstimateLeafDiffReq; +import org.apache.iotdb.mpp.rpc.thrift.TEstimateLeafDiffResp; +import org.apache.iotdb.mpp.rpc.thrift.TFinishLogicalRepairSessionReq; +import org.apache.iotdb.mpp.rpc.thrift.TGetConsistencyEligibilityReq; +import org.apache.iotdb.mpp.rpc.thrift.TGetConsistencyEligibilityResp; +import org.apache.iotdb.mpp.rpc.thrift.TGetSnapshotSubtreeReq; +import org.apache.iotdb.mpp.rpc.thrift.TGetSnapshotSubtreeResp; +import org.apache.iotdb.mpp.rpc.thrift.TLeafDiffEntry; +import org.apache.iotdb.mpp.rpc.thrift.TLeafDiffEstimate; +import org.apache.iotdb.mpp.rpc.thrift.TLogicalRepairBatch; +import org.apache.iotdb.mpp.rpc.thrift.TLogicalRepairLeafSelector; +import org.apache.iotdb.mpp.rpc.thrift.TPartitionConsistencyEligibility; +import org.apache.iotdb.mpp.rpc.thrift.TSnapshotSubtreeNode; +import org.apache.iotdb.mpp.rpc.thrift.TStreamLogicalRepairReq; +import org.apache.iotdb.mpp.rpc.thrift.TStreamLogicalRepairResp; import org.apache.iotdb.rpc.RpcUtils; import org.apache.iotdb.rpc.TSStatusCode; @@ -55,302 +62,515 @@ import org.slf4j.LoggerFactory; import java.io.DataOutputStream; -import java.io.File; import java.io.IOException; import java.nio.ByteBuffer; import java.util.ArrayList; +import java.util.Collections; import java.util.Comparator; -import java.util.HashMap; import java.util.List; -import java.util.Map; -import java.util.UUID; +import java.util.stream.Collectors; -/** DataNode-side snapshot and direct TsFile repair primitives for replica consistency repair. */ +/** DataNode-side logical snapshot and logical repair primitives for replica consistency repair. */ public class DataRegionConsistencyRepairService { private static final Logger LOGGER = LoggerFactory.getLogger(DataRegionConsistencyRepairService.class); - private static final long MAX_PIECE_NODE_SIZE = - IoTDBDescriptor.getInstance().getConfig().getThriftMaxFrameSize() >> 2; - private final StorageEngine storageEngine = StorageEngine.getInstance(); + private final DataRegionConsistencyManager consistencyManager = + DataRegionConsistencyManager.getInstance(); + private final LogicalRepairSessionJournal repairSessionJournal = + new LogicalRepairSessionJournal(); - private final IClientManager clientManager = - new IClientManager.Factory() - .createClientManager( - new ClientPoolFactory.SyncDataNodeInternalServiceClientPoolFactory()); - - public TDataRegionConsistencySnapshotResp getSnapshot(TDataRegionConsistencySnapshotReq req) { + public TGetConsistencyEligibilityResp getConsistencyEligibility( + TGetConsistencyEligibilityReq req) { DataRegion dataRegion = getDataRegion(req.getConsensusGroupId()); if (dataRegion == null) { - return new TDataRegionConsistencySnapshotResp( + return new TGetConsistencyEligibilityResp( RpcUtils.getStatus( TSStatusCode.DATAREGION_PROCESS_ERROR, - "DataRegion " + req.getConsensusGroupId() + " is not found on this DataNode")); + "DataRegion " + req.getConsensusGroupId() + " is not found on this DataNode"), + Long.MAX_VALUE, + Long.MIN_VALUE); } - List partitionViews = new ArrayList<>(); - List timePartitions = new ArrayList<>(dataRegion.getTimePartitions()); - timePartitions.sort(Long::compareTo); - try { + long syncLag = + IoTConsensusV2SyncLagManager.getInstance( + normalizeConsensusGroupIdString(req.getConsensusGroupId())) + .calculateSyncLag(); + long safeWatermark = + dataRegion.getDelayAnalyzer() == null + ? Long.MAX_VALUE + : dataRegion.getDelayAnalyzer().getSafeWatermark(System.currentTimeMillis()); + List regionDeletionSummaries = + collectDeletionSummaries(req.getConsensusGroupId()); + List timePartitions = new ArrayList<>(dataRegion.getTimePartitions()); + augmentTimePartitionsWithDeletionRanges(timePartitions, regionDeletionSummaries); + timePartitions.sort(Long::compareTo); + + List partitions = new ArrayList<>(); for (Long timePartition : timePartitions) { - List merkleFiles = collectMerkleFiles(dataRegion, timePartition); - if (!merkleFiles.isEmpty()) { - partitionViews.add(new TTimePartitionConsistencyView(timePartition, merkleFiles)); - } + // Eligibility must expose follower partitions even when the follower-local DelayAnalyzer + // has + // not warmed up yet. ConfigNode applies cold-partition pruning from the leader view; if we + // filter here on every replica, a follower can disappear from the compare set and keep the + // partition stuck in DIRTY forever despite the leader already being safe. + List partitionDeletionSummaries = + filterDeletionSummariesForPartition(regionDeletionSummaries, timePartition); + DataRegionConsistencyManager.PartitionInspection inspection = + consistencyManager.inspectPartition( + req.getConsensusGroupId(), dataRegion, timePartition, partitionDeletionSummaries); + partitions.add( + new TPartitionConsistencyEligibility( + inspection.getPartitionId(), + inspection.getPartitionMutationEpoch(), + inspection.getSnapshotEpoch(), + inspection.getSnapshotState().name(), + inspection.getLiveRootDigest().getXorHash(), + inspection.getLiveRootDigest().getAdditiveHash(), + inspection.getTombstoneRootDigest().getXorHash(), + inspection.getTombstoneRootDigest().getAdditiveHash())); } - return new TDataRegionConsistencySnapshotResp(RpcUtils.SUCCESS_STATUS) - .setTimePartitionViews(partitionViews); + + return new TGetConsistencyEligibilityResp(RpcUtils.SUCCESS_STATUS, syncLag, safeWatermark) + .setPartitions(partitions); } catch (Exception e) { LOGGER.warn( - "Failed to build consistency snapshot for region {}", req.getConsensusGroupId(), e); - return new TDataRegionConsistencySnapshotResp( - RpcUtils.getStatus(TSStatusCode.EXECUTE_STATEMENT_ERROR, e.getMessage())); + "Failed to build consistency eligibility for region {}", req.getConsensusGroupId(), e); + return new TGetConsistencyEligibilityResp( + RpcUtils.getStatus(TSStatusCode.EXECUTE_STATEMENT_ERROR, e.getMessage()), + Long.MAX_VALUE, + Long.MIN_VALUE); } } - public TSStatus repairTransferTsFile(TRepairTransferTsFileReq req) { + public TGetSnapshotSubtreeResp getSnapshotSubtree(TGetSnapshotSubtreeReq req) { DataRegion dataRegion = getDataRegion(req.getConsensusGroupId()); if (dataRegion == null) { - return RpcUtils.getStatus( - TSStatusCode.DATAREGION_PROCESS_ERROR, - "DataRegion " + req.getConsensusGroupId() + " is not found on this DataNode"); + return new TGetSnapshotSubtreeResp( + RpcUtils.getStatus( + TSStatusCode.DATAREGION_PROCESS_ERROR, + "DataRegion " + req.getConsensusGroupId() + " is not found on this DataNode"), + req.getTimePartitionId(), + req.getSnapshotEpoch()) + .setStale(true); } - if (req.getTargetDataNodes() == null || req.getTargetDataNodes().isEmpty()) { - return RpcUtils.getStatus( - TSStatusCode.ILLEGAL_PARAMETER, "Repair transfer requires at least one target DataNode"); + try { + List partitionDeletionSummaries = + filterDeletionSummariesForPartition( + collectDeletionSummaries(req.getConsensusGroupId()), req.getTimePartitionId()); + DataRegionConsistencyManager.SnapshotSubtreeResult subtreeResult = + consistencyManager.getSnapshotSubtree( + req.getConsensusGroupId(), + dataRegion, + req.getTimePartitionId(), + req.getSnapshotEpoch(), + req.getTreeKind(), + req.getNodeHandles(), + partitionDeletionSummaries); + List nodes = + subtreeResult.getNodes().stream() + .map( + node -> + new TSnapshotSubtreeNode( + node.getParentNodeHandle(), + node.getNodeHandle(), + req.getTreeKind(), + node.getDepth(), + node.isLeaf(), + node.getDigest().getXorHash(), + node.getDigest().getAdditiveHash(), + node.getItemCount()) + .setLeafId(node.getLeafId()) + .setKeyRangeStart(node.getKeyRangeStart()) + .setKeyRangeEnd(node.getKeyRangeEnd())) + .collect(Collectors.toList()); + return new TGetSnapshotSubtreeResp( + RpcUtils.SUCCESS_STATUS, req.getTimePartitionId(), subtreeResult.getSnapshotEpoch()) + .setStale(subtreeResult.isStale()) + .setNodes(nodes); + } catch (Exception e) { + LOGGER.warn( + "Failed to build snapshot subtree for region {} partition {}", + req.getConsensusGroupId(), + req.getTimePartitionId(), + e); + return new TGetSnapshotSubtreeResp( + RpcUtils.getStatus(TSStatusCode.EXECUTE_STATEMENT_ERROR, e.getMessage()), + req.getTimePartitionId(), + req.getSnapshotEpoch()) + .setStale(true); } + } - TsFileResource tsFileResource = findTsFileResource(dataRegion, req.getSourceTsFilePath()); - if (tsFileResource == null) { - return RpcUtils.getStatus( - TSStatusCode.LOAD_FILE_ERROR, - "Cannot find sealed TsFile " + req.getSourceTsFilePath() + " on leader"); - } - if (!tsFileResource.isClosed() - || tsFileResource.isDeleted() - || !tsFileResource.getTsFile().exists()) { - return RpcUtils.getStatus( - TSStatusCode.LOAD_FILE_ERROR, - "TsFile " + req.getSourceTsFilePath() + " is not available for repair transfer"); + public TEstimateLeafDiffResp estimateLeafDiff(TEstimateLeafDiffReq req) { + DataRegion dataRegion = getDataRegion(req.getConsensusGroupId()); + if (dataRegion == null) { + return new TEstimateLeafDiffResp( + RpcUtils.getStatus( + TSStatusCode.DATAREGION_PROCESS_ERROR, + "DataRegion " + req.getConsensusGroupId() + " is not found on this DataNode"), + req.getTimePartitionId(), + req.getSnapshotEpoch()) + .setStale(true); } - if (tsFileResource.isSpanMultiTimePartitions()) { - return RpcUtils.getStatus( - TSStatusCode.UNSUPPORTED_OPERATION, - "Replica consistency repair does not support multi-time-partition TsFiles yet"); + try { + List partitionDeletionSummaries = + filterDeletionSummariesForPartition( + collectDeletionSummaries(req.getConsensusGroupId()), req.getTimePartitionId()); + DataRegionConsistencyManager.LeafEstimate estimate = + consistencyManager.estimateLeaf( + req.getConsensusGroupId(), + dataRegion, + req.getTimePartitionId(), + req.getSnapshotEpoch(), + req.getTreeKind(), + req.getLeafId(), + partitionDeletionSummaries); + if (estimate == null) { + return new TEstimateLeafDiffResp( + RpcUtils.SUCCESS_STATUS, req.getTimePartitionId(), req.getSnapshotEpoch()) + .setStale(true); + } + return new TEstimateLeafDiffResp( + RpcUtils.SUCCESS_STATUS, req.getTimePartitionId(), estimate.getSnapshotEpoch()) + .setLeafDiff( + new TLeafDiffEstimate( + estimate.getPartitionId(), + estimate.getSnapshotEpoch(), + estimate.getTreeKind(), + estimate.getLeafId(), + estimate.getRowCount(), + estimate.getTombstoneCount(), + estimate.getStrataEstimate()) + .setKeyRangeStart(estimate.getKeyRangeStart()) + .setKeyRangeEnd(estimate.getKeyRangeEnd())) + .setStale(false); + } catch (Exception e) { + LOGGER.warn( + "Failed to estimate leaf diff for region {} partition {} leaf {}", + req.getConsensusGroupId(), + req.getTimePartitionId(), + req.getLeafId(), + e); + return new TEstimateLeafDiffResp( + RpcUtils.getStatus(TSStatusCode.EXECUTE_STATEMENT_ERROR, e.getMessage()), + req.getTimePartitionId(), + req.getSnapshotEpoch()) + .setStale(true); } + } - for (TDataNodeLocation targetDataNode : req.getTargetDataNodes()) { - TSStatus status = - transferOneTarget(req.getConsensusGroupId(), tsFileResource, targetDataNode); - if (status.getCode() != TSStatusCode.SUCCESS_STATUS.getStatusCode()) { - return status; + public TDecodeLeafDiffResp decodeLeafDiff(TDecodeLeafDiffReq req) { + DataRegion dataRegion = getDataRegion(req.getConsensusGroupId()); + if (dataRegion == null) { + return new TDecodeLeafDiffResp( + RpcUtils.getStatus( + TSStatusCode.DATAREGION_PROCESS_ERROR, + "DataRegion " + req.getConsensusGroupId() + " is not found on this DataNode"), + req.getTimePartitionId(), + req.getSnapshotEpoch()) + .setStale(true); + } + try { + List partitionDeletionSummaries = + filterDeletionSummariesForPartition( + collectDeletionSummaries(req.getConsensusGroupId()), req.getTimePartitionId()); + List entries = + consistencyManager.decodeLeaf( + req.getConsensusGroupId(), + dataRegion, + req.getTimePartitionId(), + req.getSnapshotEpoch(), + req.getTreeKind(), + req.getLeafId(), + partitionDeletionSummaries); + if (entries == null) { + return new TDecodeLeafDiffResp( + RpcUtils.SUCCESS_STATUS, req.getTimePartitionId(), req.getSnapshotEpoch()) + .setStale(true); } + return new TDecodeLeafDiffResp( + RpcUtils.SUCCESS_STATUS, req.getTimePartitionId(), req.getSnapshotEpoch()) + .setStale(false) + .setDiffEntries( + entries.stream() + .map(entry -> new TLeafDiffEntry(entry.getLogicalKey(), entry.getDiffType())) + .collect(Collectors.toList())); + } catch (Exception e) { + LOGGER.warn( + "Failed to decode leaf diff for region {} partition {} leaf {}", + req.getConsensusGroupId(), + req.getTimePartitionId(), + req.getLeafId(), + e); + return new TDecodeLeafDiffResp( + RpcUtils.getStatus(TSStatusCode.EXECUTE_STATEMENT_ERROR, e.getMessage()), + req.getTimePartitionId(), + req.getSnapshotEpoch()) + .setStale(true); } - return RpcUtils.SUCCESS_STATUS; } - private List collectMerkleFiles(DataRegion dataRegion, long timePartition) - throws IOException { - List merkleFiles = new ArrayList<>(); - for (boolean sequence : new boolean[] {true, false}) { - for (TsFileResource tsFileResource : - dataRegion.getTsFileManager().getTsFileListSnapshot(timePartition, sequence)) { - if (!tsFileResource.isClosed() - || tsFileResource.isDeleted() - || !tsFileResource.getTsFile().exists()) { - continue; - } - - String tsFilePath = tsFileResource.getTsFilePath(); - List entries = - MerkleHashComputer.computeEntries(tsFilePath); - merkleFiles.add( - new TConsistencyMerkleFile( - tsFilePath, - tsFileResource.getTsFileSize(), - org.apache.iotdb.commons.consensus.iotv2.consistency.merkle.MerkleFileWriter - .computeFileXorHash(entries), - org.apache.iotdb.commons.consensus.iotv2.consistency.merkle.MerkleFileWriter - .computeFileAddHash(entries))); + public TStreamLogicalRepairResp streamLogicalRepair(TStreamLogicalRepairReq req) { + DataRegion dataRegion = getDataRegion(req.getConsensusGroupId()); + if (dataRegion == null) { + return new TStreamLogicalRepairResp( + RpcUtils.getStatus( + TSStatusCode.DATAREGION_PROCESS_ERROR, + "DataRegion " + req.getConsensusGroupId() + " is not found on this DataNode"), + req.getTimePartitionId()) + .setStale(true); + } + try { + List partitionDeletionSummaries = + filterDeletionSummariesForPartition( + collectDeletionSummaries(req.getConsensusGroupId()), req.getTimePartitionId()); + List leafSelectors = new ArrayList<>(); + for (TLogicalRepairLeafSelector leafSelector : req.getLeafSelectors()) { + leafSelectors.add( + new DataRegionConsistencyManager.LeafSelector( + leafSelector.getTreeKind(), leafSelector.getLeafId())); } + List batches = + consistencyManager.streamLogicalRepair( + req.getConsensusGroupId(), + dataRegion, + req.getTimePartitionId(), + req.getRepairEpoch(), + leafSelectors, + partitionDeletionSummaries); + List thriftBatches = + batches.stream() + .map( + batch -> + new TLogicalRepairBatch( + batch.getSessionId(), + batch.getTreeKind(), + batch.getLeafId(), + batch.getSeqNo(), + batch.getBatchKind(), + batch.getPayload())) + .collect(Collectors.toList()); + return new TStreamLogicalRepairResp(RpcUtils.SUCCESS_STATUS, req.getTimePartitionId()) + .setStale(false) + .setBatches(thriftBatches); + } catch (Exception e) { + LOGGER.warn( + "Failed to stream logical repair for region {} partition {}", + req.getConsensusGroupId(), + req.getTimePartitionId(), + e); + return new TStreamLogicalRepairResp( + RpcUtils.getStatus(TSStatusCode.EXECUTE_STATEMENT_ERROR, e.getMessage()), + req.getTimePartitionId()) + .setStale(true); } - - merkleFiles.sort(Comparator.comparing(TConsistencyMerkleFile::getSourceTsFilePath)); - return merkleFiles; } - private TSStatus transferOneTarget( - TConsensusGroupId consensusGroupId, - TsFileResource tsFileResource, - TDataNodeLocation targetDataNode) { - String uuid = UUID.randomUUID().toString(); - TransferTracker tracker = new TransferTracker(); - + public TSStatus applyLogicalRepairBatch(TApplyLogicalRepairBatchReq req) { + DataRegion dataRegion = getDataRegion(req.getConsensusGroupId()); + if (dataRegion == null) { + return RpcUtils.getStatus( + TSStatusCode.DATAREGION_PROCESS_ERROR, + "DataRegion " + req.getConsensusGroupId() + " is not found on this DataNode"); + } try { - sendAllPieces( - tsFileResource.getTsFile(), - uuid, - consensusGroupId, - targetDataNode.getInternalEndPoint(), - tracker); - TSStatus secondPhaseStatus = - sendLoadCommand( - targetDataNode.getInternalEndPoint(), - buildLoadCommandReq(uuid, tsFileResource, LoadTsFileScheduler.LoadCommand.EXECUTE)); - if (secondPhaseStatus.getCode() != TSStatusCode.SUCCESS_STATUS.getStatusCode()) { - return secondPhaseStatus; - } + repairSessionJournal.stageBatch( + normalizeConsensusGroupIdString(req.getConsensusGroupId()), + req.getTimePartitionId(), + req.getRepairEpoch(), + req.getSessionId(), + req.getTreeKind(), + req.getLeafId(), + req.getSeqNo(), + req.getBatchKind(), + req.bufferForPayload()); return RpcUtils.SUCCESS_STATUS; } catch (Exception e) { LOGGER.warn( - "Failed to transfer TsFile {} to follower {}", - tsFileResource.getTsFilePath(), - targetDataNode.getDataNodeId(), + "Failed to stage logical repair batch for region {} partition {} session {} seq {}", + req.getConsensusGroupId(), + req.getTimePartitionId(), + req.getSessionId(), + req.getSeqNo(), e); - if (tracker.hasSentPieces) { - try { - TSStatus rollbackStatus = - sendLoadCommand( - targetDataNode.getInternalEndPoint(), - buildLoadCommandReq( - uuid, tsFileResource, LoadTsFileScheduler.LoadCommand.ROLLBACK)); - if (rollbackStatus.getCode() != TSStatusCode.SUCCESS_STATUS.getStatusCode()) { - rollbackStatus.setMessage( - rollbackStatus.getMessage() + ", original transfer failure: " + e.getMessage()); - return rollbackStatus; - } - } catch (IOException rollbackBuildException) { - return RpcUtils.getStatus( - TSStatusCode.LOAD_FILE_ERROR, - "Failed to build rollback command for TsFile " - + tsFileResource.getTsFilePath() - + ": " - + rollbackBuildException.getMessage() - + ", original transfer failure: " - + e.getMessage()); - } - } - return RpcUtils.getStatus( - TSStatusCode.LOAD_FILE_ERROR, - "Failed to transfer TsFile " - + tsFileResource.getTsFilePath() - + " to follower " - + targetDataNode.getDataNodeId() - + ": " - + e.getMessage()); + return RpcUtils.getStatus(TSStatusCode.EXECUTE_STATEMENT_ERROR, e.getMessage()); } } - private void sendAllPieces( - File tsFile, - String uuid, - TConsensusGroupId consensusGroupId, - TEndPoint targetEndPoint, - TransferTracker tracker) - throws IOException, LoadFileException { - final LoadTsFilePieceNode[] pieceHolder = { - new LoadTsFilePieceNode(new PlanNodeId("repair-tsfile-piece"), tsFile) - }; - - new TsFileSplitter( - tsFile, - tsFileData -> { - pieceHolder[0].addTsFileData(tsFileData); - if (pieceHolder[0].getDataSize() >= MAX_PIECE_NODE_SIZE) { - dispatchPieceNode(targetEndPoint, uuid, consensusGroupId, pieceHolder[0]); - tracker.hasSentPieces = true; - pieceHolder[0] = - new LoadTsFilePieceNode(new PlanNodeId("repair-tsfile-piece"), tsFile); - } - return true; - }) - .splitTsFileByDataPartition(); - - if (pieceHolder[0].getDataSize() > 0) { - dispatchPieceNode(targetEndPoint, uuid, consensusGroupId, pieceHolder[0]); - tracker.hasSentPieces = true; + public TSStatus finishLogicalRepairSession(TFinishLogicalRepairSessionReq req) { + DataRegion dataRegion = getDataRegion(req.getConsensusGroupId()); + if (dataRegion == null) { + return RpcUtils.getStatus( + TSStatusCode.DATAREGION_PROCESS_ERROR, + "DataRegion " + req.getConsensusGroupId() + " is not found on this DataNode"); + } + try { + List stagedBatches = + repairSessionJournal.loadStagedBatches( + normalizeConsensusGroupIdString(req.getConsensusGroupId()), + req.getTimePartitionId(), + req.getRepairEpoch(), + req.getSessionId()); + if (stagedBatches.isEmpty()) { + repairSessionJournal.completeSession(req.getSessionId()); + return RpcUtils.SUCCESS_STATUS; + } + consistencyManager.runWithLogicalRepairMutation( + req.getConsensusGroupId(), + req.getTimePartitionId(), + req.getRepairEpoch(), + () -> { + for (LogicalRepairSessionJournal.StagedBatch stagedBatch : stagedBatches) { + applyStagedBatch(dataRegion, req.getTimePartitionId(), stagedBatch); + } + return null; + }); + repairSessionJournal.completeSession(req.getSessionId()); + return RpcUtils.SUCCESS_STATUS; + } catch (Exception e) { + LOGGER.warn( + "Failed to finish logical repair session for region {} partition {} session {}", + req.getConsensusGroupId(), + req.getTimePartitionId(), + req.getSessionId(), + e); + return RpcUtils.getStatus(TSStatusCode.EXECUTE_STATEMENT_ERROR, e.getMessage()); } } - private void dispatchPieceNode( - TEndPoint targetEndPoint, - String uuid, - TConsensusGroupId consensusGroupId, - LoadTsFilePieceNode pieceNode) - throws LoadFileException { - TTsFilePieceReq request = - new TTsFilePieceReq(pieceNode.serializeToByteBuffer(), uuid, consensusGroupId); - try (SyncDataNodeInternalServiceClient client = clientManager.borrowClient(targetEndPoint)) { - TLoadResp response = client.sendTsFilePieceNode(request); - if (!response.isAccepted()) { - throw new LoadFileException( - response.isSetStatus() ? response.getStatus().getMessage() : response.getMessage()); + private void applyStagedBatch( + DataRegion dataRegion, + long timePartitionId, + LogicalRepairSessionJournal.StagedBatch stagedBatch) + throws Exception { + if ("RESET_SCOPE".equals(stagedBatch.getBatchKind())) { + consistencyManager.resetLiveScope( + dataRegion, timePartitionId, stagedBatch.duplicatePayload()); + return; + } + if ("RESET_LEAF".equals(stagedBatch.getBatchKind())) { + consistencyManager.resetLiveLeaf(dataRegion, timePartitionId, stagedBatch.getLeafId()); + return; + } + PlanNode planNode = PlanNodeType.deserialize(stagedBatch.duplicatePayload()); + if (planNode instanceof InsertRowsNode) { + dataRegion.insert((InsertRowsNode) planNode); + return; + } + if (planNode instanceof DeleteDataNode) { + DeleteDataNode deleteDataNode = (DeleteDataNode) planNode; + for (MeasurementPath path : deleteDataNode.getPathList()) { + dataRegion.deleteByDevice(path, deleteDataNode); } - } catch (LoadFileException e) { - throw e; - } catch (Exception e) { - throw new LoadFileException( - "Failed to dispatch TsFile piece to DataNode " + targetEndPoint, e); + return; } + throw new UnsupportedOperationException( + "Unsupported logical repair batch payload: " + planNode.getClass().getSimpleName()); } - private TSStatus sendLoadCommand(TEndPoint targetEndPoint, TLoadCommandReq request) { - try (SyncDataNodeInternalServiceClient client = clientManager.borrowClient(targetEndPoint)) { - TLoadResp response = client.sendLoadCommand(request); - if (response.isAccepted()) { - return RpcUtils.SUCCESS_STATUS; + private List collectDeletionSummaries( + TConsensusGroupId consensusGroupId) throws IOException { + DeletionResourceManager deletionResourceManager = + DeletionResourceManager.getInstance(consensusGroupId.getId()); + if (deletionResourceManager == null) { + return Collections.emptyList(); + } + + List deletionSummaries = new ArrayList<>(); + for (DeletionResource deletionResource : deletionResourceManager.getAllDeletionResources()) { + AbstractDeleteDataNode deleteDataNode = deletionResource.getDeleteDataNode(); + if (!(deleteDataNode instanceof DeleteDataNode)) { + continue; } - if (response.isSetStatus()) { - return response.getStatus(); + DeleteDataNode treeDeleteNode = (DeleteDataNode) deleteDataNode; + ByteBuffer serializedProgressIndex = + serializeProgressIndex(treeDeleteNode.getProgressIndex()); + for (MeasurementPath path : treeDeleteNode.getPathList()) { + deletionSummaries.add( + new TConsistencyDeletionSummary( + path.getFullPath(), + treeDeleteNode.getDeleteStartTime(), + treeDeleteNode.getDeleteEndTime(), + serializedProgressIndex.duplicate())); } - return RpcUtils.getStatus(TSStatusCode.LOAD_FILE_ERROR, response.getMessage()); - } catch (Exception e) { - return RpcUtils.getStatus(TSStatusCode.LOAD_FILE_ERROR, e.getMessage()); } + deletionSummaries.sort( + Comparator.comparing(TConsistencyDeletionSummary::getPathPattern) + .thenComparingLong(TConsistencyDeletionSummary::getTimeRangeStart) + .thenComparingLong(TConsistencyDeletionSummary::getTimeRangeEnd)); + return deletionSummaries; } - private TLoadCommandReq buildLoadCommandReq( - String uuid, TsFileResource tsFileResource, LoadTsFileScheduler.LoadCommand loadCommand) - throws IOException { - TLoadCommandReq request = new TLoadCommandReq(loadCommand.ordinal(), uuid); - Map timePartition2ProgressIndex = new HashMap<>(); - timePartition2ProgressIndex.put( - new TTimePartitionSlot(tsFileResource.getTimePartition()), - serializeProgressIndex(tsFileResource.getMaxProgressIndex())); - request.setTimePartition2ProgressIndex(timePartition2ProgressIndex); - request.setIsGeneratedByPipe(false); - return request; - } + private List filterDeletionSummariesForPartition( + List regionDeletionSummaries, long timePartition) { + if (regionDeletionSummaries.isEmpty()) { + return Collections.emptyList(); + } - private ByteBuffer serializeProgressIndex(ProgressIndex progressIndex) throws IOException { - try (PublicBAOS byteArrayOutputStream = new PublicBAOS(); - DataOutputStream dataOutputStream = new DataOutputStream(byteArrayOutputStream)) { - progressIndex.serialize(dataOutputStream); - return ByteBuffer.wrap(byteArrayOutputStream.getBuf(), 0, byteArrayOutputStream.size()); + long timePartitionInterval = + org.apache.iotdb.commons.conf.CommonDescriptor.getInstance() + .getConfig() + .getTimePartitionInterval(); + if (timePartitionInterval <= 0) { + return new ArrayList<>(regionDeletionSummaries); + } + + long partitionStart = timePartition * timePartitionInterval; + long partitionEnd = + Long.MAX_VALUE - timePartitionInterval < partitionStart + ? Long.MAX_VALUE + : partitionStart + timePartitionInterval - 1; + + List filtered = new ArrayList<>(); + for (TConsistencyDeletionSummary summary : regionDeletionSummaries) { + if (summary.getTimeRangeEnd() >= partitionStart + && summary.getTimeRangeStart() <= partitionEnd) { + filtered.add(summary); + } } + return filtered; } - private TsFileResource findTsFileResource(DataRegion dataRegion, String sourceTsFilePath) { - List timePartitions = new ArrayList<>(dataRegion.getTimePartitions()); - timePartitions.sort(Long::compareTo); + private void augmentTimePartitionsWithDeletionRanges( + List timePartitions, List regionDeletionSummaries) { + long timePartitionInterval = + org.apache.iotdb.commons.conf.CommonDescriptor.getInstance() + .getConfig() + .getTimePartitionInterval(); + if (timePartitionInterval <= 0 || regionDeletionSummaries.isEmpty()) { + return; + } - for (Long timePartition : timePartitions) { - for (boolean sequence : new boolean[] {true, false}) { - for (TsFileResource tsFileResource : - dataRegion.getTsFileManager().getTsFileListSnapshot(timePartition, sequence)) { - if (sourceTsFilePath.equals(tsFileResource.getTsFilePath())) { - return tsFileResource; - } + for (TConsistencyDeletionSummary summary : regionDeletionSummaries) { + long startPartition = Math.floorDiv(summary.getTimeRangeStart(), timePartitionInterval); + long endPartition = Math.floorDiv(summary.getTimeRangeEnd(), timePartitionInterval); + for (long partition = startPartition; partition <= endPartition; partition++) { + if (!timePartitions.contains(partition)) { + timePartitions.add(partition); } } } - return null; + } + + private ByteBuffer serializeProgressIndex(ProgressIndex progressIndex) throws IOException { + try (PublicBAOS byteArrayOutputStream = new PublicBAOS(); + DataOutputStream outputStream = new DataOutputStream(byteArrayOutputStream)) { + if (progressIndex == null) { + outputStream.writeByte(0); + } else { + progressIndex.serialize(outputStream); + } + return ByteBuffer.wrap(byteArrayOutputStream.getBuf(), 0, byteArrayOutputStream.size()); + } } private DataRegion getDataRegion(TConsensusGroupId consensusGroupId) { + if (consensusGroupId == null) { + return null; + } ConsensusGroupId groupId = ConsensusGroupId.Factory.createFromTConsensusGroupId(consensusGroupId); if (!(groupId instanceof DataRegionId)) { @@ -359,7 +579,7 @@ private DataRegion getDataRegion(TConsensusGroupId consensusGroupId) { return storageEngine.getDataRegion((DataRegionId) groupId); } - private static final class TransferTracker { - private boolean hasSentPieces; + private String normalizeConsensusGroupIdString(TConsensusGroupId consensusGroupId) { + return ConsensusGroupId.Factory.createFromTConsensusGroupId(consensusGroupId).toString(); } } diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/consistency/LogicalRepairSessionJournal.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/consistency/LogicalRepairSessionJournal.java new file mode 100644 index 0000000000000..ea281fb5955f7 --- /dev/null +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/consistency/LogicalRepairSessionJournal.java @@ -0,0 +1,308 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.iotdb.db.storageengine.dataregion.consistency; + +import org.apache.iotdb.db.conf.IoTDBDescriptor; + +import org.apache.tsfile.utils.ReadWriteIOUtils; + +import java.io.FileOutputStream; +import java.io.IOException; +import java.io.InputStream; +import java.io.OutputStream; +import java.nio.ByteBuffer; +import java.nio.file.AtomicMoveNotSupportedException; +import java.nio.file.Files; +import java.nio.file.Path; +import java.nio.file.Paths; +import java.nio.file.StandardCopyOption; +import java.nio.file.StandardOpenOption; +import java.util.ArrayList; +import java.util.Arrays; +import java.util.Collections; +import java.util.List; +import java.util.Map; +import java.util.Objects; +import java.util.TreeMap; +import java.util.concurrent.ConcurrentHashMap; + +/** Durable staging journal for follower-side logical repair sessions. */ +class LogicalRepairSessionJournal { + + private static final int FORMAT_VERSION = 1; + + private final Path journalDir; + private final ConcurrentHashMap sessions = new ConcurrentHashMap<>(); + + LogicalRepairSessionJournal() { + this( + Paths.get( + IoTDBDescriptor.getInstance().getConfig().getSystemDir(), + "consistency-repair", + "sessions")); + } + + LogicalRepairSessionJournal(Path journalDir) { + this.journalDir = journalDir; + } + + public synchronized void stageBatch( + String consensusGroupKey, + long partitionId, + String repairEpoch, + String sessionId, + String treeKind, + String leafId, + int seqNo, + String batchKind, + ByteBuffer payload) + throws IOException { + SessionState sessionState = + loadOrCreateSession(consensusGroupKey, partitionId, repairEpoch, sessionId); + if (sessionState.batches.containsKey(seqNo)) { + return; + } + sessionState.batches.put( + seqNo, + new StagedBatch(sessionId, treeKind, leafId, seqNo, batchKind, duplicatePayload(payload))); + persist(sessionState); + } + + public synchronized List loadStagedBatches( + String consensusGroupKey, long partitionId, String repairEpoch, String sessionId) + throws IOException { + SessionState sessionState = loadSession(sessionId); + if (sessionState == null) { + return Collections.emptyList(); + } + validateSession(sessionState, consensusGroupKey, partitionId, repairEpoch, sessionId); + return new ArrayList<>(sessionState.batches.values()); + } + + public synchronized void completeSession(String sessionId) throws IOException { + sessions.remove(sessionId); + Files.deleteIfExists(sessionPath(sessionId)); + } + + private SessionState loadOrCreateSession( + String consensusGroupKey, long partitionId, String repairEpoch, String sessionId) + throws IOException { + SessionState existing = loadSession(sessionId); + if (existing != null) { + validateSession(existing, consensusGroupKey, partitionId, repairEpoch, sessionId); + return existing; + } + + SessionState created = + new SessionState(consensusGroupKey, partitionId, repairEpoch, sessionId, new TreeMap<>()); + sessions.put(sessionId, created); + persist(created); + return created; + } + + private SessionState loadSession(String sessionId) throws IOException { + SessionState cached = sessions.get(sessionId); + if (cached != null) { + return cached; + } + + Path sessionPath = sessionPath(sessionId); + if (!Files.exists(sessionPath)) { + return null; + } + + try (InputStream inputStream = Files.newInputStream(sessionPath, StandardOpenOption.READ)) { + SessionState loaded = deserialize(inputStream); + sessions.put(sessionId, loaded); + return loaded; + } + } + + private void validateSession( + SessionState sessionState, + String consensusGroupKey, + long partitionId, + String repairEpoch, + String sessionId) { + if (!Objects.equals(sessionState.consensusGroupKey, consensusGroupKey) + || sessionState.partitionId != partitionId + || !Objects.equals(sessionState.repairEpoch, repairEpoch) + || !Objects.equals(sessionState.sessionId, sessionId)) { + throw new IllegalStateException( + "Logical repair session " + sessionId + " conflicts with current request metadata"); + } + } + + private void persist(SessionState sessionState) throws IOException { + Files.createDirectories(journalDir); + Path sessionPath = sessionPath(sessionState.sessionId); + Path tmpPath = sessionPath.resolveSibling(sessionPath.getFileName() + ".tmp"); + + try (FileOutputStream outputStream = new FileOutputStream(tmpPath.toFile())) { + serialize(sessionState, outputStream); + outputStream.getFD().sync(); + } + + try { + Files.move( + tmpPath, + sessionPath, + StandardCopyOption.ATOMIC_MOVE, + StandardCopyOption.REPLACE_EXISTING); + } catch (AtomicMoveNotSupportedException e) { + Files.move(tmpPath, sessionPath, StandardCopyOption.REPLACE_EXISTING); + } + } + + private Path sessionPath(String sessionId) { + return journalDir.resolve(sessionId + ".session"); + } + + private static byte[] duplicatePayload(ByteBuffer payload) { + if (payload == null) { + return new byte[0]; + } + ByteBuffer duplicate = payload.duplicate(); + byte[] bytes = new byte[duplicate.remaining()]; + duplicate.get(bytes); + return bytes; + } + + private static void serialize(SessionState sessionState, OutputStream outputStream) + throws IOException { + ReadWriteIOUtils.write(FORMAT_VERSION, outputStream); + ReadWriteIOUtils.write(sessionState.consensusGroupKey, outputStream); + ReadWriteIOUtils.write(sessionState.partitionId, outputStream); + ReadWriteIOUtils.write(sessionState.repairEpoch, outputStream); + ReadWriteIOUtils.write(sessionState.sessionId, outputStream); + ReadWriteIOUtils.write(sessionState.batches.size(), outputStream); + for (StagedBatch stagedBatch : sessionState.batches.values()) { + ReadWriteIOUtils.write(stagedBatch.sessionId, outputStream); + ReadWriteIOUtils.write(stagedBatch.treeKind, outputStream); + ReadWriteIOUtils.write(stagedBatch.leafId, outputStream); + ReadWriteIOUtils.write(stagedBatch.seqNo, outputStream); + ReadWriteIOUtils.write(stagedBatch.batchKind, outputStream); + ReadWriteIOUtils.write(stagedBatch.payload.length, outputStream); + outputStream.write(stagedBatch.payload); + } + } + + private static SessionState deserialize(InputStream inputStream) throws IOException { + int formatVersion = ReadWriteIOUtils.readInt(inputStream); + if (formatVersion != FORMAT_VERSION) { + throw new IOException("Unsupported logical repair session format " + formatVersion); + } + String consensusGroupKey = ReadWriteIOUtils.readString(inputStream); + long partitionId = ReadWriteIOUtils.readLong(inputStream); + String repairEpoch = ReadWriteIOUtils.readString(inputStream); + String sessionId = ReadWriteIOUtils.readString(inputStream); + int batchCount = ReadWriteIOUtils.readInt(inputStream); + Map batches = new TreeMap<>(); + for (int i = 0; i < batchCount; i++) { + String batchSessionId = ReadWriteIOUtils.readString(inputStream); + String treeKind = ReadWriteIOUtils.readString(inputStream); + String leafId = ReadWriteIOUtils.readString(inputStream); + int seqNo = ReadWriteIOUtils.readInt(inputStream); + String batchKind = ReadWriteIOUtils.readString(inputStream); + int payloadSize = ReadWriteIOUtils.readInt(inputStream); + byte[] payload = new byte[payloadSize]; + int offset = 0; + while (offset < payloadSize) { + int read = inputStream.read(payload, offset, payloadSize - offset); + if (read < 0) { + throw new IOException("Unexpected end of logical repair session journal"); + } + offset += read; + } + batches.put( + seqNo, new StagedBatch(batchSessionId, treeKind, leafId, seqNo, batchKind, payload)); + } + return new SessionState(consensusGroupKey, partitionId, repairEpoch, sessionId, batches); + } + + static final class StagedBatch { + private final String sessionId; + private final String treeKind; + private final String leafId; + private final int seqNo; + private final String batchKind; + private final byte[] payload; + + private StagedBatch( + String sessionId, + String treeKind, + String leafId, + int seqNo, + String batchKind, + byte[] payload) { + this.sessionId = sessionId; + this.treeKind = treeKind; + this.leafId = leafId; + this.seqNo = seqNo; + this.batchKind = batchKind; + this.payload = payload == null ? new byte[0] : Arrays.copyOf(payload, payload.length); + } + + String getSessionId() { + return sessionId; + } + + String getTreeKind() { + return treeKind; + } + + String getLeafId() { + return leafId; + } + + int getSeqNo() { + return seqNo; + } + + String getBatchKind() { + return batchKind; + } + + ByteBuffer duplicatePayload() { + return ByteBuffer.wrap(Arrays.copyOf(payload, payload.length)); + } + } + + private static final class SessionState { + private final String consensusGroupKey; + private final long partitionId; + private final String repairEpoch; + private final String sessionId; + private final Map batches; + + private SessionState( + String consensusGroupKey, + long partitionId, + String repairEpoch, + String sessionId, + Map batches) { + this.consensusGroupKey = consensusGroupKey; + this.partitionId = partitionId; + this.repairEpoch = repairEpoch; + this.sessionId = sessionId; + this.batches = batches; + } + } +} diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/consistency/MerkleCompactionHook.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/consistency/MerkleCompactionHook.java deleted file mode 100644 index 1a00532fd1ed9..0000000000000 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/consistency/MerkleCompactionHook.java +++ /dev/null @@ -1,162 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, - * software distributed under the License is distributed on an - * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY - * KIND, either express or implied. See the License for the - * specific language governing permissions and limitations - * under the License. - */ - -package org.apache.iotdb.db.storageengine.dataregion.consistency; - -import org.apache.iotdb.commons.consensus.iotv2.consistency.ConsistencyMerkleTree; -import org.apache.iotdb.commons.consensus.iotv2.consistency.DualDigest; -import org.apache.iotdb.commons.consensus.iotv2.consistency.merkle.MerkleEntry; -import org.apache.iotdb.commons.consensus.iotv2.consistency.merkle.MerkleFileCache; -import org.apache.iotdb.commons.consensus.iotv2.consistency.merkle.MerkleFileContent; -import org.apache.iotdb.commons.consensus.iotv2.consistency.merkle.MerkleFileWriter; -import org.apache.iotdb.db.storageengine.dataregion.tsfile.TsFileResource; - -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -import java.io.File; -import java.io.IOException; -import java.util.ArrayList; -import java.util.List; - -/** - * Hook into TsFileManager.replace() for compaction-aware Merkle tree updates. When source TsFiles - * are merged into target TsFiles during compaction, this hook: - * - *

    - *
  1. XOR-out each source file's root hash from the partition digest - *
  2. Scan target TsFiles and generate .merkle files - *
  3. XOR-in each target file's root hash - *
  4. Delete source .merkle files - *
  5. Invalidate cache entries for source files - *
- */ -public class MerkleCompactionHook { - - private static final Logger LOGGER = LoggerFactory.getLogger(MerkleCompactionHook.class); - - private final ConsistencyMerkleTree merkleTree; - private final MerkleFileCache merkleFileCache; - - public MerkleCompactionHook(ConsistencyMerkleTree merkleTree, MerkleFileCache merkleFileCache) { - this.merkleTree = merkleTree; - this.merkleFileCache = merkleFileCache; - } - - /** - * Called after compaction replaces source TsFiles with target TsFiles. - * - * @param seqSourceFiles source sequence TsFiles being removed - * @param unseqSourceFiles source unsequence TsFiles being removed - * @param targetFiles newly created target TsFiles - * @param timePartition the time partition being compacted - */ - public void onCompaction( - List seqSourceFiles, - List unseqSourceFiles, - List targetFiles, - long timePartition) { - try { - List sourceDigests = new ArrayList<>(); - boolean digestUpdatePossible = true; - - // Collect and XOR-out source file hashes - for (TsFileResource source : seqSourceFiles) { - DualDigest digest = getFileDigest(source); - if (digest == null) { - digestUpdatePossible = false; - } else { - sourceDigests.add(digest); - } - } - for (TsFileResource source : unseqSourceFiles) { - DualDigest digest = getFileDigest(source); - if (digest == null) { - digestUpdatePossible = false; - } else { - sourceDigests.add(digest); - } - } - - // Compute target file hashes and generate .merkle files - List targetDigests = new ArrayList<>(); - for (TsFileResource target : targetFiles) { - if (target.isDeleted()) { - continue; - } - String tsFilePath = target.getTsFilePath(); - List entries = MerkleHashComputer.computeEntries(tsFilePath); - if (entries.isEmpty()) { - continue; - } - - long fileXorHash = MerkleFileWriter.computeFileXorHash(entries); - long fileAddHash = MerkleFileWriter.computeFileAddHash(entries); - MerkleFileWriter.write(tsFilePath + ".merkle", entries, fileXorHash, fileAddHash); - targetDigests.add(new DualDigest(fileXorHash, fileAddHash)); - } - - if (digestUpdatePossible) { - merkleTree.onCompaction(sourceDigests, targetDigests, timePartition); - } else { - merkleTree.markPartitionDirty(timePartition); - } - - // Cleanup: delete source .merkle files and invalidate cache - cleanupSourceMerkleFiles(seqSourceFiles); - cleanupSourceMerkleFiles(unseqSourceFiles); - - LOGGER.debug( - "Compaction hook: updated partition {} Merkle tree, removed {} source digests", - timePartition, - sourceDigests.size()); - - } catch (IOException e) { - LOGGER.warn( - "Failed to update Merkle tree during compaction for partition {}: {}", - timePartition, - e.getMessage(), - e); - merkleTree.markPartitionDirty(timePartition); - } - } - - private DualDigest getFileDigest(TsFileResource source) { - String tsFilePath = source.getTsFilePath(); - try { - MerkleFileContent content = merkleFileCache.get(tsFilePath); - return content.getFileDigest(); - } catch (IOException e) { - // .merkle file might not exist (e.g., created before consistency module was enabled) - LOGGER.debug("No .merkle file for source TsFile {}: {}", tsFilePath, e.getMessage()); - return null; - } - } - - private void cleanupSourceMerkleFiles(List sourceFiles) { - for (TsFileResource source : sourceFiles) { - String tsFilePath = source.getTsFilePath(); - merkleFileCache.invalidate(tsFilePath); - File merkleFile = new File(tsFilePath + ".merkle"); - if (merkleFile.exists() && !merkleFile.delete()) { - LOGGER.warn("Failed to delete .merkle file: {}", merkleFile.getAbsolutePath()); - } - } - } -} diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/consistency/MerkleDeletionHook.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/consistency/MerkleDeletionHook.java deleted file mode 100644 index 16e49abbb865d..0000000000000 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/consistency/MerkleDeletionHook.java +++ /dev/null @@ -1,97 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, - * software distributed under the License is distributed on an - * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY - * KIND, either express or implied. See the License for the - * specific language governing permissions and limitations - * under the License. - */ - -package org.apache.iotdb.db.storageengine.dataregion.consistency; - -import org.apache.iotdb.commons.consensus.iotv2.consistency.ConsistencyMerkleTree; -import org.apache.iotdb.db.storageengine.dataregion.tsfile.TsFileResource; - -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -import java.util.List; -import java.util.Set; -import java.util.TreeSet; - -/** - * Hook into DataRegion's deletion path to mark affected time partitions as dirty in the - * ConsistencyMerkleTree. Rather than attempting to XOR-out exact deleted hashes (which requires - * expensive disk reads and has crash-consistency risks), we use lazy invalidation: mark the - * partition dirty and trigger a full rescan during the next consistency check cycle. - */ -public class MerkleDeletionHook { - - private static final Logger LOGGER = LoggerFactory.getLogger(MerkleDeletionHook.class); - - private final ConsistencyMerkleTree merkleTree; - - public MerkleDeletionHook(ConsistencyMerkleTree merkleTree) { - this.merkleTree = merkleTree; - } - - /** - * Called when a deletion (mod entry) affects one or more TsFiles. Marks the time partitions of - * all affected TsFiles as dirty. - * - * @param affectedTsFiles list of TsFileResource objects that overlap with the deletion - */ - public void onDeletion(List affectedTsFiles) { - Set affectedPartitions = new TreeSet<>(); - for (TsFileResource resource : affectedTsFiles) { - long partitionId = resource.getTimePartition(); - affectedPartitions.add(partitionId); - } - - for (long partitionId : affectedPartitions) { - merkleTree.markPartitionDirty(partitionId); - } - - LOGGER.debug( - "Deletion affected {} partitions, marked dirty: {}", - affectedPartitions.size(), - affectedPartitions); - } - - /** - * Called when a deletion affects a specific time range. Marks all tracked partitions that overlap - * with the range as dirty. - * - * @param startTime inclusive start timestamp of deletion - * @param endTime inclusive end timestamp of deletion - * @param timePartitionInterval the time partition interval in ms - */ - public void onDeletion(long startTime, long endTime, long timePartitionInterval) { - if (timePartitionInterval <= 0) { - return; - } - long startPartition = startTime / timePartitionInterval; - long endPartition = endTime / timePartitionInterval; - - for (long partitionId = startPartition; partitionId <= endPartition; partitionId++) { - merkleTree.markPartitionDirty(partitionId); - } - - LOGGER.debug( - "Deletion [{}, {}] marked partitions [{}, {}] as dirty", - startTime, - endTime, - startPartition, - endPartition); - } -} diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/consistency/MerkleFlushListener.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/consistency/MerkleFlushListener.java deleted file mode 100644 index c35fd0779db28..0000000000000 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/consistency/MerkleFlushListener.java +++ /dev/null @@ -1,95 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, - * software distributed under the License is distributed on an - * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY - * KIND, either express or implied. See the License for the - * specific language governing permissions and limitations - * under the License. - */ - -package org.apache.iotdb.db.storageengine.dataregion.consistency; - -import org.apache.iotdb.commons.consensus.iotv2.consistency.ConsistencyMerkleTree; -import org.apache.iotdb.commons.consensus.iotv2.consistency.DualDigest; -import org.apache.iotdb.commons.consensus.iotv2.consistency.merkle.MerkleEntry; -import org.apache.iotdb.commons.consensus.iotv2.consistency.merkle.MerkleFileWriter; -import org.apache.iotdb.db.storageengine.dataregion.flush.FlushListener; -import org.apache.iotdb.db.storageengine.dataregion.memtable.IMemTable; -import org.apache.iotdb.db.storageengine.dataregion.tsfile.TsFileResource; - -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -import java.io.IOException; -import java.util.List; - -/** - * FlushListener that computes hash digests for a newly flushed TsFile and generates the - * corresponding .merkle sidecar file. Also updates the in-memory ConsistencyMerkleTree with the - * file's root hash. - */ -public class MerkleFlushListener implements FlushListener { - - private static final Logger LOGGER = LoggerFactory.getLogger(MerkleFlushListener.class); - - private final TsFileResource tsFileResource; - private final ConsistencyMerkleTree merkleTree; - - public MerkleFlushListener(TsFileResource tsFileResource, ConsistencyMerkleTree merkleTree) { - this.tsFileResource = tsFileResource; - this.merkleTree = merkleTree; - } - - @Override - public void onMemTableFlushStarted(IMemTable memTable) { - // No action needed at flush start - } - - @Override - public void onMemTableFlushed(IMemTable memTable) { - try { - String tsFilePath = tsFileResource.getTsFilePath(); - long partitionId = tsFileResource.getTimePartition(); - - // Scan the flushed TsFile and compute per-(device, measurement, timeBucket) hashes - List entries = MerkleHashComputer.computeEntries(tsFilePath); - - if (entries.isEmpty()) { - return; - } - - long fileXorHash = MerkleFileWriter.computeFileXorHash(entries); - long fileAddHash = MerkleFileWriter.computeFileAddHash(entries); - - // Write .merkle sidecar file - MerkleFileWriter.write(tsFilePath + ".merkle", entries, fileXorHash, fileAddHash); - - // Update in-memory Merkle tree - merkleTree.onTsFileFlushed(partitionId, new DualDigest(fileXorHash, fileAddHash)); - - LOGGER.debug( - "Generated .merkle file for {} with {} entries, xorHash=0x{}, addHash=0x{}, partitionId={}", - tsFilePath, - entries.size(), - Long.toHexString(fileXorHash), - Long.toHexString(fileAddHash), - partitionId); - } catch (IOException e) { - LOGGER.warn( - "Failed to generate .merkle file for {}: {}", - tsFileResource.getTsFilePath(), - e.getMessage(), - e); - } - } -} diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/consistency/MerkleHashComputer.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/consistency/MerkleHashComputer.java deleted file mode 100644 index 2f24dccb9b6fe..0000000000000 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/consistency/MerkleHashComputer.java +++ /dev/null @@ -1,232 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, - * software distributed under the License is distributed on an - * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY - * KIND, either express or implied. See the License for the - * specific language governing permissions and limitations - * under the License. - */ - -package org.apache.iotdb.db.storageengine.dataregion.consistency; - -import org.apache.iotdb.commons.consensus.iotv2.consistency.merkle.MerkleEntry; -import org.apache.iotdb.db.queryengine.execution.operator.source.relational.aggregation.grouped.hash.XxHash64; -import org.apache.iotdb.db.utils.EncryptDBUtils; - -import org.apache.tsfile.enums.TSDataType; -import org.apache.tsfile.file.metadata.ChunkMetadata; -import org.apache.tsfile.file.metadata.IDeviceID; -import org.apache.tsfile.read.TsFileDeviceIterator; -import org.apache.tsfile.read.TsFileSequenceReader; -import org.apache.tsfile.read.common.BatchData; -import org.apache.tsfile.read.common.Chunk; -import org.apache.tsfile.read.reader.chunk.ChunkReader; -import org.apache.tsfile.utils.Pair; - -import java.io.IOException; -import java.util.ArrayList; -import java.util.Iterator; -import java.util.List; -import java.util.Map; -import java.util.TreeMap; - -/** - * Scans a TsFile and computes per-(device, measurement, timeBucket) hash entries for the .merkle - * sidecar file. Each entry's hash is the XxHash64 of all sorted (timestamp, value) pairs in that - * bucket. - */ -public class MerkleHashComputer { - - static final long DEFAULT_TIME_BUCKET_INTERVAL_MS = 3_600_000L; // 1 hour - - private MerkleHashComputer() {} - - /** - * Scan a TsFile and compute all MerkleEntry hashes. - * - * @param tsFilePath absolute path to the TsFile - * @return sorted list of MerkleEntries (sorted by device -> measurement -> timeBucketStart) - */ - public static List computeEntries(String tsFilePath) throws IOException { - return computeEntries(tsFilePath, DEFAULT_TIME_BUCKET_INTERVAL_MS); - } - - public static List computeEntries(String tsFilePath, long timeBucketIntervalMs) - throws IOException { - // Accumulator: device -> measurement -> bucketStart -> BucketAccumulator - TreeMap>> accumulator = - new TreeMap<>(); - - try (TsFileSequenceReader reader = - new TsFileSequenceReader( - tsFilePath, EncryptDBUtils.getFirstEncryptParamFromTSFilePath(tsFilePath))) { - - TsFileDeviceIterator deviceIterator = reader.getAllDevicesIteratorWithIsAligned(); - while (deviceIterator.hasNext()) { - Pair devicePair = deviceIterator.next(); - IDeviceID deviceId = devicePair.left; - String deviceIdStr = deviceId.toString(); - - TreeMap> deviceAccum = - accumulator.computeIfAbsent(deviceIdStr, k -> new TreeMap<>()); - - Iterator>> measurementChunkIter = - reader.getMeasurementChunkMetadataListMapIterator(deviceId); - - while (measurementChunkIter.hasNext()) { - Map> measurementChunks = measurementChunkIter.next(); - for (Map.Entry> entry : measurementChunks.entrySet()) { - String measurement = entry.getKey(); - // Skip empty or time-column measurements - if (measurement.isEmpty()) { - continue; - } - TreeMap measAccum = - deviceAccum.computeIfAbsent(measurement, k -> new TreeMap<>()); - - for (ChunkMetadata chunkMeta : entry.getValue()) { - TSDataType dataType = chunkMeta.getDataType(); - Chunk chunk = reader.readMemChunk(chunkMeta); - ChunkReader chunkReader = new ChunkReader(chunk); - - while (chunkReader.hasNextSatisfiedPage()) { - BatchData batchData = chunkReader.nextPageData(); - while (batchData.hasCurrent()) { - long timestamp = batchData.currentTime(); - long bucketStart = (timestamp / timeBucketIntervalMs) * timeBucketIntervalMs; - - BucketAccumulator bucket = - measAccum.computeIfAbsent( - bucketStart, - bs -> new BucketAccumulator(bs, bs + timeBucketIntervalMs, dataType)); - bucket.addPoint(timestamp, batchData.currentValue()); - batchData.next(); - } - } - } - } - } - } - } - - return flattenToEntries(accumulator); - } - - private static List flattenToEntries( - TreeMap>> accumulator) { - List entries = new ArrayList<>(); - for (Map.Entry>> deviceEntry : - accumulator.entrySet()) { - String deviceId = deviceEntry.getKey(); - for (Map.Entry> measEntry : - deviceEntry.getValue().entrySet()) { - String measurement = measEntry.getKey(); - for (BucketAccumulator bucket : measEntry.getValue().values()) { - entries.add( - new MerkleEntry( - deviceId, - measurement, - bucket.bucketStart, - bucket.bucketEnd, - bucket.pointCount, - bucket.computeHash())); - } - } - } - return entries; - } - - /** Accumulates (timestamp, value) pairs in a time bucket and computes their combined hash. */ - static class BucketAccumulator { - final long bucketStart; - final long bucketEnd; - final TSDataType dataType; - int pointCount; - final TreeMap points; - - BucketAccumulator(long bucketStart, long bucketEnd, TSDataType dataType) { - this.bucketStart = bucketStart; - this.bucketEnd = bucketEnd; - this.dataType = dataType; - this.pointCount = 0; - this.points = new TreeMap<>(); - } - - void addPoint(long timestamp, Object value) { - points.put(timestamp, value); - pointCount++; - } - - long computeHash() { - XxHash64 hasher = new XxHash64(); - byte[] buf = new byte[8]; - for (Map.Entry entry : points.entrySet()) { - longToBytes(entry.getKey(), buf); - hasher.update(buf); - byte[] valueBytes = valueToBytes(entry.getValue(), dataType); - hasher.update(valueBytes); - } - return hasher.hash(); - } - } - - static void longToBytes(long v, byte[] buf) { - buf[0] = (byte) (v >>> 56); - buf[1] = (byte) (v >>> 48); - buf[2] = (byte) (v >>> 40); - buf[3] = (byte) (v >>> 32); - buf[4] = (byte) (v >>> 24); - buf[5] = (byte) (v >>> 16); - buf[6] = (byte) (v >>> 8); - buf[7] = (byte) v; - } - - static byte[] valueToBytes(Object value, TSDataType dataType) { - if (value == null) { - return new byte[0]; - } - switch (dataType) { - case BOOLEAN: - return new byte[] {(byte) ((Boolean) value ? 1 : 0)}; - case INT32: - case DATE: - return intToBytes((Integer) value); - case INT64: - case TIMESTAMP: - return longToBytesNew((Long) value); - case FLOAT: - return intToBytes(Float.floatToIntBits((Float) value)); - case DOUBLE: - return longToBytesNew(Double.doubleToLongBits((Double) value)); - case TEXT: - case STRING: - case BLOB: - if (value instanceof byte[]) { - return (byte[]) value; - } - return value.toString().getBytes(); - default: - return value.toString().getBytes(); - } - } - - private static byte[] intToBytes(int v) { - return new byte[] {(byte) (v >>> 24), (byte) (v >>> 16), (byte) (v >>> 8), (byte) v}; - } - - private static byte[] longToBytesNew(long v) { - byte[] buf = new byte[8]; - longToBytes(v, buf); - return buf; - } -} diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/read/control/QueryResourceManager.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/read/control/QueryResourceManager.java index 930d68b4e891d..911d50e37f31f 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/read/control/QueryResourceManager.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/read/control/QueryResourceManager.java @@ -49,6 +49,16 @@ public long assignQueryId() { return queryIdAtom.incrementAndGet(); } + /** + * Register a unique internal read id that also participates in QueryFileManager reference + * tracking. Use this for non-session background reads that manage file references directly. + */ + public long assignInternalQueryId() { + long queryId = queryIdAtom.incrementAndGet(); + filePathsManager.addQueryId(queryId); + return queryId; + } + /** * Register a read id for compaction. The name of the compaction thread is * 'pool-x-IoTDB-Compaction-xx', xx in which is usually an integer from 0 to diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/tsfile/TsFileManager.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/tsfile/TsFileManager.java index b7c1ba2c14fb4..c44d03bc3a8cf 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/tsfile/TsFileManager.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/tsfile/TsFileManager.java @@ -19,8 +19,11 @@ package org.apache.iotdb.db.storageengine.dataregion.tsfile; +import org.apache.iotdb.common.rpc.thrift.TConsensusGroupId; +import org.apache.iotdb.common.rpc.thrift.TConsensusGroupType; import org.apache.iotdb.commons.utils.TimePartitionUtils; import org.apache.iotdb.db.pipe.resource.PipeDataNodeResourceManager; +import org.apache.iotdb.db.storageengine.dataregion.consistency.DataRegionConsistencyManager; import org.apache.iotdb.db.storageengine.dataregion.modification.ModFileManagement; import org.apache.iotdb.db.storageengine.dataregion.modification.PartitionLevelModFileManager; import org.apache.iotdb.db.storageengine.dataregion.tsfile.timeindex.FileTimeIndexCacheRecorder; @@ -349,6 +352,14 @@ public void replace( unseqFileResources, targetFileResources); } + + DataRegionConsistencyManager.getInstance() + .onCompaction( + new TConsensusGroupId(TConsensusGroupType.DataRegion, Integer.parseInt(dataRegionId)), + seqFileResources, + unseqFileResources, + targetFileResources, + timePartition); } public boolean contains(TsFileResource tsFileResource, boolean sequence) { diff --git a/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/pipe/sink/protocol/iotconsensusv2/IoTConsensusV2AsyncSinkTest.java b/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/pipe/sink/protocol/iotconsensusv2/IoTConsensusV2AsyncSinkTest.java new file mode 100644 index 0000000000000..b7ef3e9570310 --- /dev/null +++ b/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/pipe/sink/protocol/iotconsensusv2/IoTConsensusV2AsyncSinkTest.java @@ -0,0 +1,80 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.iotdb.db.pipe.sink.protocol.iotconsensusv2; + +import org.apache.iotdb.commons.pipe.event.EnrichedEvent; +import org.apache.iotdb.db.pipe.consensus.ReplicateProgressDataNodeManager; +import org.apache.iotdb.db.pipe.consensus.metric.IoTConsensusV2SinkMetrics; + +import org.junit.Assert; +import org.junit.Test; +import org.mockito.Mockito; + +import java.lang.reflect.Field; +import java.lang.reflect.Method; + +public class IoTConsensusV2AsyncSinkTest { + + @Test + public void testLeaderReplicateProgressIgnoresPreAssignedButDiscardedEvents() throws Exception { + IoTConsensusV2AsyncSink sink = new IoTConsensusV2AsyncSink(); + setField(sink, "iotConsensusV2SinkMetrics", Mockito.mock(IoTConsensusV2SinkMetrics.class)); + + String pipeName = "__consensus.test_" + System.nanoTime(); + try { + Assert.assertEquals( + 1L, ReplicateProgressDataNodeManager.assignReplicateIndexForIoTV2(pipeName)); + Assert.assertEquals( + "Source-side replicate index assignment should not affect connector sync lag progress", + 0L, + sink.getLeaderReplicateProgress()); + + EnrichedEvent enqueuedEvent = Mockito.mock(EnrichedEvent.class); + Mockito.when(enqueuedEvent.getReplicateIndexForIoTV2()).thenReturn(1L); + Mockito.when(enqueuedEvent.increaseReferenceCount(Mockito.anyString())).thenReturn(true); + + Assert.assertTrue(invokeAddEvent2Buffer(sink, enqueuedEvent)); + Assert.assertEquals(1L, sink.getLeaderReplicateProgress()); + + Assert.assertEquals( + 2L, ReplicateProgressDataNodeManager.assignReplicateIndexForIoTV2(pipeName)); + Assert.assertEquals( + "Discarded events that never enter the connector must not create phantom lag", + 1L, + sink.getLeaderReplicateProgress()); + } finally { + ReplicateProgressDataNodeManager.resetReplicateIndexForIoTV2(pipeName); + } + } + + private boolean invokeAddEvent2Buffer(IoTConsensusV2AsyncSink sink, EnrichedEvent event) + throws Exception { + Method method = + IoTConsensusV2AsyncSink.class.getDeclaredMethod("addEvent2Buffer", EnrichedEvent.class); + method.setAccessible(true); + return (boolean) method.invoke(sink, event); + } + + private void setField(Object target, String fieldName, Object value) throws Exception { + Field field = target.getClass().getDeclaredField(fieldName); + field.setAccessible(true); + field.set(target, value); + } +} diff --git a/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/consensus/iotv2/consistency/StaleSnapshotException.java b/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/queryengine/plan/relational/planner/optimizations/DataNodeLocationSupplierFactoryTest.java similarity index 61% rename from iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/consensus/iotv2/consistency/StaleSnapshotException.java rename to iotdb-core/datanode/src/test/java/org/apache/iotdb/db/queryengine/plan/relational/planner/optimizations/DataNodeLocationSupplierFactoryTest.java index 3a1c14f549206..eac9b59250937 100644 --- a/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/consensus/iotv2/consistency/StaleSnapshotException.java +++ b/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/queryengine/plan/relational/planner/optimizations/DataNodeLocationSupplierFactoryTest.java @@ -17,15 +17,21 @@ * under the License. */ -package org.apache.iotdb.commons.consensus.iotv2.consistency; +package org.apache.iotdb.db.queryengine.plan.relational.planner.optimizations; -/** - * Thrown when a partition is modified (by flush, compaction, or deletion) during consistency view - * construction, indicating the snapshot is stale and the operation should be retried. - */ -public class StaleSnapshotException extends Exception { +import org.apache.iotdb.commons.schema.table.InformationSchema; + +import org.junit.Assert; +import org.junit.Test; + +public class DataNodeLocationSupplierFactoryTest { - public StaleSnapshotException(String message) { - super(message); + @Test + public void testRepairProgressUsesLocalInformationSchemaSupplier() { + Assert.assertEquals( + 1, + DataNodeLocationSupplierFactory.getSupplier() + .getDataNodeLocations(InformationSchema.REPAIR_PROGRESS) + .size()); } } diff --git a/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/storageengine/dataregion/consistency/DataRegionConsistencyManagerTest.java b/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/storageengine/dataregion/consistency/DataRegionConsistencyManagerTest.java new file mode 100644 index 0000000000000..79add27513f22 --- /dev/null +++ b/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/storageengine/dataregion/consistency/DataRegionConsistencyManagerTest.java @@ -0,0 +1,444 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.iotdb.db.storageengine.dataregion.consistency; + +import org.apache.iotdb.common.rpc.thrift.TConsensusGroupId; +import org.apache.iotdb.common.rpc.thrift.TConsensusGroupType; +import org.apache.iotdb.commons.consensus.iotv2.consistency.RepairProgressTable; +import org.apache.iotdb.db.storageengine.dataregion.DataRegion; +import org.apache.iotdb.db.storageengine.dataregion.compaction.execute.utils.MultiTsFileDeviceIterator; +import org.apache.iotdb.db.storageengine.dataregion.compaction.utils.CompactionTestFileWriter; +import org.apache.iotdb.db.storageengine.dataregion.memtable.AlignedWritableMemChunkGroup; +import org.apache.iotdb.db.storageengine.dataregion.memtable.IMemTable; +import org.apache.iotdb.db.storageengine.dataregion.memtable.IWritableMemChunkGroup; +import org.apache.iotdb.db.storageengine.dataregion.memtable.PrimitiveMemTable; +import org.apache.iotdb.db.storageengine.dataregion.memtable.TsFileProcessor; +import org.apache.iotdb.db.storageengine.dataregion.memtable.WritableMemChunkGroup; +import org.apache.iotdb.db.storageengine.dataregion.read.control.FileReaderManager; +import org.apache.iotdb.db.storageengine.dataregion.tsfile.TsFileManager; +import org.apache.iotdb.db.storageengine.dataregion.tsfile.TsFileResource; + +import org.apache.tsfile.enums.TSDataType; +import org.apache.tsfile.file.metadata.IDeviceID; +import org.apache.tsfile.file.metadata.enums.CompressionType; +import org.apache.tsfile.file.metadata.enums.TSEncoding; +import org.apache.tsfile.read.common.TimeRange; +import org.apache.tsfile.write.schema.IMeasurementSchema; +import org.apache.tsfile.write.schema.MeasurementSchema; +import org.junit.Assert; +import org.junit.Test; +import org.mockito.Mockito; + +import java.io.File; +import java.io.IOException; +import java.lang.reflect.Field; +import java.lang.reflect.Method; +import java.nio.channels.ClosedChannelException; +import java.nio.charset.StandardCharsets; +import java.nio.file.Files; +import java.nio.file.Path; +import java.util.Base64; +import java.util.Collections; +import java.util.Comparator; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import java.util.concurrent.ConcurrentHashMap; +import java.util.concurrent.ConcurrentLinkedDeque; + +public class DataRegionConsistencyManagerTest { + + @Test + public void compactionShouldNotAdvancePartitionMutationEpoch() throws Exception { + DataRegionConsistencyManager consistencyManager = DataRegionConsistencyManager.getInstance(); + TConsensusGroupId consensusGroupId = new TConsensusGroupId(TConsensusGroupType.DataRegion, 901); + + consistencyManager.onPartitionMutation(consensusGroupId, 7L); + long beforeEpoch = getPartitionMutationEpoch(consistencyManager, consensusGroupId, 7L); + + consistencyManager.onCompaction( + consensusGroupId, + Collections.emptyList(), + Collections.emptyList(), + Collections.emptyList(), + 7L); + + Assert.assertEquals( + beforeEpoch, getPartitionMutationEpoch(consistencyManager, consensusGroupId, 7L)); + Assert.assertEquals( + RepairProgressTable.SnapshotState.DIRTY, + getSnapshotState(consistencyManager, consensusGroupId, 7L)); + } + + @Test + public void deletionShouldMarkAffectedPartitionDirty() throws Exception { + DataRegionConsistencyManager consistencyManager = DataRegionConsistencyManager.getInstance(); + TConsensusGroupId consensusGroupId = new TConsensusGroupId(TConsensusGroupType.DataRegion, 902); + + consistencyManager.onDeletion(consensusGroupId, 0L, 0L); + + Assert.assertEquals(1L, getPartitionMutationEpoch(consistencyManager, consensusGroupId, 0L)); + Assert.assertEquals( + RepairProgressTable.SnapshotState.DIRTY, + getSnapshotState(consistencyManager, consensusGroupId, 0L)); + } + + @Test + public void logicalRepairMutationShouldNotAdvancePartitionMutationEpoch() throws Exception { + DataRegionConsistencyManager consistencyManager = DataRegionConsistencyManager.getInstance(); + TConsensusGroupId consensusGroupId = new TConsensusGroupId(TConsensusGroupType.DataRegion, 903); + + consistencyManager.onPartitionMutation(consensusGroupId, 11L); + long beforeEpoch = getPartitionMutationEpoch(consistencyManager, consensusGroupId, 11L); + + consistencyManager.runWithLogicalRepairMutation( + consensusGroupId, + 11L, + "1:11:0:" + beforeEpoch + ":" + beforeEpoch, + () -> { + consistencyManager.onPartitionMutation(consensusGroupId, 11L); + return null; + }); + + Assert.assertEquals( + beforeEpoch, getPartitionMutationEpoch(consistencyManager, consensusGroupId, 11L)); + Assert.assertEquals( + RepairProgressTable.SnapshotState.DIRTY, + getSnapshotState(consistencyManager, consensusGroupId, 11L)); + } + + @Test + public void exactRepairSelectorShouldRoundTripExactKeys() throws Exception { + Class selectorClass = + Class.forName( + "org.apache.iotdb.db.storageengine.dataregion.consistency.DataRegionConsistencyManager$LogicalLeafSelector"); + Method parseMethod = selectorClass.getDeclaredMethod("parse", String.class); + parseMethod.setAccessible(true); + + String selectorToken = + "leaf:4:9@@@" + + Base64.getUrlEncoder() + .withoutPadding() + .encodeToString( + String.join("\n", "root.db.d1|s1|1|INT64|1", "root.db.d1|s2|2|INT64|2") + .getBytes(StandardCharsets.UTF_8)); + Object selector = parseMethod.invoke(null, selectorToken); + + Field exactKeysField = selectorClass.getDeclaredField("exactKeys"); + exactKeysField.setAccessible(true); + + @SuppressWarnings("unchecked") + java.util.Set exactKeys = (java.util.Set) exactKeysField.get(selector); + Assert.assertEquals(2, exactKeys.size()); + Assert.assertTrue(exactKeys.contains("root.db.d1|s1|1|INT64|1")); + Assert.assertTrue(exactKeys.contains("root.db.d1|s2|2|INT64|2")); + } + + @Test + public void rangeRepairSelectorShouldHonorLogicalKeyRange() throws Exception { + Class selectorClass = + Class.forName( + "org.apache.iotdb.db.storageengine.dataregion.consistency.DataRegionConsistencyManager$LogicalLeafSelector"); + Method parseMethod = selectorClass.getDeclaredMethod("parse", String.class); + parseMethod.setAccessible(true); + + Method computeDeviceShardMethod = + DataRegionConsistencyManager.class.getDeclaredMethod("computeDeviceShard", String.class); + computeDeviceShardMethod.setAccessible(true); + int shard = (Integer) computeDeviceShardMethod.invoke(null, "root.db.d1"); + + String selectorToken = + "leaf:" + + shard + + ":0@" + + encodeBase64("root.db.d1|s1|1|INT64|1") + + "@" + + encodeBase64("root.db.d1|s1|3|INT64|3") + + "@"; + Object selector = parseMethod.invoke(null, selectorToken); + + Method matchesLiveCellMethod = + selectorClass.getDeclaredMethod( + "matchesLiveCell", + String.class, + String.class, + TSDataType.class, + long.class, + Object.class); + matchesLiveCellMethod.setAccessible(true); + Method requiresScopedResetMethod = selectorClass.getDeclaredMethod("requiresScopedReset"); + requiresScopedResetMethod.setAccessible(true); + + Assert.assertTrue((Boolean) requiresScopedResetMethod.invoke(selector)); + Assert.assertTrue( + (Boolean) + matchesLiveCellMethod.invoke(selector, "root.db.d1", "s1", TSDataType.INT64, 2L, 2L)); + Assert.assertFalse( + (Boolean) + matchesLiveCellMethod.invoke(selector, "root.db.d1", "s1", TSDataType.INT64, 4L, 4L)); + Assert.assertFalse( + (Boolean) + matchesLiveCellMethod.invoke(selector, "root.db.d1", "s1", TSDataType.INT64, 0L, 0L)); + } + + @Test + public void snapshotTreeShouldTrackLogicalKeyBounds() throws Exception { + Class snapshotTreeClass = + Class.forName( + "org.apache.iotdb.db.storageengine.dataregion.consistency.DataRegionConsistencyManager$SnapshotTree"); + Method emptyMethod = snapshotTreeClass.getDeclaredMethod("empty"); + emptyMethod.setAccessible(true); + Object snapshotTree = emptyMethod.invoke(null); + + Method addLeafEntryMethod = + snapshotTreeClass.getDeclaredMethod( + "addLeafEntry", String.class, int.class, String.class, long.class, long.class); + addLeafEntryMethod.setAccessible(true); + addLeafEntryMethod.invoke(snapshotTree, "leaf:7:0", 7, "root.db.d1|s1|2|INT64|2", 11L, 1L); + addLeafEntryMethod.invoke(snapshotTree, "leaf:7:0", 7, "root.db.d1|s1|1|INT64|1", 13L, 1L); + addLeafEntryMethod.invoke(snapshotTree, "leaf:7:0", 7, "root.db.d1|s1|3|INT64|3", 17L, 1L); + + Field nodesByHandleField = snapshotTreeClass.getDeclaredField("nodesByHandle"); + nodesByHandleField.setAccessible(true); + @SuppressWarnings("unchecked") + Map nodesByHandle = (Map) nodesByHandleField.get(snapshotTree); + Object leafNode = nodesByHandle.get("leaf:7:0"); + Assert.assertNotNull(leafNode); + + Field keyRangeStartField = leafNode.getClass().getDeclaredField("keyRangeStart"); + keyRangeStartField.setAccessible(true); + Field keyRangeEndField = leafNode.getClass().getDeclaredField("keyRangeEnd"); + keyRangeEndField.setAccessible(true); + + Assert.assertEquals("root.db.d1|s1|1|INT64|1", keyRangeStartField.get(leafNode)); + Assert.assertEquals("root.db.d1|s1|3|INT64|3", keyRangeEndField.get(leafNode)); + } + + @Test + public void memTableSeriesDiscoveryShouldIncludeUnsealedMeasurements() throws Exception { + DataRegionConsistencyManager consistencyManager = DataRegionConsistencyManager.getInstance(); + + IMeasurementSchema alignedS1 = new MeasurementSchema("s1", TSDataType.INT64); + IMeasurementSchema alignedS2 = new MeasurementSchema("s2", TSDataType.DOUBLE); + List alignedSchemas = java.util.Arrays.asList(alignedS1, alignedS2); + AlignedWritableMemChunkGroup alignedGroup = + new AlignedWritableMemChunkGroup(alignedSchemas, false); + alignedGroup.writeRow(1L, new Object[] {1L, 1.0d}, alignedSchemas); + + IMeasurementSchema nonAlignedSchema = new MeasurementSchema("s3", TSDataType.INT32); + WritableMemChunkGroup nonAlignedGroup = new WritableMemChunkGroup(); + nonAlignedGroup.writeRow(2L, new Object[] {1}, Collections.singletonList(nonAlignedSchema)); + + Map memTableMap = new HashMap<>(); + memTableMap.put(IDeviceID.Factory.DEFAULT_FACTORY.create("root.db.d1"), alignedGroup); + memTableMap.put(IDeviceID.Factory.DEFAULT_FACTORY.create("root.db.d2"), nonAlignedGroup); + PrimitiveMemTable memTable = new PrimitiveMemTable("root.db", "1", memTableMap); + + Method collectMemTableSeriesContexts = + DataRegionConsistencyManager.class.getDeclaredMethod( + "collectMemTableSeriesContexts", IMemTable.class, Map.class); + collectMemTableSeriesContexts.setAccessible(true); + + Map deviceSeriesContexts = new java.util.TreeMap<>(); + collectMemTableSeriesContexts.invoke(consistencyManager, memTable, deviceSeriesContexts); + + Assert.assertEquals(2, deviceSeriesContexts.size()); + assertDeviceSeriesContext(deviceSeriesContexts.get("root.db.d1"), true, "s1", "s2"); + assertDeviceSeriesContext(deviceSeriesContexts.get("root.db.d2"), false, "s3"); + } + + @Test + public void closedChannelFailuresShouldBeClassifiedAsRetryable() throws Exception { + DataRegionConsistencyManager consistencyManager = DataRegionConsistencyManager.getInstance(); + Method method = + DataRegionConsistencyManager.class.getDeclaredMethod( + "isRetryableSnapshotReadFailure", Throwable.class); + method.setAccessible(true); + + Assert.assertTrue((Boolean) method.invoke(consistencyManager, new ClosedChannelException())); + Assert.assertTrue( + (Boolean) + method.invoke( + consistencyManager, new IOException("wrapped", new ClosedChannelException()))); + Assert.assertFalse( + (Boolean) method.invoke(consistencyManager, new IOException("other io failure"))); + } + + @Test + public void inspectPartitionShouldRebuildSnapshotEvenWhenWorkingProcessorsExist() + throws Exception { + DataRegionConsistencyManager consistencyManager = DataRegionConsistencyManager.getInstance(); + TConsensusGroupId consensusGroupId = new TConsensusGroupId(TConsensusGroupType.DataRegion, 904); + + DataRegion dataRegion = Mockito.mock(DataRegion.class); + TsFileManager tsFileManager = Mockito.mock(TsFileManager.class); + Mockito.when(dataRegion.getTsFileManager()).thenReturn(tsFileManager); + Mockito.when(tsFileManager.getTsFileListSnapshot(1L, true)).thenReturn(Collections.emptyList()); + Mockito.when(tsFileManager.getTsFileListSnapshot(1L, false)) + .thenReturn(Collections.emptyList()); + TsFileProcessor processor = Mockito.mock(TsFileProcessor.class); + IMemTable emptyMemTable = Mockito.mock(IMemTable.class); + Mockito.when(processor.getTimeRangeId()).thenReturn(1L); + Mockito.when(processor.tryReadLock(1_000L)).thenReturn(true); + Mockito.when(processor.getWorkMemTable()).thenReturn(emptyMemTable); + Mockito.when(processor.getFlushingMemTable()).thenReturn(new ConcurrentLinkedDeque<>()); + Mockito.doNothing().when(processor).readUnLock(); + Mockito.when(emptyMemTable.getMemTableMap()).thenReturn(Collections.emptyMap()); + Mockito.when(dataRegion.getWorkSequenceTsFileProcessors()) + .thenReturn(Collections.singletonList(processor)); + Mockito.when(dataRegion.getWorkUnsequenceTsFileProcessors()) + .thenReturn(Collections.emptyList()); + + DataRegionConsistencyManager.PartitionInspection inspection = + consistencyManager.inspectPartition( + consensusGroupId, dataRegion, 1L, Collections.emptyList()); + + Assert.assertEquals(1L, inspection.getPartitionId()); + Assert.assertEquals(RepairProgressTable.SnapshotState.READY, inspection.getSnapshotState()); + Assert.assertEquals(0L, inspection.getPartitionMutationEpoch()); + Assert.assertEquals(0L, inspection.getSnapshotEpoch()); + } + + @Test + public void collectLogicalSeriesContextsShouldNotPoisonSharedReaders() throws Exception { + DataRegionConsistencyManager consistencyManager = DataRegionConsistencyManager.getInstance(); + Path tempDir = Files.createTempDirectory("consistency-manager-reader-regression"); + TsFileResource resource = null; + MultiTsFileDeviceIterator verificationIterator = null; + try { + Path tsFileDir = tempDir.resolve("sequence/root.testsg/1/1"); + Files.createDirectories(tsFileDir); + File tsFile = tsFileDir.resolve("1-1-0-0.tsfile").toFile(); + resource = new TsFileResource(tsFile); + try (CompactionTestFileWriter writer = new CompactionTestFileWriter(resource)) { + writer.startChunkGroup("d1"); + writer.generateSimpleNonAlignedSeriesToCurrentDevice( + "s1", new TimeRange[] {new TimeRange(1, 3)}, TSEncoding.PLAIN, CompressionType.LZ4); + writer.endChunkGroup(); + writer.endFile(); + } + resource.close(); + + TsFileManager tsFileManager = Mockito.mock(TsFileManager.class); + DataRegion dataRegion = Mockito.mock(DataRegion.class); + Mockito.when(dataRegion.getTsFileManager()).thenReturn(tsFileManager); + Mockito.when(tsFileManager.getTsFileListSnapshot(1L, true)) + .thenReturn(Collections.singletonList(resource)); + Mockito.when(tsFileManager.getTsFileListSnapshot(1L, false)) + .thenReturn(Collections.emptyList()); + Mockito.when(dataRegion.getWorkSequenceTsFileProcessors()) + .thenReturn(Collections.emptyList()); + Mockito.when(dataRegion.getWorkUnsequenceTsFileProcessors()) + .thenReturn(Collections.emptyList()); + + Method method = + DataRegionConsistencyManager.class.getDeclaredMethod( + "collectLogicalSeriesContexts", DataRegion.class, long.class); + method.setAccessible(true); + + @SuppressWarnings("unchecked") + Map deviceSeriesContexts = + (Map) method.invoke(consistencyManager, dataRegion, 1L); + assertDeviceSeriesContext(deviceSeriesContexts.get("root.testsg.d1"), false, "s1"); + + verificationIterator = + new MultiTsFileDeviceIterator( + Collections.singletonList(resource), Collections.emptyList()); + Assert.assertTrue(verificationIterator.hasNextDevice()); + Assert.assertEquals("root.testsg.d1", String.valueOf(verificationIterator.nextDevice().left)); + Assert.assertTrue(verificationIterator.getAllSchemasOfCurrentDevice().containsKey("s1")); + } finally { + if (resource != null) { + FileReaderManager.getInstance().closeFileAndRemoveReader(resource.getTsFileID()); + } + Files.walk(tempDir).sorted(Comparator.reverseOrder()).map(Path::toFile).forEach(File::delete); + } + } + + private String encodeBase64(String value) { + return Base64.getUrlEncoder() + .withoutPadding() + .encodeToString(value.getBytes(StandardCharsets.UTF_8)); + } + + @SuppressWarnings("unchecked") + private void assertDeviceSeriesContext( + Object deviceSeriesContext, boolean expectedAligned, String... expectedMeasurements) + throws Exception { + Assert.assertNotNull(deviceSeriesContext); + + Field alignedField = deviceSeriesContext.getClass().getDeclaredField("aligned"); + alignedField.setAccessible(true); + Assert.assertEquals(expectedAligned, alignedField.getBoolean(deviceSeriesContext)); + + Field measurementSchemasField = + deviceSeriesContext.getClass().getDeclaredField("measurementSchemas"); + measurementSchemasField.setAccessible(true); + Map measurementSchemas = + (Map) measurementSchemasField.get(deviceSeriesContext); + Assert.assertEquals(expectedMeasurements.length, measurementSchemas.size()); + for (String expectedMeasurement : expectedMeasurements) { + Assert.assertTrue(measurementSchemas.containsKey(expectedMeasurement)); + } + } + + @SuppressWarnings("unchecked") + private Object getPartitionState( + DataRegionConsistencyManager consistencyManager, + TConsensusGroupId consensusGroupId, + long partitionId) + throws Exception { + Field regionStatesField = DataRegionConsistencyManager.class.getDeclaredField("regionStates"); + regionStatesField.setAccessible(true); + ConcurrentHashMap regionStates = + (ConcurrentHashMap) regionStatesField.get(consistencyManager); + Object regionState = regionStates.get(consensusGroupId.toString()); + Assert.assertNotNull(regionState); + + Field partitionsField = regionState.getClass().getDeclaredField("partitions"); + partitionsField.setAccessible(true); + Map partitions = (Map) partitionsField.get(regionState); + Object partitionState = partitions.get(partitionId); + Assert.assertNotNull(partitionState); + return partitionState; + } + + private long getPartitionMutationEpoch( + DataRegionConsistencyManager consistencyManager, + TConsensusGroupId consensusGroupId, + long partitionId) + throws Exception { + Object partitionState = getPartitionState(consistencyManager, consensusGroupId, partitionId); + Field mutationEpochField = partitionState.getClass().getDeclaredField("partitionMutationEpoch"); + mutationEpochField.setAccessible(true); + return mutationEpochField.getLong(partitionState); + } + + private RepairProgressTable.SnapshotState getSnapshotState( + DataRegionConsistencyManager consistencyManager, + TConsensusGroupId consensusGroupId, + long partitionId) + throws Exception { + Object partitionState = getPartitionState(consistencyManager, consensusGroupId, partitionId); + Field snapshotStateField = partitionState.getClass().getDeclaredField("snapshotState"); + snapshotStateField.setAccessible(true); + return (RepairProgressTable.SnapshotState) snapshotStateField.get(partitionState); + } +} diff --git a/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/storageengine/dataregion/consistency/LogicalRepairSessionJournalTest.java b/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/storageengine/dataregion/consistency/LogicalRepairSessionJournalTest.java new file mode 100644 index 0000000000000..a6a07f5baae2c --- /dev/null +++ b/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/storageengine/dataregion/consistency/LogicalRepairSessionJournalTest.java @@ -0,0 +1,179 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.iotdb.db.storageengine.dataregion.consistency; + +import org.junit.Assert; +import org.junit.Test; + +import java.nio.ByteBuffer; +import java.nio.file.Files; +import java.nio.file.Path; +import java.util.Comparator; +import java.util.List; + +public class LogicalRepairSessionJournalTest { + + @Test + public void stagedBatchesShouldPersistAcrossJournalReload() throws Exception { + Path journalDir = Files.createTempDirectory("logical-repair-journal"); + try { + LogicalRepairSessionJournal journal = new LogicalRepairSessionJournal(journalDir); + String repairEpoch = "1:7:1000:2000:2000:1-1_2_3"; + + journal.stageBatch( + "DataRegion-7", + 7L, + repairEpoch, + "session-a", + "LIVE", + "leaf:1:0", + 2, + "INSERT_ROWS", + ByteBuffer.wrap(new byte[] {2})); + journal.stageBatch( + "DataRegion-7", + 7L, + repairEpoch, + "session-a", + "LIVE", + "leaf:1:0", + 0, + "RESET_LEAF", + ByteBuffer.allocate(0)); + journal.stageBatch( + "DataRegion-7", + 7L, + repairEpoch, + "session-a", + "LIVE", + "leaf:1:0", + 1, + "INSERT_ROWS", + ByteBuffer.wrap(new byte[] {1})); + + LogicalRepairSessionJournal recovered = new LogicalRepairSessionJournal(journalDir); + List recoveredBatches = + recovered.loadStagedBatches("DataRegion-7", 7L, repairEpoch, "session-a"); + + Assert.assertEquals(3, recoveredBatches.size()); + Assert.assertEquals(0, recoveredBatches.get(0).getSeqNo()); + Assert.assertEquals("RESET_LEAF", recoveredBatches.get(0).getBatchKind()); + Assert.assertEquals(1, recoveredBatches.get(1).getSeqNo()); + Assert.assertEquals(2, recoveredBatches.get(2).getSeqNo()); + Assert.assertArrayEquals( + new byte[] {1}, toByteArray(recoveredBatches.get(1).duplicatePayload())); + Assert.assertArrayEquals( + new byte[] {2}, toByteArray(recoveredBatches.get(2).duplicatePayload())); + } finally { + deleteRecursively(journalDir); + } + } + + @Test + public void duplicateSeqNoShouldBeDeduplicatedDurably() throws Exception { + Path journalDir = Files.createTempDirectory("logical-repair-journal"); + try { + LogicalRepairSessionJournal journal = new LogicalRepairSessionJournal(journalDir); + String repairEpoch = "1:8:1000:2000:2000:1-1_2_3"; + + journal.stageBatch( + "DataRegion-8", + 8L, + repairEpoch, + "session-b", + "TOMBSTONE", + "leaf:2:0", + 0, + "DELETE_DATA", + ByteBuffer.wrap(new byte[] {1, 2, 3})); + journal.stageBatch( + "DataRegion-8", + 8L, + repairEpoch, + "session-b", + "TOMBSTONE", + "leaf:2:0", + 0, + "DELETE_DATA", + ByteBuffer.wrap(new byte[] {9, 9, 9})); + + LogicalRepairSessionJournal recovered = new LogicalRepairSessionJournal(journalDir); + List recoveredBatches = + recovered.loadStagedBatches("DataRegion-8", 8L, repairEpoch, "session-b"); + + Assert.assertEquals(1, recoveredBatches.size()); + Assert.assertArrayEquals( + new byte[] {1, 2, 3}, toByteArray(recoveredBatches.get(0).duplicatePayload())); + } finally { + deleteRecursively(journalDir); + } + } + + @Test + public void completeShouldDeletePersistedSession() throws Exception { + Path journalDir = Files.createTempDirectory("logical-repair-journal"); + try { + LogicalRepairSessionJournal journal = new LogicalRepairSessionJournal(journalDir); + String repairEpoch = "1:9:1000:2000:2000:1-1_2_3"; + + journal.stageBatch( + "DataRegion-9", + 9L, + repairEpoch, + "session-c", + "LIVE", + "leaf:3:0", + 0, + "RESET_LEAF", + ByteBuffer.allocate(0)); + journal.completeSession("session-c"); + + LogicalRepairSessionJournal recovered = new LogicalRepairSessionJournal(journalDir); + Assert.assertTrue( + recovered.loadStagedBatches("DataRegion-9", 9L, repairEpoch, "session-c").isEmpty()); + } finally { + deleteRecursively(journalDir); + } + } + + private byte[] toByteArray(ByteBuffer buffer) { + ByteBuffer duplicate = buffer.duplicate(); + byte[] bytes = new byte[duplicate.remaining()]; + duplicate.get(bytes); + return bytes; + } + + private void deleteRecursively(Path dir) throws Exception { + if (dir == null || !Files.exists(dir)) { + return; + } + try (java.util.stream.Stream stream = Files.walk(dir)) { + stream.sorted(Comparator.reverseOrder()).forEach(this::deleteSilently); + } + } + + private void deleteSilently(Path path) { + try { + Files.deleteIfExists(path); + } catch (Exception ignored) { + // best effort for test cleanup + } + } +} diff --git a/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/consensus/iotv2/consistency/ConsistencyCheckScheduler.java b/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/consensus/iotv2/consistency/ConsistencyCheckScheduler.java deleted file mode 100644 index 20aca89ff29ca..0000000000000 --- a/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/consensus/iotv2/consistency/ConsistencyCheckScheduler.java +++ /dev/null @@ -1,217 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, - * software distributed under the License is distributed on an - * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY - * KIND, either express or implied. See the License for the - * specific language governing permissions and limitations - * under the License. - */ - -package org.apache.iotdb.commons.consensus.iotv2.consistency; - -import org.apache.iotdb.commons.concurrent.threadpool.ScheduledExecutorUtil; - -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -import java.util.Map; -import java.util.concurrent.ConcurrentHashMap; -import java.util.concurrent.ScheduledExecutorService; -import java.util.concurrent.ScheduledFuture; -import java.util.concurrent.TimeUnit; -import java.util.concurrent.atomic.AtomicBoolean; - -/** - * Background scheduler for periodic consistency checks across all consensus groups managed by this - * DataNode. Integrates with IoTV2GlobalComponentContainer's background task service. - * - *

Features: - * - *

    - *
  • Configurable check interval (default: 1 hour) - *
  • Rate limiting to prevent resource saturation - *
  • Per-region tracking to avoid overlapping checks - *
  • SyncLag pre-check gate (only checks regions with completed replication) - *
- */ -public class ConsistencyCheckScheduler { - - private static final Logger LOGGER = LoggerFactory.getLogger(ConsistencyCheckScheduler.class); - - private static final long DEFAULT_CHECK_INTERVAL_MS = 3_600_000L; // 1 hour - private static final long DEFAULT_INITIAL_DELAY_MS = 300_000L; // 5 minutes - - /** Callback interface for triggering consistency checks. */ - public interface ConsistencyCheckTrigger { - /** - * Trigger a consistency check for a specific consensus group. - * - * @param consensusGroupId the group to check - * @return true if the check was successfully initiated - */ - boolean triggerCheck(String consensusGroupId); - - /** - * Check if replication is complete for a consensus group. - * - * @param consensusGroupId the group to check - * @return true if syncLag <= 0 - */ - boolean isReplicationComplete(String consensusGroupId); - } - - private final ScheduledExecutorService executorService; - private final ConsistencyCheckTrigger trigger; - private final long checkIntervalMs; - private final long initialDelayMs; - private final AtomicBoolean running; - private ScheduledFuture scheduledFuture; - private final ConcurrentHashMap lastCheckTimes; - private final ConcurrentHashMap activeChecks; - - public ConsistencyCheckScheduler( - ScheduledExecutorService executorService, ConsistencyCheckTrigger trigger) { - this(executorService, trigger, DEFAULT_CHECK_INTERVAL_MS, DEFAULT_INITIAL_DELAY_MS); - } - - public ConsistencyCheckScheduler( - ScheduledExecutorService executorService, - ConsistencyCheckTrigger trigger, - long checkIntervalMs, - long initialDelayMs) { - this.executorService = executorService; - this.trigger = trigger; - this.checkIntervalMs = checkIntervalMs; - this.initialDelayMs = initialDelayMs; - this.running = new AtomicBoolean(false); - this.lastCheckTimes = new ConcurrentHashMap<>(); - this.activeChecks = new ConcurrentHashMap<>(); - } - - /** Start the periodic scheduling. */ - public void start() { - if (!running.compareAndSet(false, true)) { - LOGGER.warn("ConsistencyCheckScheduler already running"); - return; - } - scheduledFuture = - ScheduledExecutorUtil.safelyScheduleWithFixedDelay( - executorService, - this::runChecks, - initialDelayMs, - checkIntervalMs, - TimeUnit.MILLISECONDS); - LOGGER.info( - "ConsistencyCheckScheduler started: initialDelay={}ms, interval={}ms", - initialDelayMs, - checkIntervalMs); - } - - /** Stop the scheduler gracefully. */ - public void stop() { - if (!running.compareAndSet(true, false)) { - return; - } - if (scheduledFuture != null) { - scheduledFuture.cancel(false); - } - LOGGER.info("ConsistencyCheckScheduler stopped"); - } - - /** Register a consensus group for periodic checking. */ - public void registerGroup(String consensusGroupId) { - lastCheckTimes.putIfAbsent(consensusGroupId, 0L); - activeChecks.putIfAbsent(consensusGroupId, false); - LOGGER.debug("Registered consensus group {} for consistency checks", consensusGroupId); - } - - /** Unregister a consensus group. */ - public void unregisterGroup(String consensusGroupId) { - lastCheckTimes.remove(consensusGroupId); - activeChecks.remove(consensusGroupId); - } - - /** Trigger an immediate check for a specific group (e.g., from CLI). */ - public boolean triggerManualCheck(String consensusGroupId) { - if (activeChecks.getOrDefault(consensusGroupId, false)) { - LOGGER.info("Check already active for group {}, skipping", consensusGroupId); - return false; - } - return checkGroup(consensusGroupId); - } - - private void runChecks() { - if (!running.get()) { - return; - } - - long now = System.currentTimeMillis(); - for (Map.Entry entry : lastCheckTimes.entrySet()) { - String groupId = entry.getKey(); - long lastCheck = entry.getValue(); - - // Rate limiting: skip if checked recently - if (now - lastCheck < checkIntervalMs) { - continue; - } - - // Skip if a check is already active - if (activeChecks.getOrDefault(groupId, false)) { - continue; - } - - checkGroup(groupId); - } - } - - private boolean checkGroup(String consensusGroupId) { - try { - // SyncLag pre-check - if (!trigger.isReplicationComplete(consensusGroupId)) { - LOGGER.debug( - "Skipping consistency check for group {}: replication in progress", consensusGroupId); - return false; - } - - activeChecks.put(consensusGroupId, true); - lastCheckTimes.put(consensusGroupId, System.currentTimeMillis()); - - boolean triggered = trigger.triggerCheck(consensusGroupId); - if (triggered) { - LOGGER.info("Triggered consistency check for group {}", consensusGroupId); - } - return triggered; - } catch (Exception e) { - LOGGER.error( - "Error triggering consistency check for group {}: {}", - consensusGroupId, - e.getMessage(), - e); - return false; - } finally { - activeChecks.put(consensusGroupId, false); - } - } - - public boolean isRunning() { - return running.get(); - } - - public long getCheckIntervalMs() { - return checkIntervalMs; - } - - public int getRegisteredGroupCount() { - return lastCheckTimes.size(); - } -} diff --git a/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/consensus/iotv2/consistency/ConsistencyMerkleTree.java b/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/consensus/iotv2/consistency/ConsistencyMerkleTree.java deleted file mode 100644 index 35e7935abe9c6..0000000000000 --- a/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/consensus/iotv2/consistency/ConsistencyMerkleTree.java +++ /dev/null @@ -1,242 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, - * software distributed under the License is distributed on an - * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY - * KIND, either express or implied. See the License for the - * specific language governing permissions and limitations - * under the License. - */ - -package org.apache.iotdb.commons.consensus.iotv2.consistency; - -import java.util.ArrayList; -import java.util.Collections; -import java.util.List; -import java.util.Map; -import java.util.TreeMap; -import java.util.concurrent.locks.ReadWriteLock; -import java.util.concurrent.locks.ReentrantReadWriteLock; - -/** - * In-memory incremental Merkle tree that holds Level 0 (region digest) and Level 1 (per-partition - * digest) as resident data. Level 2+ (device/measurement/timeBucket) are loaded on-demand from - * .merkle files. Supports dual-digest aggregation and lazy invalidation for deletions. - */ -public class ConsistencyMerkleTree { - - private volatile DualDigest regionDigest; - private final TreeMap partitions; - private final ReadWriteLock treeLock; - - public ConsistencyMerkleTree() { - this.regionDigest = DualDigest.ZERO; - this.partitions = new TreeMap<>(); - this.treeLock = new ReentrantReadWriteLock(); - } - - /** - * Get or create the node for a given partition. Lazily initializes partition nodes on first - * encounter. - */ - public TimePartitionMerkleNode getOrCreatePartitionNode(long partitionId) { - treeLock.readLock().lock(); - try { - TimePartitionMerkleNode node = partitions.get(partitionId); - if (node != null) { - return node; - } - } finally { - treeLock.readLock().unlock(); - } - - treeLock.writeLock().lock(); - try { - return partitions.computeIfAbsent(partitionId, TimePartitionMerkleNode::new); - } finally { - treeLock.writeLock().unlock(); - } - } - - public TimePartitionMerkleNode getPartitionNode(long partitionId) { - treeLock.readLock().lock(); - try { - return partitions.get(partitionId); - } finally { - treeLock.readLock().unlock(); - } - } - - /** - * Called after a TsFile flush to incorporate the new file's hash into the appropriate partition's - * dual-digest. - */ - public void onTsFileFlushed(long partitionId, long fileRootHash) { - onTsFileFlushed(partitionId, DualDigest.fromSingleHash(fileRootHash)); - } - - public void onTsFileFlushed(long partitionId, DualDigest fileDigest) { - TimePartitionMerkleNode node = getOrCreatePartitionNode(partitionId); - node.addDigest(fileDigest); - recomputeRegionDigest(); - } - - /** - * Called during compaction when source TsFiles are removed and a target TsFile is created. - * - * @param partitionId the partition being compacted - * @param sourceFileRootHashes root hashes of the source TsFiles being removed - * @param targetFileRootHash root hash of the newly created target TsFile - */ - public void onCompaction( - long partitionId, List sourceFileRootHashes, long targetFileRootHash) { - onCompaction( - partitionId, - sourceFileRootHashes.stream() - .map(DualDigest::fromSingleHash) - .collect(java.util.stream.Collectors.toList()), - Collections.singletonList(DualDigest.fromSingleHash(targetFileRootHash))); - } - - public void onCompaction( - long partitionId, List sourceDigests, List targetDigests) { - TimePartitionMerkleNode node = getOrCreatePartitionNode(partitionId); - for (DualDigest sourceDigest : sourceDigests) { - node.removeDigest(sourceDigest); - } - for (DualDigest targetDigest : targetDigests) { - node.addDigest(targetDigest); - } - recomputeRegionDigest(); - } - - /** - * Overloaded compaction hook accepting pre-collected source hashes and a combined target hash. - * - * @param sourceHashes root hashes of source TsFiles being removed - * @param targetCombinedHash XOR of all target TsFile root hashes - * @param partitionId the partition being compacted - */ - public void onCompaction(List sourceHashes, long targetCombinedHash, long partitionId) { - onCompaction( - partitionId, - sourceHashes.stream() - .map(DualDigest::fromSingleHash) - .collect(java.util.stream.Collectors.toList()), - Collections.singletonList(DualDigest.fromSingleHash(targetCombinedHash))); - } - - public void onCompaction( - List sourceDigests, List targetDigests, long partitionId) { - TimePartitionMerkleNode node = getOrCreatePartitionNode(partitionId); - for (DualDigest sourceDigest : sourceDigests) { - node.removeDigest(sourceDigest); - } - for (DualDigest targetDigest : targetDigests) { - node.addDigest(targetDigest); - } - recomputeRegionDigest(); - } - - /** - * Mark a partition as dirty when deletions affect it. The next consistency check cycle will - * trigger a full rescan and rebuild for this partition. - */ - public void markPartitionDirty(long partitionId) { - TimePartitionMerkleNode node = getOrCreatePartitionNode(partitionId); - node.markDirty(); - } - - /** - * Prune a verified cold partition from the in-memory tree. Called after successful verification - * and commit to reduce memory footprint. - */ - public void prunePartition(long partitionId) { - treeLock.writeLock().lock(); - try { - partitions.remove(partitionId); - recomputeRegionDigestLocked(); - } finally { - treeLock.writeLock().unlock(); - } - } - - /** Get the current region-level dual-digest (Level 0). */ - public DualDigest getRegionDigest() { - return regionDigest; - } - - /** Return all partition IDs within the given time range [startInclusive, endExclusive). */ - public List getPartitionIds(long startInclusive, long endExclusive) { - treeLock.readLock().lock(); - try { - return new ArrayList<>(partitions.subMap(startInclusive, endExclusive).keySet()); - } finally { - treeLock.readLock().unlock(); - } - } - - /** Return all partition IDs in the tree. */ - public List getAllPartitionIds() { - treeLock.readLock().lock(); - try { - return new ArrayList<>(partitions.keySet()); - } finally { - treeLock.readLock().unlock(); - } - } - - /** Return an unmodifiable view of dirty partitions. */ - public List getDirtyPartitionIds() { - treeLock.readLock().lock(); - try { - List dirtyIds = new ArrayList<>(); - for (Map.Entry entry : partitions.entrySet()) { - if (entry.getValue().isDirty()) { - dirtyIds.add(entry.getKey()); - } - } - return Collections.unmodifiableList(dirtyIds); - } finally { - treeLock.readLock().unlock(); - } - } - - /** Recompute Level-0 region digest from all partition digests. */ - private void recomputeRegionDigest() { - treeLock.readLock().lock(); - try { - recomputeRegionDigestLocked(); - } finally { - treeLock.readLock().unlock(); - } - } - - /** Must be called while holding at least a read lock on treeLock. */ - private void recomputeRegionDigestLocked() { - DualDigest digest = DualDigest.ZERO; - for (TimePartitionMerkleNode node : partitions.values()) { - digest = digest.merge(node.getPartitionDigest()); - } - this.regionDigest = digest; - } - - /** Return the number of tracked partitions. */ - public int getPartitionCount() { - treeLock.readLock().lock(); - try { - return partitions.size(); - } finally { - treeLock.readLock().unlock(); - } - } -} diff --git a/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/consensus/iotv2/consistency/ConsistencyViewBuilder.java b/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/consensus/iotv2/consistency/ConsistencyViewBuilder.java deleted file mode 100644 index 9076fc5406176..0000000000000 --- a/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/consensus/iotv2/consistency/ConsistencyViewBuilder.java +++ /dev/null @@ -1,133 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, - * software distributed under the License is distributed on an - * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY - * KIND, either express or implied. See the License for the - * specific language governing permissions and limitations - * under the License. - */ - -package org.apache.iotdb.commons.consensus.iotv2.consistency; - -import org.apache.iotdb.commons.consensus.iotv2.consistency.merkle.MerkleFileCache; -import org.apache.iotdb.commons.consensus.iotv2.consistency.merkle.MerkleFileContent; - -import java.io.IOException; -import java.util.ArrayList; -import java.util.List; - -/** - * Builds a consistency view for a partition using the snapshot-then-release pattern: acquires a - * short read lock to snapshot the TsFile list, then releases the lock before performing heavy - * .merkle file I/O. Includes staleness detection via version counter. - */ -public class ConsistencyViewBuilder { - - /** - * Interface to decouple from TsFileManager; allows the builder to be used without direct - * dependency on the storage engine module. - */ - public interface TsFileListProvider { - - /** Acquire a read lock on the TsFile resource list. */ - void readLock(); - - /** Release the read lock. */ - void readUnlock(); - - /** - * Get all TsFile paths for the given partition. - * - * @param partitionId the time partition to query - * @return list of TsFile paths (both sequence and unsequence) - */ - List getTsFileSnapshots(long partitionId); - } - - /** Lightweight snapshot of a TsFile reference for off-lock processing. */ - public static class TsFileSnapshot { - private final String tsFilePath; - private final boolean deleted; - - public TsFileSnapshot(String tsFilePath, boolean deleted) { - this.tsFilePath = tsFilePath; - this.deleted = deleted; - } - - public String getTsFilePath() { - return tsFilePath; - } - - public boolean isDeleted() { - return deleted; - } - } - - private final MerkleFileCache merkleFileCache; - - public ConsistencyViewBuilder(MerkleFileCache merkleFileCache) { - this.merkleFileCache = merkleFileCache; - } - - /** - * Build a consistency view for a specific partition. - * - * @param provider the TsFile list provider (wraps TsFileManager) - * @param partitionNode the partition's Merkle node for staleness checks - * @return list of loaded MerkleFileContent for all active TsFiles in the partition - * @throws StaleSnapshotException if the partition was modified while building the view - * @throws IOException on .merkle file read failure - */ - public List buildView( - TsFileListProvider provider, TimePartitionMerkleNode partitionNode) - throws IOException, StaleSnapshotException { - long expectedVersion = partitionNode.getVersion(); - - // Step 1: Short read lock -- snapshot the TsFile list (microseconds) - List snapshot; - provider.readLock(); - try { - snapshot = new ArrayList<>(provider.getTsFileSnapshots(partitionNode.getPartitionId())); - } finally { - provider.readUnlock(); - } - - // Step 2: Off-lock -- heavy I/O to load .merkle files (milliseconds-seconds) - List contents = new ArrayList<>(); - for (TsFileSnapshot tsFile : snapshot) { - if (tsFile.isDeleted()) { - continue; - } - try { - MerkleFileContent content = merkleFileCache.get(tsFile.getTsFilePath()); - contents.add(content); - } catch (IOException e) { - // .merkle file may have been cleaned up by compaction; skip this file - if (!tsFile.isDeleted()) { - throw e; - } - } - } - - // Step 3: Staleness check -- verify snapshot is still valid - long currentVersion = partitionNode.getVersion(); - if (currentVersion != expectedVersion) { - throw new StaleSnapshotException( - String.format( - "Partition %d modified during view build (expected version=%d, current=%d), retry", - partitionNode.getPartitionId(), expectedVersion, currentVersion)); - } - - return contents; - } -} diff --git a/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/consensus/iotv2/consistency/LogicalMismatchScope.java b/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/consensus/iotv2/consistency/LogicalMismatchScope.java new file mode 100644 index 0000000000000..f9f4c8921caef --- /dev/null +++ b/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/consensus/iotv2/consistency/LogicalMismatchScope.java @@ -0,0 +1,273 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.iotdb.commons.consensus.iotv2.consistency; + +import java.nio.charset.StandardCharsets; +import java.util.ArrayList; +import java.util.Base64; +import java.util.Collection; +import java.util.Collections; +import java.util.Comparator; +import java.util.LinkedHashSet; +import java.util.List; +import java.util.Objects; +import java.util.stream.Collectors; + +/** + * Durable partition-scoped mismatch description produced by logical snapshot compare. + * + *

The persisted string keeps backward compatibility with the legacy {@code treeKind@leafId} + * format while allowing the new logical-snapshot workflow to optionally carry a narrowed key range + * for each mismatched leaf. + */ +public final class LogicalMismatchScope { + + private static final String ENTRY_SEPARATOR = ","; + private static final String TOKEN_SEPARATOR = "@"; + + private LogicalMismatchScope() {} + + public enum RepairDirective { + REPAIRABLE, + FOLLOWER_EXTRA_TOMBSTONE + } + + public static String serialize(Collection scopes) { + if (scopes == null || scopes.isEmpty()) { + return null; + } + return scopes.stream() + .filter(Objects::nonNull) + .distinct() + .sorted(Comparator.comparing(Scope::toPersistentString)) + .map(Scope::toPersistentString) + .collect(Collectors.joining(ENTRY_SEPARATOR)); + } + + public static List deserialize(String serialized) { + if (serialized == null || serialized.trim().isEmpty()) { + return Collections.emptyList(); + } + + LinkedHashSet scopes = new LinkedHashSet<>(); + for (String rawEntry : serialized.split(ENTRY_SEPARATOR)) { + String entry = rawEntry.trim(); + if (entry.isEmpty()) { + continue; + } + String[] tokens = entry.split(TOKEN_SEPARATOR, -1); + if (tokens.length < 2) { + continue; + } + String treeKind = tokens[0]; + String leafId = tokens[1]; + String keyRangeStart = tokens.length >= 3 ? decodeNullable(tokens[2]) : null; + String keyRangeEnd = tokens.length >= 4 ? decodeNullable(tokens[3]) : null; + List exactKeys = + tokens.length >= 5 ? decodeStringList(tokens[4]) : Collections.emptyList(); + RepairDirective repairDirective = + tokens.length >= 6 ? decodeRepairDirective(tokens[5]) : RepairDirective.REPAIRABLE; + scopes.add( + new Scope(treeKind, leafId, keyRangeStart, keyRangeEnd, exactKeys, repairDirective)); + } + return new ArrayList<>(scopes); + } + + private static String encodeNullable(String value) { + if (value == null) { + return ""; + } + return Base64.getUrlEncoder() + .withoutPadding() + .encodeToString(value.getBytes(StandardCharsets.UTF_8)); + } + + private static String decodeNullable(String value) { + if (value == null || value.isEmpty()) { + return null; + } + return new String(Base64.getUrlDecoder().decode(value), StandardCharsets.UTF_8); + } + + private static String encodeStringList(List values) { + if (values == null || values.isEmpty()) { + return ""; + } + return encodeNullable(String.join("\n", values)); + } + + private static List decodeStringList(String value) { + String decoded = decodeNullable(value); + if (decoded == null || decoded.isEmpty()) { + return Collections.emptyList(); + } + LinkedHashSet result = new LinkedHashSet<>(); + for (String key : decoded.split("\n")) { + if (!key.isEmpty()) { + result.add(key); + } + } + return new ArrayList<>(result); + } + + private static RepairDirective decodeRepairDirective(String value) { + String decoded = decodeNullable(value); + if (decoded == null || decoded.isEmpty()) { + return RepairDirective.REPAIRABLE; + } + try { + return RepairDirective.valueOf(decoded); + } catch (IllegalArgumentException ignored) { + return RepairDirective.REPAIRABLE; + } + } + + public static final class Scope { + private final String treeKind; + private final String leafId; + private final String keyRangeStart; + private final String keyRangeEnd; + private final List exactKeys; + private final RepairDirective repairDirective; + + public Scope(String treeKind, String leafId) { + this(treeKind, leafId, null, null, Collections.emptyList(), RepairDirective.REPAIRABLE); + } + + public Scope(String treeKind, String leafId, String keyRangeStart, String keyRangeEnd) { + this( + treeKind, + leafId, + keyRangeStart, + keyRangeEnd, + Collections.emptyList(), + RepairDirective.REPAIRABLE); + } + + public Scope( + String treeKind, + String leafId, + String keyRangeStart, + String keyRangeEnd, + List exactKeys) { + this(treeKind, leafId, keyRangeStart, keyRangeEnd, exactKeys, RepairDirective.REPAIRABLE); + } + + public Scope( + String treeKind, + String leafId, + String keyRangeStart, + String keyRangeEnd, + List exactKeys, + RepairDirective repairDirective) { + this.treeKind = treeKind; + this.leafId = leafId; + this.keyRangeStart = keyRangeStart; + this.keyRangeEnd = keyRangeEnd; + this.exactKeys = + exactKeys == null + ? Collections.emptyList() + : Collections.unmodifiableList( + exactKeys.stream() + .filter(Objects::nonNull) + .distinct() + .collect(Collectors.toList())); + this.repairDirective = repairDirective == null ? RepairDirective.REPAIRABLE : repairDirective; + } + + public String getTreeKind() { + return treeKind; + } + + public String getLeafId() { + return leafId; + } + + public String getKeyRangeStart() { + return keyRangeStart; + } + + public String getKeyRangeEnd() { + return keyRangeEnd; + } + + public List getExactKeys() { + return exactKeys; + } + + public RepairDirective getRepairDirective() { + return repairDirective; + } + + public boolean isRepairable() { + return repairDirective == RepairDirective.REPAIRABLE; + } + + public String toPersistentString() { + if (keyRangeStart == null + && keyRangeEnd == null + && exactKeys.isEmpty() + && repairDirective == RepairDirective.REPAIRABLE) { + return treeKind + TOKEN_SEPARATOR + leafId; + } + String base = + treeKind + + TOKEN_SEPARATOR + + leafId + + TOKEN_SEPARATOR + + encodeNullable(keyRangeStart) + + TOKEN_SEPARATOR + + encodeNullable(keyRangeEnd) + + TOKEN_SEPARATOR + + encodeStringList(exactKeys); + if (repairDirective == RepairDirective.REPAIRABLE) { + return base; + } + return base + TOKEN_SEPARATOR + encodeNullable(repairDirective.name()); + } + + @Override + public String toString() { + return toPersistentString(); + } + + @Override + public boolean equals(Object object) { + if (this == object) { + return true; + } + if (!(object instanceof Scope)) { + return false; + } + Scope that = (Scope) object; + return Objects.equals(treeKind, that.treeKind) + && Objects.equals(leafId, that.leafId) + && Objects.equals(keyRangeStart, that.keyRangeStart) + && Objects.equals(keyRangeEnd, that.keyRangeEnd) + && Objects.equals(exactKeys, that.exactKeys) + && repairDirective == that.repairDirective; + } + + @Override + public int hashCode() { + return Objects.hash(treeKind, leafId, keyRangeStart, keyRangeEnd, exactKeys, repairDirective); + } + } +} diff --git a/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/consensus/iotv2/consistency/RepairProgressTable.java b/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/consensus/iotv2/consistency/RepairProgressTable.java index 2fcaf948fc1e5..3f40c0f1aaa5f 100644 --- a/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/consensus/iotv2/consistency/RepairProgressTable.java +++ b/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/consensus/iotv2/consistency/RepairProgressTable.java @@ -19,109 +19,487 @@ package org.apache.iotdb.commons.consensus.iotv2.consistency; -import java.util.Collection; +import org.apache.tsfile.utils.ReadWriteIOUtils; + +import java.io.IOException; +import java.io.InputStream; +import java.io.OutputStream; +import java.nio.ByteBuffer; +import java.util.ArrayList; +import java.util.Collections; +import java.util.List; import java.util.Map; +import java.util.Objects; import java.util.concurrent.ConcurrentHashMap; /** - * In-memory representation of the partition-level repair progress tracking for a single consensus - * group. This serves as the runtime cache backed by the __system.repair_progress system table. - * - *

Schema: - * - *

    - *
  • Region-level row: (consensus_group_id, global_repaired_watermark, last_repair_time, - * repair_status) - *
  • Partition-level rows: (consensus_group_id, partition_id, repaired_to, status, - * last_failure_reason) - *
+ * In-memory representation of the partition-scoped consistency check / repair progress for a single + * consensus group. */ public class RepairProgressTable { - /** Status of repair for a single partition. */ - public enum PartitionRepairStatus { + public enum CheckState { PENDING, + DIRTY, VERIFIED, - FAILED, - DIRTY + MISMATCH, + FAILED } - /** Status of the overall repair cycle for a region. */ - public enum RegionRepairStatus { + public enum RepairState { IDLE, + PENDING, RUNNING, + SUCCEEDED, + FAILED + } + + public enum SnapshotState { + PENDING, + BUILDING, + READY, + DIRTY, FAILED } - /** Progress entry for a single partition. */ public static class PartitionProgress { private final long partitionId; - private volatile long repairedTo; - private volatile PartitionRepairStatus status; - private volatile String lastFailureReason; + private volatile CheckState checkState; + private volatile RepairState repairState; + private volatile long lastCheckedAt; + private volatile long lastSafeWatermark; + private volatile long partitionMutationEpoch; + private volatile long snapshotEpoch; + private volatile SnapshotState snapshotState; + private volatile long lastMismatchAt; + private volatile String mismatchScopeRef; + private volatile int mismatchLeafCount; + private volatile String repairEpoch; + private volatile String replicaObservationToken; + private volatile String lastErrorCode; + private volatile String lastErrorMessage; public PartitionProgress(long partitionId) { this.partitionId = partitionId; - this.repairedTo = 0L; - this.status = PartitionRepairStatus.PENDING; - this.lastFailureReason = null; + this.checkState = CheckState.PENDING; + this.repairState = RepairState.IDLE; + this.lastCheckedAt = 0L; + this.lastSafeWatermark = Long.MIN_VALUE; + this.partitionMutationEpoch = Long.MIN_VALUE; + this.snapshotEpoch = Long.MIN_VALUE; + this.snapshotState = SnapshotState.PENDING; + this.lastMismatchAt = 0L; + this.mismatchScopeRef = null; + this.mismatchLeafCount = 0; + this.repairEpoch = null; + this.replicaObservationToken = null; + this.lastErrorCode = null; + this.lastErrorMessage = null; } public long getPartitionId() { return partitionId; } - public long getRepairedTo() { - return repairedTo; + public CheckState getCheckState() { + return checkState; + } + + public RepairState getRepairState() { + return repairState; + } + + public long getLastCheckedAt() { + return lastCheckedAt; + } + + public long getLastSafeWatermark() { + return lastSafeWatermark; + } + + public long getPartitionMutationEpoch() { + return partitionMutationEpoch; + } + + public long getSnapshotEpoch() { + return snapshotEpoch; + } + + public SnapshotState getSnapshotState() { + return snapshotState; + } + + public long getLastMismatchAt() { + return lastMismatchAt; + } + + public String getMismatchScopeRef() { + return mismatchScopeRef; + } + + public int getMismatchLeafCount() { + return mismatchLeafCount; } - public void setRepairedTo(long repairedTo) { - this.repairedTo = repairedTo; + public String getRepairEpoch() { + return repairEpoch; } - public PartitionRepairStatus getStatus() { - return status; + public String getReplicaObservationToken() { + return replicaObservationToken; } - public void setStatus(PartitionRepairStatus status) { - this.status = status; + public String getLastErrorCode() { + return lastErrorCode; } - public String getLastFailureReason() { - return lastFailureReason; + public String getLastErrorMessage() { + return lastErrorMessage; } - public void markVerified(long newRepairedTo) { - this.repairedTo = newRepairedTo; - this.status = PartitionRepairStatus.VERIFIED; - this.lastFailureReason = null; + public boolean shouldCheck( + long candidatePartitionMutationEpoch, + long candidateSnapshotEpoch, + SnapshotState candidateSnapshotState) { + return shouldCheck( + candidatePartitionMutationEpoch, candidateSnapshotEpoch, candidateSnapshotState, null); } - public void markFailed(String reason) { - this.status = PartitionRepairStatus.FAILED; - this.lastFailureReason = reason; + public boolean shouldCheck( + long candidatePartitionMutationEpoch, + long candidateSnapshotEpoch, + SnapshotState candidateSnapshotState, + String candidateReplicaObservationToken) { + return checkState != CheckState.VERIFIED + || partitionMutationEpoch != candidatePartitionMutationEpoch + || snapshotEpoch != candidateSnapshotEpoch + || snapshotState != candidateSnapshotState + || !Objects.equals(replicaObservationToken, candidateReplicaObservationToken); + } + + public void markVerified( + long checkedAt, + long safeWatermark, + long newPartitionMutationEpoch, + long newSnapshotEpoch, + SnapshotState newSnapshotState) { + markVerified( + checkedAt, + safeWatermark, + newPartitionMutationEpoch, + newSnapshotEpoch, + newSnapshotState, + null); + } + + public void markVerified( + long checkedAt, + long safeWatermark, + long newPartitionMutationEpoch, + long newSnapshotEpoch, + SnapshotState newSnapshotState, + String newReplicaObservationToken) { + checkState = CheckState.VERIFIED; + lastCheckedAt = checkedAt; + lastSafeWatermark = safeWatermark; + partitionMutationEpoch = newPartitionMutationEpoch; + snapshotEpoch = newSnapshotEpoch; + snapshotState = newSnapshotState; + replicaObservationToken = newReplicaObservationToken; + mismatchScopeRef = null; + mismatchLeafCount = 0; + clearError(); + if (repairState == RepairState.RUNNING + || repairState == RepairState.PENDING + || repairState == RepairState.FAILED) { + repairState = RepairState.SUCCEEDED; + } + } + + public void markMismatch( + long checkedAt, + long safeWatermark, + long newPartitionMutationEpoch, + long newSnapshotEpoch, + SnapshotState newSnapshotState, + String newMismatchScopeRef, + int newMismatchLeafCount, + String newRepairEpoch) { + markMismatch( + checkedAt, + safeWatermark, + newPartitionMutationEpoch, + newSnapshotEpoch, + newSnapshotState, + newMismatchScopeRef, + newMismatchLeafCount, + newRepairEpoch, + null); + } + + public void markMismatch( + long checkedAt, + long safeWatermark, + long newPartitionMutationEpoch, + long newSnapshotEpoch, + SnapshotState newSnapshotState, + String newMismatchScopeRef, + int newMismatchLeafCount, + String newRepairEpoch, + String newReplicaObservationToken) { + checkState = CheckState.MISMATCH; + repairState = RepairState.PENDING; + lastCheckedAt = checkedAt; + lastSafeWatermark = safeWatermark; + partitionMutationEpoch = newPartitionMutationEpoch; + snapshotEpoch = newSnapshotEpoch; + snapshotState = newSnapshotState; + lastMismatchAt = checkedAt; + mismatchScopeRef = newMismatchScopeRef; + mismatchLeafCount = newMismatchLeafCount; + repairEpoch = newRepairEpoch; + replicaObservationToken = newReplicaObservationToken; + clearError(); + } + + public void markCheckFailed( + long checkedAt, + long safeWatermark, + long newPartitionMutationEpoch, + long newSnapshotEpoch, + SnapshotState newSnapshotState, + String errorCode, + String errorMessage) { + markCheckFailed( + checkedAt, + safeWatermark, + newPartitionMutationEpoch, + newSnapshotEpoch, + newSnapshotState, + errorCode, + errorMessage, + null); + } + + public void markCheckFailed( + long checkedAt, + long safeWatermark, + long newPartitionMutationEpoch, + long newSnapshotEpoch, + SnapshotState newSnapshotState, + String errorCode, + String errorMessage, + String newReplicaObservationToken) { + checkState = CheckState.FAILED; + lastCheckedAt = checkedAt; + lastSafeWatermark = safeWatermark; + partitionMutationEpoch = newPartitionMutationEpoch; + snapshotEpoch = newSnapshotEpoch; + snapshotState = newSnapshotState; + replicaObservationToken = newReplicaObservationToken; + lastErrorCode = errorCode; + lastErrorMessage = errorMessage; } public void markDirty() { - this.status = PartitionRepairStatus.DIRTY; + checkState = CheckState.DIRTY; + snapshotState = SnapshotState.DIRTY; + mismatchScopeRef = null; + mismatchLeafCount = 0; + if (repairState == RepairState.SUCCEEDED) { + repairState = RepairState.IDLE; + } + } + + public void markRepairRunning(String newRepairEpoch) { + repairState = RepairState.RUNNING; + repairEpoch = newRepairEpoch; + clearError(); + } + + public void markRepairSucceeded( + long checkedAt, + long safeWatermark, + long newPartitionMutationEpoch, + long newSnapshotEpoch, + SnapshotState newSnapshotState, + String newRepairEpoch) { + markRepairSucceeded( + checkedAt, + safeWatermark, + newPartitionMutationEpoch, + newSnapshotEpoch, + newSnapshotState, + newRepairEpoch, + null); + } + + public void markRepairSucceeded( + long checkedAt, + long safeWatermark, + long newPartitionMutationEpoch, + long newSnapshotEpoch, + SnapshotState newSnapshotState, + String newRepairEpoch, + String newReplicaObservationToken) { + repairState = RepairState.SUCCEEDED; + repairEpoch = newRepairEpoch; + markVerified( + checkedAt, + safeWatermark, + newPartitionMutationEpoch, + newSnapshotEpoch, + newSnapshotState, + newReplicaObservationToken); + repairState = RepairState.SUCCEEDED; + } + + public void markRepairFailed(String newRepairEpoch, String errorCode, String errorMessage) { + repairState = RepairState.FAILED; + repairEpoch = newRepairEpoch; + lastErrorCode = errorCode; + lastErrorMessage = errorMessage; + } + + public PartitionProgress copy() { + PartitionProgress copy = new PartitionProgress(partitionId); + copy.checkState = checkState; + copy.repairState = repairState; + copy.lastCheckedAt = lastCheckedAt; + copy.lastSafeWatermark = lastSafeWatermark; + copy.partitionMutationEpoch = partitionMutationEpoch; + copy.snapshotEpoch = snapshotEpoch; + copy.snapshotState = snapshotState; + copy.lastMismatchAt = lastMismatchAt; + copy.mismatchScopeRef = mismatchScopeRef; + copy.mismatchLeafCount = mismatchLeafCount; + copy.repairEpoch = repairEpoch; + copy.replicaObservationToken = replicaObservationToken; + copy.lastErrorCode = lastErrorCode; + copy.lastErrorMessage = lastErrorMessage; + return copy; + } + + private void clearError() { + lastErrorCode = null; + lastErrorMessage = null; + } + + private void serialize(OutputStream outputStream) throws IOException { + ReadWriteIOUtils.write(partitionId, outputStream); + ReadWriteIOUtils.write(checkState.ordinal(), outputStream); + ReadWriteIOUtils.write(repairState.ordinal(), outputStream); + ReadWriteIOUtils.write(lastCheckedAt, outputStream); + ReadWriteIOUtils.write(lastSafeWatermark, outputStream); + ReadWriteIOUtils.write(partitionMutationEpoch, outputStream); + ReadWriteIOUtils.write(snapshotEpoch, outputStream); + ReadWriteIOUtils.write(snapshotState.ordinal(), outputStream); + ReadWriteIOUtils.write(lastMismatchAt, outputStream); + ReadWriteIOUtils.write(mismatchScopeRef, outputStream); + ReadWriteIOUtils.write(mismatchLeafCount, outputStream); + ReadWriteIOUtils.write(repairEpoch, outputStream); + ReadWriteIOUtils.write(replicaObservationToken, outputStream); + ReadWriteIOUtils.write(lastErrorCode, outputStream); + ReadWriteIOUtils.write(lastErrorMessage, outputStream); + } + + private static PartitionProgress deserialize(InputStream inputStream) throws IOException { + PartitionProgress progress = new PartitionProgress(ReadWriteIOUtils.readLong(inputStream)); + progress.checkState = CheckState.values()[ReadWriteIOUtils.readInt(inputStream)]; + progress.repairState = RepairState.values()[ReadWriteIOUtils.readInt(inputStream)]; + progress.lastCheckedAt = ReadWriteIOUtils.readLong(inputStream); + progress.lastSafeWatermark = ReadWriteIOUtils.readLong(inputStream); + progress.partitionMutationEpoch = ReadWriteIOUtils.readLong(inputStream); + progress.snapshotEpoch = ReadWriteIOUtils.readLong(inputStream); + progress.snapshotState = SnapshotState.values()[ReadWriteIOUtils.readInt(inputStream)]; + progress.lastMismatchAt = ReadWriteIOUtils.readLong(inputStream); + progress.mismatchScopeRef = ReadWriteIOUtils.readString(inputStream); + progress.mismatchLeafCount = ReadWriteIOUtils.readInt(inputStream); + progress.repairEpoch = ReadWriteIOUtils.readString(inputStream); + progress.replicaObservationToken = ReadWriteIOUtils.readString(inputStream); + progress.lastErrorCode = ReadWriteIOUtils.readString(inputStream); + progress.lastErrorMessage = ReadWriteIOUtils.readString(inputStream); + return progress; + } + + private static PartitionProgress deserialize(ByteBuffer byteBuffer) { + PartitionProgress progress = new PartitionProgress(ReadWriteIOUtils.readLong(byteBuffer)); + progress.checkState = CheckState.values()[ReadWriteIOUtils.readInt(byteBuffer)]; + progress.repairState = RepairState.values()[ReadWriteIOUtils.readInt(byteBuffer)]; + progress.lastCheckedAt = ReadWriteIOUtils.readLong(byteBuffer); + progress.lastSafeWatermark = ReadWriteIOUtils.readLong(byteBuffer); + progress.partitionMutationEpoch = ReadWriteIOUtils.readLong(byteBuffer); + progress.snapshotEpoch = ReadWriteIOUtils.readLong(byteBuffer); + progress.snapshotState = SnapshotState.values()[ReadWriteIOUtils.readInt(byteBuffer)]; + progress.lastMismatchAt = ReadWriteIOUtils.readLong(byteBuffer); + progress.mismatchScopeRef = ReadWriteIOUtils.readString(byteBuffer); + progress.mismatchLeafCount = ReadWriteIOUtils.readInt(byteBuffer); + progress.repairEpoch = ReadWriteIOUtils.readString(byteBuffer); + progress.replicaObservationToken = ReadWriteIOUtils.readString(byteBuffer); + progress.lastErrorCode = ReadWriteIOUtils.readString(byteBuffer); + progress.lastErrorMessage = ReadWriteIOUtils.readString(byteBuffer); + return progress; + } + + @Override + public boolean equals(Object object) { + if (this == object) { + return true; + } + if (!(object instanceof PartitionProgress)) { + return false; + } + PartitionProgress that = (PartitionProgress) object; + return partitionId == that.partitionId + && lastCheckedAt == that.lastCheckedAt + && lastSafeWatermark == that.lastSafeWatermark + && partitionMutationEpoch == that.partitionMutationEpoch + && snapshotEpoch == that.snapshotEpoch + && lastMismatchAt == that.lastMismatchAt + && mismatchLeafCount == that.mismatchLeafCount + && checkState == that.checkState + && repairState == that.repairState + && snapshotState == that.snapshotState + && Objects.equals(mismatchScopeRef, that.mismatchScopeRef) + && Objects.equals(repairEpoch, that.repairEpoch) + && Objects.equals(replicaObservationToken, that.replicaObservationToken) + && Objects.equals(lastErrorCode, that.lastErrorCode) + && Objects.equals(lastErrorMessage, that.lastErrorMessage); + } + + @Override + public int hashCode() { + return Objects.hash( + partitionId, + checkState, + repairState, + lastCheckedAt, + lastSafeWatermark, + partitionMutationEpoch, + snapshotEpoch, + snapshotState, + lastMismatchAt, + mismatchScopeRef, + mismatchLeafCount, + repairEpoch, + replicaObservationToken, + lastErrorCode, + lastErrorMessage); } } private final String consensusGroupId; - private volatile long globalRepairedWatermark; - private volatile long lastRepairTime; - private volatile RegionRepairStatus regionStatus; private final ConcurrentHashMap partitionProgress; public RepairProgressTable(String consensusGroupId) { this.consensusGroupId = consensusGroupId; - this.globalRepairedWatermark = 0L; - this.lastRepairTime = 0L; - this.regionStatus = RegionRepairStatus.IDLE; this.partitionProgress = new ConcurrentHashMap<>(); } - /** Get or create progress entry for a partition. */ + public String getConsensusGroupId() { + return consensusGroupId; + } + public PartitionProgress getOrCreatePartition(long partitionId) { return partitionProgress.computeIfAbsent(partitionId, PartitionProgress::new); } @@ -130,100 +508,246 @@ public PartitionProgress getPartition(long partitionId) { return partitionProgress.get(partitionId); } - /** - * Commit a partition as verified with a new repaired_to watermark. - * - * @param partitionId the partition to commit - * @param repairedTo the new watermark for this partition - */ - public void commitPartition(long partitionId, long repairedTo) { - PartitionProgress progress = getOrCreatePartition(partitionId); - progress.markVerified(repairedTo); - } - - /** - * Mark a partition as failed. - * - * @param partitionId the partition that failed - * @param reason the failure reason - */ - public void failPartition(long partitionId, String reason) { - PartitionProgress progress = getOrCreatePartition(partitionId); - progress.markFailed(reason); - } - - /** - * Mark a partition as dirty (modified after verification). - * - * @param partitionId the partition to mark dirty - */ - public void dirtyPartition(long partitionId) { - PartitionProgress progress = getOrCreatePartition(partitionId); - progress.markDirty(); - } - - /** - * Advance the global watermark. Rule: global_repaired_watermark = MIN(p.repaired_to for ALL - * partitions p in effective range). Only advances when every partition has been successfully - * verified. - * - * @return the new global watermark - */ - public long advanceGlobalWatermark() { - long minRepairedTo = Long.MAX_VALUE; - boolean hasPartitions = false; + public void markVerified( + long partitionId, + long checkedAt, + long safeWatermark, + long partitionMutationEpoch, + long snapshotEpoch, + SnapshotState snapshotState) { + markVerified( + partitionId, + checkedAt, + safeWatermark, + partitionMutationEpoch, + snapshotEpoch, + snapshotState, + null); + } - for (PartitionProgress progress : partitionProgress.values()) { - if (progress.getStatus() == PartitionRepairStatus.VERIFIED) { - hasPartitions = true; - minRepairedTo = Math.min(minRepairedTo, progress.getRepairedTo()); - } else if (progress.getStatus() == PartitionRepairStatus.FAILED - || progress.getStatus() == PartitionRepairStatus.DIRTY) { - // Failed/dirty partitions block watermark advancement - hasPartitions = true; - minRepairedTo = Math.min(minRepairedTo, progress.getRepairedTo()); - } - } + public void markVerified( + long partitionId, + long checkedAt, + long safeWatermark, + long partitionMutationEpoch, + long snapshotEpoch, + SnapshotState snapshotState, + String replicaObservationToken) { + getOrCreatePartition(partitionId) + .markVerified( + checkedAt, + safeWatermark, + partitionMutationEpoch, + snapshotEpoch, + snapshotState, + replicaObservationToken); + } - if (hasPartitions && minRepairedTo != Long.MAX_VALUE) { - this.globalRepairedWatermark = minRepairedTo; - } - this.lastRepairTime = System.currentTimeMillis(); - return this.globalRepairedWatermark; + public void markMismatch( + long partitionId, + long checkedAt, + long safeWatermark, + long partitionMutationEpoch, + long snapshotEpoch, + SnapshotState snapshotState, + String mismatchScopeRef, + int mismatchLeafCount, + String repairEpoch) { + markMismatch( + partitionId, + checkedAt, + safeWatermark, + partitionMutationEpoch, + snapshotEpoch, + snapshotState, + mismatchScopeRef, + mismatchLeafCount, + repairEpoch, + null); } - /** Get all partitions that need repair (PENDING, FAILED, or DIRTY). */ - public Collection getPartitionsNeedingRepair() { - return partitionProgress.values().stream() - .filter( - p -> - p.getStatus() == PartitionRepairStatus.PENDING - || p.getStatus() == PartitionRepairStatus.FAILED - || p.getStatus() == PartitionRepairStatus.DIRTY) - .collect(java.util.stream.Collectors.toList()); + public void markMismatch( + long partitionId, + long checkedAt, + long safeWatermark, + long partitionMutationEpoch, + long snapshotEpoch, + SnapshotState snapshotState, + String mismatchScopeRef, + int mismatchLeafCount, + String repairEpoch, + String replicaObservationToken) { + getOrCreatePartition(partitionId) + .markMismatch( + checkedAt, + safeWatermark, + partitionMutationEpoch, + snapshotEpoch, + snapshotState, + mismatchScopeRef, + mismatchLeafCount, + repairEpoch, + replicaObservationToken); } - public String getConsensusGroupId() { - return consensusGroupId; + public void markCheckFailed( + long partitionId, + long checkedAt, + long safeWatermark, + long partitionMutationEpoch, + long snapshotEpoch, + SnapshotState snapshotState, + String errorCode, + String errorMessage) { + markCheckFailed( + partitionId, + checkedAt, + safeWatermark, + partitionMutationEpoch, + snapshotEpoch, + snapshotState, + errorCode, + errorMessage, + null); } - public long getGlobalRepairedWatermark() { - return globalRepairedWatermark; + public void markCheckFailed( + long partitionId, + long checkedAt, + long safeWatermark, + long partitionMutationEpoch, + long snapshotEpoch, + SnapshotState snapshotState, + String errorCode, + String errorMessage, + String replicaObservationToken) { + getOrCreatePartition(partitionId) + .markCheckFailed( + checkedAt, + safeWatermark, + partitionMutationEpoch, + snapshotEpoch, + snapshotState, + errorCode, + errorMessage, + replicaObservationToken); } - public long getLastRepairTime() { - return lastRepairTime; + public void markDirty(long partitionId) { + getOrCreatePartition(partitionId).markDirty(); } - public RegionRepairStatus getRegionStatus() { - return regionStatus; + public void markRepairRunning(long partitionId, String repairEpoch) { + getOrCreatePartition(partitionId).markRepairRunning(repairEpoch); } - public void setRegionStatus(RegionRepairStatus status) { - this.regionStatus = status; + public void markRepairSucceeded( + long partitionId, + long checkedAt, + long safeWatermark, + long partitionMutationEpoch, + long snapshotEpoch, + SnapshotState snapshotState, + String repairEpoch) { + markRepairSucceeded( + partitionId, + checkedAt, + safeWatermark, + partitionMutationEpoch, + snapshotEpoch, + snapshotState, + repairEpoch, + null); + } + + public void markRepairSucceeded( + long partitionId, + long checkedAt, + long safeWatermark, + long partitionMutationEpoch, + long snapshotEpoch, + SnapshotState snapshotState, + String repairEpoch, + String replicaObservationToken) { + getOrCreatePartition(partitionId) + .markRepairSucceeded( + checkedAt, + safeWatermark, + partitionMutationEpoch, + snapshotEpoch, + snapshotState, + repairEpoch, + replicaObservationToken); + } + + public void markRepairFailed( + long partitionId, String repairEpoch, String errorCode, String errorMessage) { + getOrCreatePartition(partitionId).markRepairFailed(repairEpoch, errorCode, errorMessage); + } + + public List getAllPartitions() { + List result = new ArrayList<>(); + for (PartitionProgress progress : partitionProgress.values()) { + result.add(progress.copy()); + } + result.sort((left, right) -> Long.compare(left.getPartitionId(), right.getPartitionId())); + return Collections.unmodifiableList(result); } public Map getAllPartitionProgress() { - return java.util.Collections.unmodifiableMap(partitionProgress); + return Collections.unmodifiableMap(partitionProgress); + } + + public RepairProgressTable copy() { + RepairProgressTable copy = new RepairProgressTable(consensusGroupId); + partitionProgress.forEach( + (partitionId, progress) -> copy.partitionProgress.put(partitionId, progress.copy())); + return copy; + } + + public void serialize(OutputStream outputStream) throws IOException { + ReadWriteIOUtils.write(consensusGroupId, outputStream); + ReadWriteIOUtils.write(partitionProgress.size(), outputStream); + for (PartitionProgress progress : getAllPartitions()) { + progress.serialize(outputStream); + } + } + + public static RepairProgressTable deserialize(InputStream inputStream) throws IOException { + RepairProgressTable table = new RepairProgressTable(ReadWriteIOUtils.readString(inputStream)); + int size = ReadWriteIOUtils.readInt(inputStream); + for (int i = 0; i < size; i++) { + PartitionProgress progress = PartitionProgress.deserialize(inputStream); + table.partitionProgress.put(progress.getPartitionId(), progress); + } + return table; + } + + public static RepairProgressTable deserialize(ByteBuffer byteBuffer) { + RepairProgressTable table = new RepairProgressTable(ReadWriteIOUtils.readString(byteBuffer)); + int size = ReadWriteIOUtils.readInt(byteBuffer); + for (int i = 0; i < size; i++) { + PartitionProgress progress = PartitionProgress.deserialize(byteBuffer); + table.partitionProgress.put(progress.getPartitionId(), progress); + } + return table; + } + + @Override + public boolean equals(Object object) { + if (this == object) { + return true; + } + if (!(object instanceof RepairProgressTable)) { + return false; + } + RepairProgressTable that = (RepairProgressTable) object; + return Objects.equals(consensusGroupId, that.consensusGroupId) + && Objects.equals(getAllPartitions(), that.getAllPartitions()); + } + + @Override + public int hashCode() { + return Objects.hash(consensusGroupId, getAllPartitions()); } } diff --git a/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/consensus/iotv2/consistency/TimePartitionMerkleNode.java b/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/consensus/iotv2/consistency/TimePartitionMerkleNode.java deleted file mode 100644 index 10f672c856a8b..0000000000000 --- a/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/consensus/iotv2/consistency/TimePartitionMerkleNode.java +++ /dev/null @@ -1,129 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, - * software distributed under the License is distributed on an - * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY - * KIND, either express or implied. See the License for the - * specific language governing permissions and limitations - * under the License. - */ - -package org.apache.iotdb.commons.consensus.iotv2.consistency; - -import java.util.concurrent.atomic.AtomicLong; -import java.util.concurrent.locks.ReadWriteLock; -import java.util.concurrent.locks.ReentrantReadWriteLock; - -/** - * Represents a Level-1 node in the consistency Merkle tree, holding the dual-digest for a single - * time partition. Uses fine-grained locking to support concurrent flush/compaction/check threads. - */ -public class TimePartitionMerkleNode { - - private final long partitionId; - private volatile DualDigest partitionDigest; - private volatile boolean dirty; - private volatile long lastVerifiedTime; - private final AtomicLong version; - private final ReadWriteLock lock; - - public TimePartitionMerkleNode(long partitionId) { - this.partitionId = partitionId; - this.partitionDigest = DualDigest.ZERO; - this.dirty = false; - this.lastVerifiedTime = 0L; - this.version = new AtomicLong(0); - this.lock = new ReentrantReadWriteLock(); - } - - /** - * XOR a file's root hash into this partition's dual-digest. Called when a new TsFile is flushed - * or a compaction target is created. - */ - public void xorIn(long fileRootHash) { - addDigest(DualDigest.fromSingleHash(fileRootHash)); - } - - public void addDigest(DualDigest digest) { - lock.writeLock().lock(); - try { - partitionDigest = partitionDigest.merge(digest); - version.incrementAndGet(); - } finally { - lock.writeLock().unlock(); - } - } - - /** - * XOR a file's root hash out of this partition's dual-digest. Called when a source TsFile is - * removed during compaction. - */ - public void xorOut(long fileRootHash) { - removeDigest(DualDigest.fromSingleHash(fileRootHash)); - } - - public void removeDigest(DualDigest digest) { - lock.writeLock().lock(); - try { - partitionDigest = partitionDigest.subtract(digest); - version.incrementAndGet(); - } finally { - lock.writeLock().unlock(); - } - } - - public DualDigest getPartitionDigest() { - lock.readLock().lock(); - try { - return partitionDigest; - } finally { - lock.readLock().unlock(); - } - } - - /** Reset the digest from scratch, typically after rebuilding a dirty partition. */ - public void resetDigest(DualDigest newDigest) { - lock.writeLock().lock(); - try { - this.partitionDigest = newDigest; - this.dirty = false; - this.version.incrementAndGet(); - } finally { - lock.writeLock().unlock(); - } - } - - public void markDirty() { - this.dirty = true; - this.version.incrementAndGet(); - } - - public boolean isDirty() { - return dirty; - } - - public long getPartitionId() { - return partitionId; - } - - public long getVersion() { - return version.get(); - } - - public long getLastVerifiedTime() { - return lastVerifiedTime; - } - - public void setLastVerifiedTime(long lastVerifiedTime) { - this.lastVerifiedTime = lastVerifiedTime; - } -} diff --git a/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/consensus/iotv2/consistency/ibf/CompositeKeyCodec.java b/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/consensus/iotv2/consistency/ibf/CompositeKeyCodec.java deleted file mode 100644 index 65aee1fd88c8d..0000000000000 --- a/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/consensus/iotv2/consistency/ibf/CompositeKeyCodec.java +++ /dev/null @@ -1,88 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, - * software distributed under the License is distributed on an - * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY - * KIND, either express or implied. See the License for the - * specific language governing permissions and limitations - * under the License. - */ - -package org.apache.iotdb.commons.consensus.iotv2.consistency.ibf; - -/** - * Codec for the 8-byte composite key used in IBF cells. Layout: - * - *
- *   [deviceIndex: 2 bytes][measurementIndex: 2 bytes][timestamp_bucket_offset: 4 bytes]
- * 
- * - * The deviceIndex and measurementIndex are ordinal indices (sorted order) agreed upon by Leader and - * Follower during the NEGOTIATE_KEY_MAPPING phase. The timestamp_bucket_offset encodes (timestamp - - * timeBucketStart) in units of the minimum timestamp resolution. - */ -public final class CompositeKeyCodec { - - public static final long DEFAULT_TIMESTAMP_RESOLUTION = 1L; - public static final int MAX_DEVICE_INDEX = 0xFFFF; - public static final int MAX_MEASUREMENT_INDEX = 0xFFFF; - - private CompositeKeyCodec() {} - - /** - * Encode a data point's location into a composite key. - * - * @param deviceIndex ordinal index of the device within the partition - * @param measurementIndex ordinal index of the measurement within the device - * @param timestamp the data point's timestamp - * @param bucketStart the start of the time bucket - * @param timestampResolution minimum timestamp resolution - * @return 8-byte composite key - */ - public static long encode( - int deviceIndex, - int measurementIndex, - long timestamp, - long bucketStart, - long timestampResolution) { - int tsOffset = (int) ((timestamp - bucketStart) / timestampResolution); - return ((long) (deviceIndex & 0xFFFF) << 48) - | ((long) (measurementIndex & 0xFFFF) << 32) - | (tsOffset & 0xFFFFFFFFL); - } - - public static int extractDeviceIndex(long compositeKey) { - return (int) ((compositeKey >>> 48) & 0xFFFF); - } - - public static int extractMeasurementIndex(long compositeKey) { - return (int) ((compositeKey >>> 32) & 0xFFFF); - } - - public static int extractTimestampOffset(long compositeKey) { - return (int) (compositeKey & 0xFFFFFFFFL); - } - - /** - * Recover the original timestamp from the composite key. - * - * @param compositeKey the encoded key - * @param bucketStart the time bucket start - * @param timestampResolution the timestamp resolution - * @return the original timestamp - */ - public static long recoverTimestamp( - long compositeKey, long bucketStart, long timestampResolution) { - int tsOffset = extractTimestampOffset(compositeKey); - return bucketStart + (long) tsOffset * timestampResolution; - } -} diff --git a/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/consensus/iotv2/consistency/ibf/DataPointLocator.java b/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/consensus/iotv2/consistency/ibf/DataPointLocator.java deleted file mode 100644 index f865ca5e7b33c..0000000000000 --- a/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/consensus/iotv2/consistency/ibf/DataPointLocator.java +++ /dev/null @@ -1,71 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, - * software distributed under the License is distributed on an - * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY - * KIND, either express or implied. See the License for the - * specific language governing permissions and limitations - * under the License. - */ - -package org.apache.iotdb.commons.consensus.iotv2.consistency.ibf; - -import java.util.Objects; - -/** - * Locates a specific data point by its logical coordinates: (deviceId, measurement, timestamp). - * Resolved from a composite key via the RowRefIndex. - */ -public class DataPointLocator { - - private final String deviceId; - private final String measurement; - private final long timestamp; - - public DataPointLocator(String deviceId, String measurement, long timestamp) { - this.deviceId = deviceId; - this.measurement = measurement; - this.timestamp = timestamp; - } - - public String getDeviceId() { - return deviceId; - } - - public String getMeasurement() { - return measurement; - } - - public long getTimestamp() { - return timestamp; - } - - @Override - public boolean equals(Object o) { - if (this == o) return true; - if (!(o instanceof DataPointLocator)) return false; - DataPointLocator that = (DataPointLocator) o; - return timestamp == that.timestamp - && Objects.equals(deviceId, that.deviceId) - && Objects.equals(measurement, that.measurement); - } - - @Override - public int hashCode() { - return Objects.hash(deviceId, measurement, timestamp); - } - - @Override - public String toString() { - return String.format("(%s, %s, %d)", deviceId, measurement, timestamp); - } -} diff --git a/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/consensus/iotv2/consistency/ibf/DiffEntry.java b/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/consensus/iotv2/consistency/ibf/DiffEntry.java deleted file mode 100644 index f5d6edb164fb3..0000000000000 --- a/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/consensus/iotv2/consistency/ibf/DiffEntry.java +++ /dev/null @@ -1,59 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, - * software distributed under the License is distributed on an - * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY - * KIND, either express or implied. See the License for the - * specific language governing permissions and limitations - * under the License. - */ - -package org.apache.iotdb.commons.consensus.iotv2.consistency.ibf; - -/** Represents a single decoded diff from IBF subtraction and decoding. */ -public class DiffEntry { - - public enum DiffType { - /** Key exists on the Leader (positive side) but not on the Follower. */ - LEADER_HAS, - /** Key exists on the Follower (negative side) but not on the Leader. */ - FOLLOWER_HAS - } - - private final long compositeKey; - private final long valueHash; - private final DiffType type; - - public DiffEntry(long compositeKey, long valueHash, DiffType type) { - this.compositeKey = compositeKey; - this.valueHash = valueHash; - this.type = type; - } - - public long getCompositeKey() { - return compositeKey; - } - - public long getValueHash() { - return valueHash; - } - - public DiffType getType() { - return type; - } - - @Override - public String toString() { - return String.format( - "DiffEntry{key=0x%016X, valHash=0x%016X, type=%s}", compositeKey, valueHash, type); - } -} diff --git a/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/consensus/iotv2/consistency/ibf/IBFCell.java b/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/consensus/iotv2/consistency/ibf/IBFCell.java deleted file mode 100644 index 1fb84c5d3a4ac..0000000000000 --- a/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/consensus/iotv2/consistency/ibf/IBFCell.java +++ /dev/null @@ -1,98 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, - * software distributed under the License is distributed on an - * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY - * KIND, either express or implied. See the License for the - * specific language governing permissions and limitations - * under the License. - */ - -package org.apache.iotdb.commons.consensus.iotv2.consistency.ibf; - -import java.io.DataOutputStream; -import java.io.IOException; -import java.nio.ByteBuffer; - -/** A single cell in an Invertible Bloom Filter. */ -public class IBFCell { - - int count; - long keySum; - long valueChecksum; - - public IBFCell() { - this.count = 0; - this.keySum = 0L; - this.valueChecksum = 0L; - } - - public IBFCell(int count, long keySum, long valueChecksum) { - this.count = count; - this.keySum = keySum; - this.valueChecksum = valueChecksum; - } - - public boolean isPure() { - return count == 1 || count == -1; - } - - public boolean isEmpty() { - return count == 0 && keySum == 0L && valueChecksum == 0L; - } - - public void add(long key, long valueHash) { - count += 1; - keySum ^= key; - valueChecksum ^= valueHash; - } - - public void remove(long key, long valueHash) { - count -= 1; - keySum ^= key; - valueChecksum ^= valueHash; - } - - public void subtract(IBFCell other) { - count -= other.count; - keySum ^= other.keySum; - valueChecksum ^= other.valueChecksum; - } - - public int getCount() { - return count; - } - - public long getKeySum() { - return keySum; - } - - public long getValueChecksum() { - return valueChecksum; - } - - public void serialize(DataOutputStream out) throws IOException { - out.writeInt(count); - out.writeLong(keySum); - out.writeLong(valueChecksum); - } - - public static IBFCell deserialize(ByteBuffer buffer) { - int count = buffer.getInt(); - long keySum = buffer.getLong(); - long valueChecksum = buffer.getLong(); - return new IBFCell(count, keySum, valueChecksum); - } - - /** Serialized size in bytes: 4 (count) + 8 (keySum) + 8 (valueChecksum) = 20 */ - public static final int SERIALIZED_SIZE = 20; -} diff --git a/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/consensus/iotv2/consistency/ibf/IBFDecodeResult.java b/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/consensus/iotv2/consistency/ibf/IBFDecodeResult.java deleted file mode 100644 index 5d0dbab75acad..0000000000000 --- a/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/consensus/iotv2/consistency/ibf/IBFDecodeResult.java +++ /dev/null @@ -1,58 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, - * software distributed under the License is distributed on an - * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY - * KIND, either express or implied. See the License for the - * specific language governing permissions and limitations - * under the License. - */ - -package org.apache.iotdb.commons.consensus.iotv2.consistency.ibf; - -import java.util.Collections; -import java.util.List; - -/** The result of decoding an IBF difference. */ -public class IBFDecodeResult { - - private final boolean success; - private final List decodedEntries; - private final int partialCount; - - private IBFDecodeResult(boolean success, List decodedEntries, int partialCount) { - this.success = success; - this.decodedEntries = decodedEntries; - this.partialCount = partialCount; - } - - public static IBFDecodeResult success(List entries) { - return new IBFDecodeResult(true, entries, entries.size()); - } - - public static IBFDecodeResult failure(List partialEntries, int partialCount) { - return new IBFDecodeResult(false, partialEntries, partialCount); - } - - public boolean isSuccess() { - return success; - } - - public List getDecodedEntries() { - return Collections.unmodifiableList(decodedEntries); - } - - /** Number of entries decoded (may be partial if decode failed). */ - public int getPartialCount() { - return partialCount; - } -} diff --git a/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/consensus/iotv2/consistency/ibf/InvertibleBloomFilter.java b/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/consensus/iotv2/consistency/ibf/InvertibleBloomFilter.java deleted file mode 100644 index 1aba086bed094..0000000000000 --- a/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/consensus/iotv2/consistency/ibf/InvertibleBloomFilter.java +++ /dev/null @@ -1,210 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, - * software distributed under the License is distributed on an - * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY - * KIND, either express or implied. See the License for the - * specific language governing permissions and limitations - * under the License. - */ - -package org.apache.iotdb.commons.consensus.iotv2.consistency.ibf; - -import java.io.DataOutputStream; -import java.io.IOException; -import java.nio.ByteBuffer; -import java.util.ArrayDeque; -import java.util.ArrayList; -import java.util.Deque; -import java.util.List; - -/** - * Invertible Bloom Filter (IBF) for set difference computation with O(d) communication complexity - * where d is the number of differences. Uses k=3 hash functions to map (compositeKey, valueHash) - * pairs into cells. - */ -public class InvertibleBloomFilter { - - private static final int DEFAULT_HASH_COUNT = 3; - private static final long HASH_SEED_1 = 0x9E3779B97F4A7C15L; - private static final long HASH_SEED_2 = 0x517CC1B727220A95L; - private static final long HASH_SEED_3 = 0x6C62272E07BB0142L; - private static final long[] SEEDS = {HASH_SEED_1, HASH_SEED_2, HASH_SEED_3}; - - private final IBFCell[] cells; - private final int cellCount; - private final int hashCount; - - public InvertibleBloomFilter(int cellCount) { - this(cellCount, DEFAULT_HASH_COUNT); - } - - public InvertibleBloomFilter(int cellCount, int hashCount) { - this.cellCount = Math.max(cellCount, 1); - this.hashCount = hashCount; - this.cells = new IBFCell[this.cellCount]; - for (int i = 0; i < this.cellCount; i++) { - cells[i] = new IBFCell(); - } - } - - private InvertibleBloomFilter(IBFCell[] cells, int hashCount) { - this.cells = cells; - this.cellCount = cells.length; - this.hashCount = hashCount; - } - - public void insert(long compositeKey, long valueHash) { - for (int i = 0; i < hashCount; i++) { - int idx = hashToIndex(compositeKey, i); - cells[idx].add(compositeKey, valueHash); - } - } - - public void remove(long compositeKey, long valueHash) { - for (int i = 0; i < hashCount; i++) { - int idx = hashToIndex(compositeKey, i); - cells[idx].remove(compositeKey, valueHash); - } - } - - /** - * Subtract another IBF from this one (element-wise). The result IBF encodes the symmetric - * difference: keys in this but not other (count=+1) and keys in other but not this (count=-1). - */ - public InvertibleBloomFilter subtract(InvertibleBloomFilter other) { - if (this.cellCount != other.cellCount) { - throw new IllegalArgumentException( - "IBF cell counts must match: " + this.cellCount + " vs " + other.cellCount); - } - IBFCell[] result = new IBFCell[cellCount]; - for (int i = 0; i < cellCount; i++) { - result[i] = - new IBFCell(this.cells[i].count, this.cells[i].keySum, this.cells[i].valueChecksum); - result[i].subtract(other.cells[i]); - } - return new InvertibleBloomFilter(result, this.hashCount); - } - - /** - * Decode the IBF to recover all diff entries. The IBF should be the result of a subtraction - * (IBF_Leader - IBF_Follower). - * - * @return decode result with success flag and decoded entries - */ - public IBFDecodeResult decode() { - List entries = new ArrayList<>(); - Deque pureIndices = new ArrayDeque<>(); - - // Initial scan for pure cells - for (int i = 0; i < cellCount; i++) { - if (cells[i].isPure()) { - pureIndices.add(i); - } - } - - while (!pureIndices.isEmpty()) { - int idx = pureIndices.poll(); - IBFCell cell = cells[idx]; - - if (!cell.isPure()) { - continue; - } - - long key = cell.keySum; - long valueHash = cell.valueChecksum; - DiffEntry.DiffType type = - cell.count == 1 ? DiffEntry.DiffType.LEADER_HAS : DiffEntry.DiffType.FOLLOWER_HAS; - entries.add(new DiffEntry(key, valueHash, type)); - - // Peel this entry from all cells it hashes to - for (int i = 0; i < hashCount; i++) { - int cellIdx = hashToIndex(key, i); - if (cell.count == 1) { - cells[cellIdx].remove(key, valueHash); - } else { - cells[cellIdx].add(key, valueHash); - } - if (cells[cellIdx].isPure()) { - pureIndices.add(cellIdx); - } - } - } - - // Check if all cells are empty (complete decode) - boolean complete = true; - for (IBFCell cell : cells) { - if (!cell.isEmpty()) { - complete = false; - break; - } - } - - if (complete) { - return IBFDecodeResult.success(entries); - } else { - return IBFDecodeResult.failure(entries, entries.size()); - } - } - - /** - * Hash a composite key to a cell index for the i-th hash function, using multiplicative hashing - * with distinct seeds to minimize collisions. - */ - private int hashToIndex(long key, int hashFunctionIndex) { - long seed = SEEDS[hashFunctionIndex % SEEDS.length]; - if (hashFunctionIndex >= SEEDS.length) { - seed ^= hashFunctionIndex * 0xDEADBEEFL; - } - long hash = key * seed; - hash ^= hash >>> 33; - hash *= 0xFF51AFD7ED558CCDL; - hash ^= hash >>> 33; - return (int) ((hash & 0x7FFFFFFFFFFFFFFFL) % cellCount); - } - - public int getCellCount() { - return cellCount; - } - - public IBFCell[] getCells() { - return cells; - } - - /** Compute the optimal IBF size for an estimated diff count d with >99% decode probability. */ - public static int optimalCellCount(long estimatedDiffCount) { - return (int) Math.max(Math.ceil(2.0 * estimatedDiffCount), 3); - } - - public void serialize(DataOutputStream out) throws IOException { - out.writeInt(cellCount); - out.writeInt(hashCount); - for (IBFCell cell : cells) { - cell.serialize(out); - } - } - - public static InvertibleBloomFilter deserialize(ByteBuffer buffer) { - int cellCount = buffer.getInt(); - int hashCount = buffer.getInt(); - IBFCell[] cells = new IBFCell[cellCount]; - for (int i = 0; i < cellCount; i++) { - cells[i] = IBFCell.deserialize(buffer); - } - return new InvertibleBloomFilter(cells, hashCount); - } - - /** Total serialized byte size: 4 (cellCount) + 4 (hashCount) + cellCount * 20 bytes */ - public int serializedSize() { - return 8 + cellCount * IBFCell.SERIALIZED_SIZE; - } -} diff --git a/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/consensus/iotv2/consistency/ibf/RowRefIndex.java b/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/consensus/iotv2/consistency/ibf/RowRefIndex.java deleted file mode 100644 index 0d5fd3c90aea9..0000000000000 --- a/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/consensus/iotv2/consistency/ibf/RowRefIndex.java +++ /dev/null @@ -1,160 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, - * software distributed under the License is distributed on an - * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY - * KIND, either express or implied. See the License for the - * specific language governing permissions and limitations - * under the License. - */ - -package org.apache.iotdb.commons.consensus.iotv2.consistency.ibf; - -import java.util.ArrayList; -import java.util.Collections; -import java.util.List; -import java.util.TreeMap; - -/** - * Temporary in-memory index built during IBF construction that maps composite keys back to the - * original (deviceId, measurement, timestamp) coordinates. This is the critical bridge between IBF - * decoding (which recovers composite keys) and repair execution (which needs actual data point - * identifiers). - * - *

The index is built from the negotiated device/measurement mapping and discarded after repair. - */ -public class RowRefIndex { - - private final List deviceIdByIndex; - private final List> measurementsByDevice; - private final long timeBucketStart; - private final long timestampResolution; - - private RowRefIndex( - List deviceIdByIndex, - List> measurementsByDevice, - long timeBucketStart, - long timestampResolution) { - this.deviceIdByIndex = deviceIdByIndex; - this.measurementsByDevice = measurementsByDevice; - this.timeBucketStart = timeBucketStart; - this.timestampResolution = timestampResolution; - } - - /** - * Resolve a decoded composite key back to original data point coordinates. - * - * @param compositeKey the 8-byte composite key from IBF decoding - * @return the resolved data point locator - * @throws IndexOutOfBoundsException if indices are invalid - */ - public DataPointLocator resolve(long compositeKey) { - int deviceIdx = CompositeKeyCodec.extractDeviceIndex(compositeKey); - int measIdx = CompositeKeyCodec.extractMeasurementIndex(compositeKey); - long timestamp = - CompositeKeyCodec.recoverTimestamp(compositeKey, timeBucketStart, timestampResolution); - - return new DataPointLocator( - deviceIdByIndex.get(deviceIdx), - measurementsByDevice.get(deviceIdx).get(measIdx), - timestamp); - } - - public List getDeviceIdByIndex() { - return Collections.unmodifiableList(deviceIdByIndex); - } - - public List> getMeasurementsByDevice() { - return Collections.unmodifiableList(measurementsByDevice); - } - - public long getTimeBucketStart() { - return timeBucketStart; - } - - public long getTimestampResolution() { - return timestampResolution; - } - - /** Estimate heap memory consumed by this index. */ - public int estimatedMemoryBytes() { - int bytes = 64; // object overhead - for (String deviceId : deviceIdByIndex) { - bytes += 40 + deviceId.length() * 2; - } - for (List measurements : measurementsByDevice) { - bytes += 40; - for (String m : measurements) { - bytes += 40 + m.length() * 2; - } - } - return bytes; - } - - /** Builder for constructing RowRefIndex from device/measurement lists. */ - public static class Builder { - private final TreeMap> deviceMeasurements = new TreeMap<>(); - private long timeBucketStart = 0; - private long timestampResolution = CompositeKeyCodec.DEFAULT_TIMESTAMP_RESOLUTION; - - public Builder addDevice(String deviceId, List measurements) { - List sorted = new ArrayList<>(measurements); - Collections.sort(sorted); - deviceMeasurements.put(deviceId, sorted); - return this; - } - - public Builder setTimeBucketStart(long timeBucketStart) { - this.timeBucketStart = timeBucketStart; - return this; - } - - public Builder setTimestampResolution(long timestampResolution) { - this.timestampResolution = timestampResolution; - return this; - } - - /** - * Build the RowRefIndex and return a mapping from (deviceId, measurement) to their ordinal - * indices. - */ - public RowRefIndex build() { - List deviceList = new ArrayList<>(deviceMeasurements.keySet()); - List> measurementList = new ArrayList<>(); - for (String device : deviceList) { - measurementList.add(deviceMeasurements.get(device)); - } - return new RowRefIndex(deviceList, measurementList, timeBucketStart, timestampResolution); - } - - /** Get the device index for a given deviceId. Returns -1 if not found. */ - public int getDeviceIndex(String deviceId) { - int idx = 0; - for (String d : deviceMeasurements.keySet()) { - if (d.equals(deviceId)) { - return idx; - } - idx++; - } - return -1; - } - - /** Get the measurement index for a given device and measurement. Returns -1 if not found. */ - public int getMeasurementIndex(String deviceId, String measurement) { - List measurements = deviceMeasurements.get(deviceId); - if (measurements == null) { - return -1; - } - return Collections.binarySearch(measurements, measurement); - } - } -} diff --git a/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/consensus/iotv2/consistency/ibf/StrataEstimator.java b/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/consensus/iotv2/consistency/ibf/StrataEstimator.java deleted file mode 100644 index 82805b1ac8b88..0000000000000 --- a/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/consensus/iotv2/consistency/ibf/StrataEstimator.java +++ /dev/null @@ -1,108 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, - * software distributed under the License is distributed on an - * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY - * KIND, either express or implied. See the License for the - * specific language governing permissions and limitations - * under the License. - */ - -package org.apache.iotdb.commons.consensus.iotv2.consistency.ibf; - -import java.io.DataOutputStream; -import java.io.IOException; -import java.nio.ByteBuffer; - -/** - * Strata Estimator for estimating the number of differences between two sets before sizing the main - * IBF. Uses a stack of small IBFs at different strata levels, where each element is assigned to a - * stratum based on the number of trailing zeros in its key. Communication cost: ~50 KB. - * - *

Reference: Eppstein et al., "What's the Difference? Efficient Set Reconciliation without Prior - * Context" - */ -public class StrataEstimator { - - private static final int STRATA_COUNT = 32; - private static final int CELLS_PER_STRATUM = 80; - private static final int HASH_COUNT = 3; - - private final InvertibleBloomFilter[] strata; - - public StrataEstimator() { - strata = new InvertibleBloomFilter[STRATA_COUNT]; - for (int i = 0; i < STRATA_COUNT; i++) { - strata[i] = new InvertibleBloomFilter(CELLS_PER_STRATUM, HASH_COUNT); - } - } - - private StrataEstimator(InvertibleBloomFilter[] strata) { - this.strata = strata; - } - - /** Insert a (key, valueHash) pair into the appropriate stratum. */ - public void insert(long key, long valueHash) { - int level = Long.numberOfTrailingZeros(key); - if (level >= STRATA_COUNT) { - level = STRATA_COUNT - 1; - } - strata[level].insert(key, valueHash); - } - - /** - * Estimate the number of differences between this estimator and another. - * - * @param other the other side's strata estimator - * @return estimated diff count - */ - public long estimateDifference(StrataEstimator other) { - long estimate = 0; - - for (int i = STRATA_COUNT - 1; i >= 0; i--) { - InvertibleBloomFilter diff = this.strata[i].subtract(other.strata[i]); - IBFDecodeResult result = diff.decode(); - if (!result.isSuccess()) { - // If decode fails at this level, scale up the partial count - return (estimate + result.getPartialCount()) * (1L << (i + 1)); - } - estimate += result.getPartialCount(); - } - - return estimate; - } - - public void serialize(DataOutputStream out) throws IOException { - out.writeInt(STRATA_COUNT); - for (InvertibleBloomFilter ibf : strata) { - ibf.serialize(out); - } - } - - public static StrataEstimator deserialize(ByteBuffer buffer) { - int count = buffer.getInt(); - InvertibleBloomFilter[] strata = new InvertibleBloomFilter[count]; - for (int i = 0; i < count; i++) { - strata[i] = InvertibleBloomFilter.deserialize(buffer); - } - return new StrataEstimator(strata); - } - - /** Approximate serialized size: 4 + STRATA_COUNT * (8 + CELLS_PER_STRATUM * 20) ≈ 50 KB. */ - public int serializedSize() { - int size = 4; - for (InvertibleBloomFilter ibf : strata) { - size += ibf.serializedSize(); - } - return size; - } -} diff --git a/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/consensus/iotv2/consistency/merkle/MerkleEntry.java b/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/consensus/iotv2/consistency/merkle/MerkleEntry.java deleted file mode 100644 index 3c7b13725c03f..0000000000000 --- a/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/consensus/iotv2/consistency/merkle/MerkleEntry.java +++ /dev/null @@ -1,73 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, - * software distributed under the License is distributed on an - * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY - * KIND, either express or implied. See the License for the - * specific language governing permissions and limitations - * under the License. - */ - -package org.apache.iotdb.commons.consensus.iotv2.consistency.merkle; - -/** - * A single entry in a .merkle file representing the hash of a (device, measurement, timeBucket) - * slice. - */ -public class MerkleEntry { - - private final String deviceId; - private final String measurement; - private final long timeBucketStart; - private final long timeBucketEnd; - private final int pointCount; - private final long entryHash; - - public MerkleEntry( - String deviceId, - String measurement, - long timeBucketStart, - long timeBucketEnd, - int pointCount, - long entryHash) { - this.deviceId = deviceId; - this.measurement = measurement; - this.timeBucketStart = timeBucketStart; - this.timeBucketEnd = timeBucketEnd; - this.pointCount = pointCount; - this.entryHash = entryHash; - } - - public String getDeviceId() { - return deviceId; - } - - public String getMeasurement() { - return measurement; - } - - public long getTimeBucketStart() { - return timeBucketStart; - } - - public long getTimeBucketEnd() { - return timeBucketEnd; - } - - public int getPointCount() { - return pointCount; - } - - public long getEntryHash() { - return entryHash; - } -} diff --git a/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/consensus/iotv2/consistency/merkle/MerkleFileCache.java b/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/consensus/iotv2/consistency/merkle/MerkleFileCache.java deleted file mode 100644 index 8c737e1b4fe5c..0000000000000 --- a/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/consensus/iotv2/consistency/merkle/MerkleFileCache.java +++ /dev/null @@ -1,124 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, - * software distributed under the License is distributed on an - * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY - * KIND, either express or implied. See the License for the - * specific language governing permissions and limitations - * under the License. - */ - -package org.apache.iotdb.commons.consensus.iotv2.consistency.merkle; - -import java.io.IOException; -import java.util.LinkedHashMap; -import java.util.Map; -import java.util.concurrent.locks.ReentrantLock; - -/** - * LRU cache for on-demand loading of .merkle file contents. Level 2+ Merkle data is loaded from - * disk into this cache and evicted when memory pressure exceeds the configured maximum. - */ -public class MerkleFileCache { - - private final long maxMemoryBytes; - private volatile long currentMemoryBytes; - private final LinkedHashMap cache; - private final ReentrantLock lock; - - public MerkleFileCache(long maxMemoryBytes) { - this.maxMemoryBytes = maxMemoryBytes; - this.currentMemoryBytes = 0; - this.lock = new ReentrantLock(); - this.cache = - new LinkedHashMap(64, 0.75f, true) { - @Override - protected boolean removeEldestEntry(Map.Entry eldest) { - if (currentMemoryBytes > maxMemoryBytes) { - currentMemoryBytes -= eldest.getValue().estimatedMemoryBytes(); - return true; - } - return false; - } - }; - } - - /** - * Get the .merkle content for a TsFile. Loads from disk if not cached. - * - * @param tsFilePath path to the TsFile (the .merkle file is at tsFilePath + ".merkle") - * @return cached or freshly loaded content - */ - public MerkleFileContent get(String tsFilePath) throws IOException { - lock.lock(); - try { - MerkleFileContent content = cache.get(tsFilePath); - if (content != null) { - return content; - } - } finally { - lock.unlock(); - } - - MerkleFileContent loaded = MerkleFileReader.read(tsFilePath + ".merkle", tsFilePath); - - lock.lock(); - try { - MerkleFileContent existing = cache.get(tsFilePath); - if (existing != null) { - return existing; - } - cache.put(tsFilePath, loaded); - currentMemoryBytes += loaded.estimatedMemoryBytes(); - return loaded; - } finally { - lock.unlock(); - } - } - - /** Invalidate a specific entry, e.g., when its TsFile is compacted away. */ - public void invalidate(String tsFilePath) { - lock.lock(); - try { - MerkleFileContent removed = cache.remove(tsFilePath); - if (removed != null) { - currentMemoryBytes -= removed.estimatedMemoryBytes(); - } - } finally { - lock.unlock(); - } - } - - /** Clear all cached entries. */ - public void clear() { - lock.lock(); - try { - cache.clear(); - currentMemoryBytes = 0; - } finally { - lock.unlock(); - } - } - - public long getCurrentMemoryBytes() { - return currentMemoryBytes; - } - - public int size() { - lock.lock(); - try { - return cache.size(); - } finally { - lock.unlock(); - } - } -} diff --git a/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/consensus/iotv2/consistency/merkle/MerkleFileContent.java b/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/consensus/iotv2/consistency/merkle/MerkleFileContent.java deleted file mode 100644 index 98f31a28888c8..0000000000000 --- a/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/consensus/iotv2/consistency/merkle/MerkleFileContent.java +++ /dev/null @@ -1,82 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, - * software distributed under the License is distributed on an - * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY - * KIND, either express or implied. See the License for the - * specific language governing permissions and limitations - * under the License. - */ - -package org.apache.iotdb.commons.consensus.iotv2.consistency.merkle; - -import org.apache.iotdb.commons.consensus.iotv2.consistency.DualDigest; - -import java.util.Collections; -import java.util.List; - -/** - * In-memory representation of a parsed .merkle file. Holds the file-level dual-digest and all - * per-(device, measurement, timeBucket) entries. - */ -public class MerkleFileContent { - - private final long fileXorHash; - private final long fileAddHash; - private final List entries; - private final String sourceTsFilePath; - - public MerkleFileContent( - long fileXorHash, long fileAddHash, List entries, String sourceTsFilePath) { - this.fileXorHash = fileXorHash; - this.fileAddHash = fileAddHash; - this.entries = Collections.unmodifiableList(entries); - this.sourceTsFilePath = sourceTsFilePath; - } - - public long getFileXorHash() { - return fileXorHash; - } - - public long getFileAddHash() { - return fileAddHash; - } - - public DualDigest getFileDigest() { - return new DualDigest(fileXorHash, fileAddHash); - } - - public List getEntries() { - return entries; - } - - public String getSourceTsFilePath() { - return sourceTsFilePath; - } - - public int getTotalPointCount() { - int total = 0; - for (MerkleEntry entry : entries) { - total += entry.getPointCount(); - } - return total; - } - - /** Estimate heap memory consumed by this content for cache weighing. */ - public int estimatedMemoryBytes() { - int base = 64; - for (MerkleEntry entry : entries) { - base += 80 + entry.getDeviceId().length() * 2 + entry.getMeasurement().length() * 2; - } - return base; - } -} diff --git a/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/consensus/iotv2/consistency/merkle/MerkleFileReader.java b/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/consensus/iotv2/consistency/merkle/MerkleFileReader.java deleted file mode 100644 index 579e709396659..0000000000000 --- a/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/consensus/iotv2/consistency/merkle/MerkleFileReader.java +++ /dev/null @@ -1,148 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, - * software distributed under the License is distributed on an - * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY - * KIND, either express or implied. See the License for the - * specific language governing permissions and limitations - * under the License. - */ - -package org.apache.iotdb.commons.consensus.iotv2.consistency.merkle; - -import java.io.BufferedInputStream; -import java.io.DataInputStream; -import java.io.FileInputStream; -import java.io.IOException; -import java.io.InputStream; -import java.nio.charset.StandardCharsets; -import java.util.ArrayList; -import java.util.List; -import java.util.zip.CRC32; - -/** Reads and validates a .merkle sidecar file. Supports version 2 format with dual-digest. */ -public class MerkleFileReader { - - private static final byte[] MAGIC = {'M', 'R', 'K', 'L'}; - - private MerkleFileReader() {} - - /** - * Read a .merkle file and return its in-memory content. - * - * @param merklePath full path to the .merkle file - * @param sourceTsFilePath the associated TsFile path (for attribution) - * @return parsed content, or throws on corruption - */ - public static MerkleFileContent read(String merklePath, String sourceTsFilePath) - throws IOException { - CRC32 crc32 = new CRC32(); - try (InputStream fis = new FileInputStream(merklePath); - BufferedInputStream bis = new BufferedInputStream(fis); - CrcInputStream crcStream = new CrcInputStream(bis, crc32); - DataInputStream in = new DataInputStream(crcStream)) { - - // Header: magic (4), version (2), flags (2), fileXorHash (8), fileAddHash (8), entryCount - // (4) - byte[] magic = new byte[4]; - in.readFully(magic); - if (magic[0] != MAGIC[0] - || magic[1] != MAGIC[1] - || magic[2] != MAGIC[2] - || magic[3] != MAGIC[3]) { - throw new IOException("Invalid .merkle file magic: " + merklePath); - } - - short version = in.readShort(); - if (version < 2) { - throw new IOException("Unsupported .merkle version: " + version + " at " + merklePath); - } - in.readShort(); // flags, reserved - - long fileXorHash = in.readLong(); - long fileAddHash = in.readLong(); - int entryCount = in.readInt(); - - List entries = new ArrayList<>(entryCount); - for (int i = 0; i < entryCount; i++) { - short deviceIdLen = in.readShort(); - byte[] deviceIdBytes = new byte[deviceIdLen]; - in.readFully(deviceIdBytes); - String deviceId = new String(deviceIdBytes, StandardCharsets.UTF_8); - - short measurementLen = in.readShort(); - byte[] measurementBytes = new byte[measurementLen]; - in.readFully(measurementBytes); - String measurement = new String(measurementBytes, StandardCharsets.UTF_8); - - long timeBucketStart = in.readLong(); - long timeBucketEnd = in.readLong(); - int pointCount = in.readInt(); - long entryHash = in.readLong(); - - entries.add( - new MerkleEntry( - deviceId, measurement, timeBucketStart, timeBucketEnd, pointCount, entryHash)); - } - - // Footer: CRC32 validation -- capture CRC before reading the footer - int computedCrc = (int) crc32.getValue(); - // Read the stored CRC via DataInputStream (this will update crc32, but we already saved it) - int storedCrc = in.readInt(); - if (computedCrc != storedCrc) { - throw new IOException( - String.format( - "CRC mismatch in .merkle file %s: expected 0x%08X, got 0x%08X", - merklePath, storedCrc, computedCrc)); - } - - return new MerkleFileContent(fileXorHash, fileAddHash, entries, sourceTsFilePath); - } - } - - /** - * An InputStream wrapper that updates a CRC32 on every byte read. DataInputStream is then - * constructed on top of this, ensuring all reads flow through the CRC update. - */ - private static class CrcInputStream extends InputStream { - private final InputStream delegate; - private final CRC32 crc; - - CrcInputStream(InputStream delegate, CRC32 crc) { - this.delegate = delegate; - this.crc = crc; - } - - @Override - public int read() throws IOException { - int b = delegate.read(); - if (b >= 0) { - crc.update(b); - } - return b; - } - - @Override - public int read(byte[] buf, int off, int len) throws IOException { - int n = delegate.read(buf, off, len); - if (n > 0) { - crc.update(buf, off, n); - } - return n; - } - - @Override - public void close() throws IOException { - delegate.close(); - } - } -} diff --git a/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/consensus/iotv2/consistency/merkle/MerkleFileWriter.java b/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/consensus/iotv2/consistency/merkle/MerkleFileWriter.java deleted file mode 100644 index 74328ccb331a7..0000000000000 --- a/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/consensus/iotv2/consistency/merkle/MerkleFileWriter.java +++ /dev/null @@ -1,136 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, - * software distributed under the License is distributed on an - * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY - * KIND, either express or implied. See the License for the - * specific language governing permissions and limitations - * under the License. - */ - -package org.apache.iotdb.commons.consensus.iotv2.consistency.merkle; - -import java.io.BufferedOutputStream; -import java.io.DataOutputStream; -import java.io.FileOutputStream; -import java.io.IOException; -import java.io.OutputStream; -import java.nio.charset.StandardCharsets; -import java.util.List; -import java.util.zip.CRC32; - -/** - * Writes a .merkle sidecar file alongside a TsFile. Format version 2 supports dual-digest - * (fileXorHash + fileAddHash) in the header. - */ -public class MerkleFileWriter { - - private static final byte[] MAGIC = {'M', 'R', 'K', 'L'}; - private static final short VERSION = 2; - private static final short FLAGS = 0; - - private MerkleFileWriter() {} - - /** - * Write a .merkle file at the specified path. - * - * @param merklePath full path for the .merkle file (typically tsFilePath + ".merkle") - * @param entries all (device, measurement, timeBucket) hash entries sorted by device -> meas -> - * timeBucketStart - * @param fileXorHash XOR of all entry hashes - * @param fileAddHash SUM of all entry hashes (mod 2^64) - */ - public static void write( - String merklePath, List entries, long fileXorHash, long fileAddHash) - throws IOException { - CRC32 crc32 = new CRC32(); - try (OutputStream fos = new FileOutputStream(merklePath); - BufferedOutputStream bos = new BufferedOutputStream(fos); - CrcDataOutputStream out = new CrcDataOutputStream(bos, crc32)) { - - // Header (28 bytes) - out.write(MAGIC); - out.writeShort(VERSION); - out.writeShort(FLAGS); - out.writeLong(fileXorHash); - out.writeLong(fileAddHash); - out.writeInt(entries.size()); - - // Entries - for (MerkleEntry entry : entries) { - byte[] deviceBytes = entry.getDeviceId().getBytes(StandardCharsets.UTF_8); - out.writeShort(deviceBytes.length); - out.write(deviceBytes); - - byte[] measurementBytes = entry.getMeasurement().getBytes(StandardCharsets.UTF_8); - out.writeShort(measurementBytes.length); - out.write(measurementBytes); - - out.writeLong(entry.getTimeBucketStart()); - out.writeLong(entry.getTimeBucketEnd()); - out.writeInt(entry.getPointCount()); - out.writeLong(entry.getEntryHash()); - } - - // Footer: CRC32 checksum - out.flush(); - int crc = (int) crc32.getValue(); - // Write CRC without updating the CRC itself - bos.write((crc >>> 24) & 0xFF); - bos.write((crc >>> 16) & 0xFF); - bos.write((crc >>> 8) & 0xFF); - bos.write(crc & 0xFF); - } - } - - /** Computes the file-level dual-digest from a list of entries. */ - public static long computeFileXorHash(List entries) { - long xor = 0; - for (MerkleEntry entry : entries) { - xor ^= entry.getEntryHash(); - } - return xor; - } - - public static long computeFileAddHash(List entries) { - long add = 0; - for (MerkleEntry entry : entries) { - add += entry.getEntryHash(); - } - return add; - } - - /** - * A DataOutputStream wrapper that updates a CRC32 on every byte written, so we can compute the - * checksum in a single pass. - */ - private static class CrcDataOutputStream extends DataOutputStream { - private final CRC32 crc; - - CrcDataOutputStream(OutputStream out, CRC32 crc) { - super(out); - this.crc = crc; - } - - @Override - public synchronized void write(int b) throws IOException { - super.write(b); - crc.update(b); - } - - @Override - public synchronized void write(byte[] buf, int off, int len) throws IOException { - super.write(buf, off, len); - crc.update(buf, off, len); - } - } -} diff --git a/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/consensus/iotv2/consistency/repair/AdaptiveRepairExecutor.java b/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/consensus/iotv2/consistency/repair/AdaptiveRepairExecutor.java deleted file mode 100644 index e3d72cb0d92f5..0000000000000 --- a/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/consensus/iotv2/consistency/repair/AdaptiveRepairExecutor.java +++ /dev/null @@ -1,204 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, - * software distributed under the License is distributed on an - * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY - * KIND, either express or implied. See the License for the - * specific language governing permissions and limitations - * under the License. - */ - -package org.apache.iotdb.commons.consensus.iotv2.consistency.repair; - -import org.apache.iotdb.commons.consensus.iotv2.consistency.ibf.DiffEntry; -import org.apache.iotdb.commons.consensus.iotv2.consistency.ibf.RowRefIndex; -import org.apache.iotdb.commons.consensus.iotv2.consistency.merkle.MerkleFileContent; - -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -import java.util.ArrayList; -import java.util.HashMap; -import java.util.List; -import java.util.Map; - -/** - * Orchestrates the adaptive repair execution phase, combining direct TsFile transfers (for dense - * diffs or small TsFiles) with point-level streaming (for sparse diffs in large TsFiles). - * - *

The executor follows these steps: - * - *

    - *
  1. Attribute decoded diffs to source TsFiles via DiffAttribution - *
  2. For each TsFile, use RepairCostModel to select the optimal strategy - *
  3. Execute TsFile transfers (zero packaging cost, idempotent loading) - *
  4. Execute point streaming via RepairSession with atomic promote - *
- */ -public class AdaptiveRepairExecutor { - - private static final Logger LOGGER = LoggerFactory.getLogger(AdaptiveRepairExecutor.class); - - private final RepairCostModel costModel; - private final DiffAttribution attribution; - - /** Callback interface for executing actual repair operations. */ - public interface RepairOperationCallback { - /** Transfer an existing TsFile from Leader to Follower. */ - void transferTsFile(String tsFilePath) throws Exception; - - /** Get the size of a TsFile. */ - long getTsFileSize(String tsFilePath); - - /** Get the total point count from a TsFile's .merkle content. */ - int getTotalPointCount(String tsFilePath); - - /** Build a concrete repair record for point streaming. */ - default RepairRecord buildRepairRecord(DiffEntry diffEntry, RowRefIndex rowRefIndex) { - return null; - } - } - - public AdaptiveRepairExecutor() { - this(new RepairCostModel(), new DiffAttribution()); - } - - public AdaptiveRepairExecutor(RepairCostModel costModel, DiffAttribution attribution) { - this.costModel = costModel; - this.attribution = attribution; - } - - /** - * Build repair plans for all attributed TsFiles. - * - * @param decodedDiffs the decoded diff entries - * @param rowRefIndex the composite key resolver - * @param leaderMerkleFiles Leader's .merkle files for the checked range - * @param callback for querying TsFile metadata - * @return map of TsFile path to RepairPlan - */ - public Map buildRepairPlans( - List decodedDiffs, - RowRefIndex rowRefIndex, - List leaderMerkleFiles, - RepairOperationCallback callback) { - - // Step 1: Attribute diffs to source TsFiles - Map> attributedDiffs = - attribution.attributeToSourceTsFiles(decodedDiffs, rowRefIndex, leaderMerkleFiles); - - // Step 2: Select strategy per TsFile - Map plans = new HashMap<>(); - for (Map.Entry> entry : attributedDiffs.entrySet()) { - String tsFilePath = entry.getKey(); - List diffs = entry.getValue(); - - long tsFileSize = callback.getTsFileSize(tsFilePath); - int totalPoints = callback.getTotalPointCount(tsFilePath); - - RepairStrategy strategy = costModel.selectStrategy(tsFileSize, totalPoints, diffs.size()); - - if (strategy == RepairStrategy.DIRECT_TSFILE_TRANSFER) { - plans.put(tsFilePath, RepairPlan.directTransfer(tsFilePath, tsFileSize)); - } else { - plans.put(tsFilePath, RepairPlan.pointStreaming(tsFilePath, tsFileSize, diffs)); - } - } - - return plans; - } - - /** - * Execute all repair plans. - * - * @param plans the repair plans (per TsFile) - * @param rowRefIndex for resolving composite keys during point streaming - * @param session the RepairSession for staging point-level repairs - * @param callback for executing actual transfer operations - * @return true if all operations succeeded - */ - public boolean executeRepairPlans( - Map plans, - RowRefIndex rowRefIndex, - RepairSession session, - RepairOperationCallback callback) { - boolean allSucceeded = true; - - // Phase 1: Direct TsFile transfers (zero packaging cost) - List transferPlans = new ArrayList<>(); - List streamingPlans = new ArrayList<>(); - - for (RepairPlan plan : plans.values()) { - if (plan.getStrategy() == RepairStrategy.DIRECT_TSFILE_TRANSFER) { - transferPlans.add(plan); - } else { - streamingPlans.add(plan); - } - } - - LOGGER.info( - "Executing repair: {} TsFile transfers, {} point streaming plans", - transferPlans.size(), - streamingPlans.size()); - - for (RepairPlan plan : transferPlans) { - try { - callback.transferTsFile(plan.getTsFilePath()); - LOGGER.debug("Transferred TsFile: {}", plan.getTsFilePath()); - } catch (Exception e) { - LOGGER.error("Failed to transfer TsFile {}: {}", plan.getTsFilePath(), e.getMessage(), e); - allSucceeded = false; - } - } - - // Phase 2: Point-level streaming with atomic promote - if (!streamingPlans.isEmpty()) { - for (RepairPlan plan : streamingPlans) { - for (DiffEntry diff : plan.getDiffs()) { - RepairRecord record = callback.buildRepairRecord(diff, rowRefIndex); - if (record != null) { - session.stage(record); - LOGGER.debug("Staged diff entry for streaming: {}", diff); - } - } - } - - if (session.getStagedCount() > 0) { - boolean promoted = session.promoteAtomically(); - if (!promoted) { - LOGGER.error("Failed to atomically promote repair session"); - allSucceeded = false; - } - } - } - - return allSucceeded; - } - - /** - * Build repair plans for full-range fallback (when IBF decode fails). Transfers all overlapping - * Leader TsFiles for the affected range. - * - * @param overlappingTsFilePaths paths of Leader TsFiles overlapping the affected range - * @param callback for querying TsFile sizes - * @return list of direct transfer plans - */ - public List buildFullRangeFallbackPlans( - List overlappingTsFilePaths, RepairOperationCallback callback) { - List plans = new ArrayList<>(); - for (String tsFilePath : overlappingTsFilePaths) { - long tsFileSize = callback.getTsFileSize(tsFilePath); - plans.add(RepairPlan.directTransfer(tsFilePath, tsFileSize)); - } - return plans; - } -} diff --git a/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/consensus/iotv2/consistency/repair/DiffAttribution.java b/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/consensus/iotv2/consistency/repair/DiffAttribution.java deleted file mode 100644 index 8ea72d31c8ee4..0000000000000 --- a/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/consensus/iotv2/consistency/repair/DiffAttribution.java +++ /dev/null @@ -1,111 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, - * software distributed under the License is distributed on an - * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY - * KIND, either express or implied. See the License for the - * specific language governing permissions and limitations - * under the License. - */ - -package org.apache.iotdb.commons.consensus.iotv2.consistency.repair; - -import org.apache.iotdb.commons.consensus.iotv2.consistency.ibf.DataPointLocator; -import org.apache.iotdb.commons.consensus.iotv2.consistency.ibf.DiffEntry; -import org.apache.iotdb.commons.consensus.iotv2.consistency.ibf.RowRefIndex; -import org.apache.iotdb.commons.consensus.iotv2.consistency.merkle.MerkleEntry; -import org.apache.iotdb.commons.consensus.iotv2.consistency.merkle.MerkleFileContent; - -import java.util.ArrayList; -import java.util.HashMap; -import java.util.List; -import java.util.Map; - -/** - * Maps decoded IBF diff entries back to their source TsFiles on the Leader. Each diff point is - * attributed to the TsFile whose .merkle file contains a matching (device, measurement, timeBucket) - * entry. This attribution enables per-TsFile repair strategy selection. - */ -public class DiffAttribution { - - /** - * Attribute decoded diffs to their source TsFiles on the Leader. - * - * @param decodedDiffs the decoded diff entries from IBF - * @param rowRefIndex the index for resolving composite keys to data point locators - * @param leaderMerkleFiles .merkle file contents loaded from the Leader - * @return map from TsFile path to list of attributed diff entries - */ - public Map> attributeToSourceTsFiles( - List decodedDiffs, - RowRefIndex rowRefIndex, - List leaderMerkleFiles) { - Map>> bucketIndex = new HashMap<>(); - for (MerkleFileContent content : leaderMerkleFiles) { - for (MerkleEntry entry : content.getEntries()) { - bucketIndex - .computeIfAbsent(entry.getDeviceId(), ignored -> new HashMap<>()) - .computeIfAbsent(entry.getMeasurement(), ignored -> new ArrayList<>()) - .add( - new BucketOwner( - entry.getTimeBucketStart(), - entry.getTimeBucketEnd(), - content.getSourceTsFilePath())); - } - } - - Map> attribution = new HashMap<>(); - for (DiffEntry diff : decodedDiffs) { - DataPointLocator loc = rowRefIndex.resolve(diff.getCompositeKey()); - - String sourceTsFile = findSourceTsFile(loc, bucketIndex); - if (sourceTsFile != null) { - attribution.computeIfAbsent(sourceTsFile, k -> new ArrayList<>()).add(diff); - } - } - - return attribution; - } - - private String findSourceTsFile( - DataPointLocator loc, Map>> bucketIndex) { - List owners = - bucketIndex - .getOrDefault(loc.getDeviceId(), java.util.Collections.emptyMap()) - .get(loc.getMeasurement()); - if (owners == null) { - return null; - } - for (BucketOwner owner : owners) { - if (owner.contains(loc.getTimestamp())) { - return owner.sourceTsFilePath; - } - } - return null; - } - - private static final class BucketOwner { - private final long bucketStart; - private final long bucketEnd; - private final String sourceTsFilePath; - - private BucketOwner(long bucketStart, long bucketEnd, String sourceTsFilePath) { - this.bucketStart = bucketStart; - this.bucketEnd = bucketEnd; - this.sourceTsFilePath = sourceTsFilePath; - } - - private boolean contains(long timestamp) { - return timestamp >= bucketStart && timestamp < bucketEnd; - } - } -} diff --git a/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/consensus/iotv2/consistency/repair/ModEntrySummary.java b/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/consensus/iotv2/consistency/repair/ModEntrySummary.java deleted file mode 100644 index 95bfbee1ed312..0000000000000 --- a/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/consensus/iotv2/consistency/repair/ModEntrySummary.java +++ /dev/null @@ -1,97 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, - * software distributed under the License is distributed on an - * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY - * KIND, either express or implied. See the License for the - * specific language governing permissions and limitations - * under the License. - */ - -package org.apache.iotdb.commons.consensus.iotv2.consistency.repair; - -/** - * Summary of a mod file entry (deletion) exchanged during the NEGOTIATE_KEY_MAPPING phase. Contains - * enough information to determine if a deletion covers a specific data point, along with the - * ProgressIndex for causal ordering. - */ -public class ModEntrySummary { - - private final String devicePattern; - private final String measurementPattern; - private final long timeRangeStart; - private final long timeRangeEnd; - private final long progressIndex; - - public ModEntrySummary( - String devicePattern, - String measurementPattern, - long timeRangeStart, - long timeRangeEnd, - long progressIndex) { - this.devicePattern = devicePattern; - this.measurementPattern = measurementPattern; - this.timeRangeStart = timeRangeStart; - this.timeRangeEnd = timeRangeEnd; - this.progressIndex = progressIndex; - } - - /** - * Check if this deletion covers a specific data point. - * - * @param deviceId the device ID of the data point - * @param measurement the measurement name - * @param timestamp the timestamp of the data point - * @return true if this deletion covers the specified data point - */ - public boolean covers(String deviceId, String measurement, long timestamp) { - if (timestamp < timeRangeStart || timestamp > timeRangeEnd) { - return false; - } - return matchesPattern(devicePattern, deviceId) - && matchesPattern(measurementPattern, measurement); - } - - private boolean matchesPattern(String pattern, String value) { - if (pattern == null || pattern.equals("*") || pattern.equals("**")) { - return true; - } - return pattern.equals(value); - } - - public String getDevicePattern() { - return devicePattern; - } - - public String getMeasurementPattern() { - return measurementPattern; - } - - public long getTimeRangeStart() { - return timeRangeStart; - } - - public long getTimeRangeEnd() { - return timeRangeEnd; - } - - public long getProgressIndex() { - return progressIndex; - } - - @Override - public String toString() { - return String.format( - "ModEntrySummary{device=%s, meas=%s, range=[%d,%d], pi=%d}", - devicePattern, measurementPattern, timeRangeStart, timeRangeEnd, progressIndex); - } -} diff --git a/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/consensus/iotv2/consistency/repair/RepairAction.java b/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/consensus/iotv2/consistency/repair/RepairAction.java deleted file mode 100644 index b997733f9010e..0000000000000 --- a/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/consensus/iotv2/consistency/repair/RepairAction.java +++ /dev/null @@ -1,38 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, - * software distributed under the License is distributed on an - * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY - * KIND, either express or implied. See the License for the - * specific language governing permissions and limitations - * under the License. - */ - -package org.apache.iotdb.commons.consensus.iotv2.consistency.repair; - -/** Actions that the RepairConflictResolver can determine for a single diff entry. */ -public enum RepairAction { - /** Send the Leader's data to the Follower (insert/update). */ - SEND_TO_FOLLOWER, - /** Send the Follower's data to the Leader (insert/update). */ - SEND_TO_LEADER, - /** Delete the data on the Follower. */ - DELETE_ON_FOLLOWER, - /** Delete the data on the Leader. */ - DELETE_ON_LEADER, - /** Keep the Follower's version (it's newer or more authoritative). */ - KEEP_FOLLOWER, - /** Skip this entry (no action needed, e.g., Follower's deletion is newer). */ - SKIP, - /** Skip and raise an alert (anomalous state, e.g., data exists on Follower with no deletion). */ - SKIP_AND_ALERT -} diff --git a/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/consensus/iotv2/consistency/repair/RepairConflictResolver.java b/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/consensus/iotv2/consistency/repair/RepairConflictResolver.java deleted file mode 100644 index a99bc8ebe4aba..0000000000000 --- a/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/consensus/iotv2/consistency/repair/RepairConflictResolver.java +++ /dev/null @@ -1,145 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, - * software distributed under the License is distributed on an - * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY - * KIND, either express or implied. See the License for the - * specific language governing permissions and limitations - * under the License. - */ - -package org.apache.iotdb.commons.consensus.iotv2.consistency.repair; - -import org.apache.iotdb.commons.consensus.iotv2.consistency.ibf.DataPointLocator; - -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -import java.util.Collections; -import java.util.List; -import java.util.Optional; - -/** - * ProgressIndex-based conflict resolver for all repair decisions. Replaces the unsafe "Leader wins" - * heuristic with causal ordering via ProgressIndex comparison. This is critical for preventing - * deletion resurrection and handling leadership transfers correctly. - * - *

Diff Interpretation Matrix: - * - * - * - * - * - * - * - * - * - *
Diff TypeLeader StateFollower StateResolution
LEADER_HASPoint exists, PI=XMissingIf no Follower deletion covers it: SEND_TO_FOLLOWER. If Follower deletion PI_del > X: - * DELETE_ON_LEADER
FOLLOWER_HASMissingPoint exists, PI=YIf Leader deletion PI_del > Y: DELETE_ON_FOLLOWER. Otherwise: SEND_TO_LEADER
VALUE_DIFFPI=X, V=VlPI=Y, V=VfHigher ProgressIndex wins and repairs the stale replica
- */ -public class RepairConflictResolver { - - private static final Logger LOGGER = LoggerFactory.getLogger(RepairConflictResolver.class); - - private final List leaderDeletions; - private final List followerDeletions; - - public RepairConflictResolver( - List leaderDeletions, List followerDeletions) { - this.leaderDeletions = leaderDeletions != null ? leaderDeletions : Collections.emptyList(); - this.followerDeletions = - followerDeletions != null ? followerDeletions : Collections.emptyList(); - } - - /** - * Resolve a LEADER_HAS diff: Leader has the point but Follower doesn't. - * - * @param loc the data point location - * @param pointProgressIndex the ProgressIndex of the TsFile containing this point on the Leader - * @return the repair action to take - */ - public RepairAction resolveLeaderHas(DataPointLocator loc, long pointProgressIndex) { - Optional followerDel = findCoveringDeletion(followerDeletions, loc); - if (followerDel.isPresent() && followerDel.get().getProgressIndex() > pointProgressIndex) { - // Follower's deletion is more recent than the Leader's write -- repair the stale Leader. - LOGGER.debug( - "Deleting stale Leader point for {}: Follower deletion PI={} > point PI={}", - loc, - followerDel.get().getProgressIndex(), - pointProgressIndex); - return RepairAction.DELETE_ON_LEADER; - } - return RepairAction.SEND_TO_FOLLOWER; - } - - /** - * Resolve a FOLLOWER_HAS diff: Follower has the point but Leader doesn't. - * - * @param loc the data point location - * @param pointProgressIndex the ProgressIndex of the data on the Follower - * @return the repair action to take - */ - public RepairAction resolveFollowerHas(DataPointLocator loc, long pointProgressIndex) { - Optional leaderDel = findCoveringDeletion(leaderDeletions, loc); - if (leaderDel.isPresent() && leaderDel.get().getProgressIndex() > pointProgressIndex) { - // Leader's deletion is more recent than the Follower's write -- delete on Follower - return RepairAction.DELETE_ON_FOLLOWER; - } - - // The point can legitimately originate from a previous leadership epoch on the Follower. - return RepairAction.SEND_TO_LEADER; - } - - /** - * Resolve a VALUE_DIFF: both sides have the point but with different values. - * - * @param loc the data point location - * @param leaderProgressIndex the Leader's ProgressIndex for this point - * @param followerProgressIndex the Follower's ProgressIndex for this point - * @return the repair action to take - */ - public RepairAction resolveValueDiff( - DataPointLocator loc, long leaderProgressIndex, long followerProgressIndex) { - if (leaderProgressIndex > followerProgressIndex) { - return RepairAction.SEND_TO_FOLLOWER; - } else if (followerProgressIndex > leaderProgressIndex) { - return RepairAction.SEND_TO_LEADER; - } else { - // Concurrent writes (same ProgressIndex) -- Leader wins as tiebreaker - LOGGER.debug( - "Concurrent writes for {} with equal PI={}, using Leader as tiebreaker", - loc, - leaderProgressIndex); - return RepairAction.SEND_TO_FOLLOWER; - } - } - - /** - * Find a deletion entry that covers the given data point location. - * - * @param deletions the list of deletion summaries to search - * @param loc the data point to check - * @return the covering deletion with the highest ProgressIndex, if any - */ - static Optional findCoveringDeletion( - List deletions, DataPointLocator loc) { - ModEntrySummary bestMatch = null; - for (ModEntrySummary del : deletions) { - if (del.covers(loc.getDeviceId(), loc.getMeasurement(), loc.getTimestamp())) { - if (bestMatch == null || del.getProgressIndex() > bestMatch.getProgressIndex()) { - bestMatch = del; - } - } - } - return Optional.ofNullable(bestMatch); - } -} diff --git a/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/consensus/iotv2/consistency/repair/RepairCostModel.java b/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/consensus/iotv2/consistency/repair/RepairCostModel.java deleted file mode 100644 index 08bebc0103f0c..0000000000000 --- a/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/consensus/iotv2/consistency/repair/RepairCostModel.java +++ /dev/null @@ -1,119 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, - * software distributed under the License is distributed on an - * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY - * KIND, either express or implied. See the License for the - * specific language governing permissions and limitations - * under the License. - */ - -package org.apache.iotdb.commons.consensus.iotv2.consistency.repair; - -/** - * Cost-based strategy selector for repair execution. For each TsFile with attributed diffs, - * computes whether direct TsFile transfer or point-level streaming is more efficient. - * - *

Decision matrix: - * - *

- *                      TsFile Small (<16MB)    TsFile Medium          TsFile Large (>256MB)
- * Diff Sparse  (<1%)   Transfer (cheap)        Stream points          Stream points
- * Diff Medium (1-5%)   Transfer (cheap)        Cost-model decides     Stream points
- * Diff Dense   (>5%)   Transfer (cheap)        Transfer TsFile        Transfer TsFile
- * 
- */ -public class RepairCostModel { - - private static final double DEFAULT_COST_PER_POINT_QUERY = 0.1; - private static final double DEFAULT_COST_PER_POINT_STREAM = 0.05; - private static final double DEFAULT_COST_PER_BYTE_TRANSFER = 0.001; - private static final double DEFAULT_DIFF_DENSITY_THRESHOLD = 0.05; - private static final long DEFAULT_SMALL_TSFILE_THRESHOLD = 16L * 1024 * 1024; - - private final double costPerPointQuery; - private final double costPerPointStream; - private final double costPerByteTransfer; - private final double diffDensityThreshold; - private final long smallTsFileThreshold; - - public RepairCostModel() { - this( - DEFAULT_COST_PER_POINT_QUERY, - DEFAULT_COST_PER_POINT_STREAM, - DEFAULT_COST_PER_BYTE_TRANSFER, - DEFAULT_DIFF_DENSITY_THRESHOLD, - DEFAULT_SMALL_TSFILE_THRESHOLD); - } - - public RepairCostModel( - double costPerPointQuery, - double costPerPointStream, - double costPerByteTransfer, - double diffDensityThreshold, - long smallTsFileThreshold) { - this.costPerPointQuery = costPerPointQuery; - this.costPerPointStream = costPerPointStream; - this.costPerByteTransfer = costPerByteTransfer; - this.diffDensityThreshold = diffDensityThreshold; - this.smallTsFileThreshold = smallTsFileThreshold; - } - - /** - * Select the optimal repair strategy for a TsFile with known diff characteristics. - * - * @param tsFileSize size of the TsFile in bytes - * @param totalPointCount estimated total point count in the TsFile - * @param diffPointCount number of diff points attributed to this TsFile - * @return the selected repair strategy - */ - public RepairStrategy selectStrategy(long tsFileSize, int totalPointCount, int diffPointCount) { - // Small TsFile: always transfer (cheaper than IBF + stream overhead) - if (tsFileSize < smallTsFileThreshold) { - return RepairStrategy.DIRECT_TSFILE_TRANSFER; - } - - // High diff density: transfer whole TsFile - double diffRatio = (double) diffPointCount / Math.max(totalPointCount, 1); - if (diffRatio > diffDensityThreshold) { - return RepairStrategy.DIRECT_TSFILE_TRANSFER; - } - - // Compare costs - double streamingCost = diffPointCount * (costPerPointQuery + costPerPointStream); - double transferCost = tsFileSize * costPerByteTransfer; - - if (transferCost < streamingCost) { - return RepairStrategy.DIRECT_TSFILE_TRANSFER; - } - - return RepairStrategy.POINT_STREAMING; - } - - /** - * Determine if a TsFile is small enough to skip IBF entirely (Merkle-level short circuit). - * - * @param tsFileSize size of the TsFile in bytes - * @return true if the TsFile should be directly transferred without IBF - */ - public boolean shouldBypassIBF(long tsFileSize) { - return tsFileSize < smallTsFileThreshold; - } - - public long getSmallTsFileThreshold() { - return smallTsFileThreshold; - } - - public double getDiffDensityThreshold() { - return diffDensityThreshold; - } -} diff --git a/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/consensus/iotv2/consistency/repair/RepairPlan.java b/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/consensus/iotv2/consistency/repair/RepairPlan.java deleted file mode 100644 index 9337d64b3508f..0000000000000 --- a/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/consensus/iotv2/consistency/repair/RepairPlan.java +++ /dev/null @@ -1,80 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, - * software distributed under the License is distributed on an - * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY - * KIND, either express or implied. See the License for the - * specific language governing permissions and limitations - * under the License. - */ - -package org.apache.iotdb.commons.consensus.iotv2.consistency.repair; - -import org.apache.iotdb.commons.consensus.iotv2.consistency.ibf.DiffEntry; - -import java.util.ArrayList; -import java.util.List; - -/** - * Repair plan for a single TsFile, containing the selected strategy and the attributed diff - * entries. - */ -public class RepairPlan { - - private final String tsFilePath; - private final long tsFileSize; - private final RepairStrategy strategy; - private final List diffs; - - public RepairPlan( - String tsFilePath, long tsFileSize, RepairStrategy strategy, List diffs) { - this.tsFilePath = tsFilePath; - this.tsFileSize = tsFileSize; - this.strategy = strategy; - this.diffs = new ArrayList<>(diffs); - } - - /** Create a plan for direct TsFile transfer (no individual diff entries needed). */ - public static RepairPlan directTransfer(String tsFilePath, long tsFileSize) { - return new RepairPlan( - tsFilePath, tsFileSize, RepairStrategy.DIRECT_TSFILE_TRANSFER, new ArrayList<>()); - } - - /** Create a plan for point streaming with specific diff entries. */ - public static RepairPlan pointStreaming( - String tsFilePath, long tsFileSize, List diffs) { - return new RepairPlan(tsFilePath, tsFileSize, RepairStrategy.POINT_STREAMING, diffs); - } - - public String getTsFilePath() { - return tsFilePath; - } - - public long getTsFileSize() { - return tsFileSize; - } - - public RepairStrategy getStrategy() { - return strategy; - } - - public List getDiffs() { - return diffs; - } - - @Override - public String toString() { - return String.format( - "RepairPlan{file=%s, size=%d, strategy=%s, diffs=%d}", - tsFilePath, tsFileSize, strategy, diffs.size()); - } -} diff --git a/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/consensus/iotv2/consistency/repair/RepairRecord.java b/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/consensus/iotv2/consistency/repair/RepairRecord.java deleted file mode 100644 index 9cac62bfce38d..0000000000000 --- a/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/consensus/iotv2/consistency/repair/RepairRecord.java +++ /dev/null @@ -1,132 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, - * software distributed under the License is distributed on an - * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY - * KIND, either express or implied. See the License for the - * specific language governing permissions and limitations - * under the License. - */ - -package org.apache.iotdb.commons.consensus.iotv2.consistency.repair; - -import org.apache.iotdb.commons.consensus.iotv2.consistency.ibf.DataPointLocator; - -/** - * A single repair record representing either an insert or delete operation to be applied during - * point-level streaming repair. Each record carries the ProgressIndex from its source for conflict - * resolution. - */ -public class RepairRecord { - - public enum RecordType { - INSERT, - DELETE - } - - public enum TargetReplica { - LEADER, - FOLLOWER - } - - private final RecordType type; - private final TargetReplica targetReplica; - private final DataPointLocator locator; - private final long progressIndex; - private final Object value; - private final long timestamp; - - private RepairRecord( - RecordType type, - TargetReplica targetReplica, - DataPointLocator locator, - long progressIndex, - Object value, - long timestamp) { - this.type = type; - this.targetReplica = targetReplica; - this.locator = locator; - this.progressIndex = progressIndex; - this.value = value; - this.timestamp = timestamp; - } - - /** Create an insert repair record. */ - public static RepairRecord insert( - DataPointLocator locator, long progressIndex, Object value, long timestamp) { - return insertToFollower(locator, progressIndex, value, timestamp); - } - - /** Create a delete repair record. */ - public static RepairRecord delete(DataPointLocator locator, long progressIndex, long timestamp) { - return deleteOnFollower(locator, progressIndex, timestamp); - } - - /** Create an insert repair record that targets the leader replica. */ - public static RepairRecord insertToLeader( - DataPointLocator locator, long progressIndex, Object value, long timestamp) { - return new RepairRecord( - RecordType.INSERT, TargetReplica.LEADER, locator, progressIndex, value, timestamp); - } - - /** Create an insert repair record that targets the follower replica. */ - public static RepairRecord insertToFollower( - DataPointLocator locator, long progressIndex, Object value, long timestamp) { - return new RepairRecord( - RecordType.INSERT, TargetReplica.FOLLOWER, locator, progressIndex, value, timestamp); - } - - /** Create a delete repair record that targets the leader replica. */ - public static RepairRecord deleteOnLeader( - DataPointLocator locator, long progressIndex, long timestamp) { - return new RepairRecord( - RecordType.DELETE, TargetReplica.LEADER, locator, progressIndex, null, timestamp); - } - - /** Create a delete repair record that targets the follower replica. */ - public static RepairRecord deleteOnFollower( - DataPointLocator locator, long progressIndex, long timestamp) { - return new RepairRecord( - RecordType.DELETE, TargetReplica.FOLLOWER, locator, progressIndex, null, timestamp); - } - - public RecordType getType() { - return type; - } - - public TargetReplica getTargetReplica() { - return targetReplica; - } - - public DataPointLocator getLocator() { - return locator; - } - - public long getProgressIndex() { - return progressIndex; - } - - public Object getValue() { - return value; - } - - public long getTimestamp() { - return timestamp; - } - - @Override - public String toString() { - return String.format( - "RepairRecord{type=%s, target=%s, loc=%s, pi=%d}", - type, targetReplica, locator, progressIndex); - } -} diff --git a/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/consensus/iotv2/consistency/repair/RepairSession.java b/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/consensus/iotv2/consistency/repair/RepairSession.java deleted file mode 100644 index f795b576ed50d..0000000000000 --- a/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/consensus/iotv2/consistency/repair/RepairSession.java +++ /dev/null @@ -1,197 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, - * software distributed under the License is distributed on an - * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY - * KIND, either express or implied. See the License for the - * specific language governing permissions and limitations - * under the License. - */ - -package org.apache.iotdb.commons.consensus.iotv2.consistency.repair; - -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -import java.util.ArrayList; -import java.util.Collections; -import java.util.List; -import java.util.UUID; - -/** - * WAL-backed staging area for streaming repair records with atomic promote semantics. All staged - * records are committed atomically (all-or-nothing) via promoteAtomically(). Uncommitted sessions - * are discarded on restart for crash safety. - * - *

Each RepairSession has a unique sessionId for idempotent replay detection. - */ -public class RepairSession { - - private static final Logger LOGGER = LoggerFactory.getLogger(RepairSession.class); - - public enum SessionState { - STAGING, - COMMITTED, - ABORTED - } - - public interface RepairSessionApplier { - void apply( - String sessionId, long partitionId, List inserts, List deletes) - throws Exception; - } - - public interface RepairSessionJournal { - void append(String sessionId, RepairRecord record) throws Exception; - - void markCommitted(String sessionId) throws Exception; - - void delete(String sessionId); - } - - private static final RepairSessionApplier NO_OP_APPLIER = - (sessionId, partitionId, inserts, deletes) -> {}; - - private static final RepairSessionJournal NO_OP_JOURNAL = - new RepairSessionJournal() { - @Override - public void append(String sessionId, RepairRecord record) {} - - @Override - public void markCommitted(String sessionId) {} - - @Override - public void delete(String sessionId) {} - }; - - private final String sessionId; - private final long partitionId; - private final List stagedRecords; - private final RepairSessionApplier applier; - private final RepairSessionJournal journal; - private volatile SessionState state; - - public RepairSession(long partitionId) { - this(partitionId, NO_OP_APPLIER, NO_OP_JOURNAL); - } - - public RepairSession(long partitionId, RepairSessionApplier applier) { - this(partitionId, applier, NO_OP_JOURNAL); - } - - public RepairSession( - long partitionId, RepairSessionApplier applier, RepairSessionJournal journal) { - this.sessionId = UUID.randomUUID().toString(); - this.partitionId = partitionId; - this.stagedRecords = new ArrayList<>(); - this.applier = applier == null ? NO_OP_APPLIER : applier; - this.journal = journal == null ? NO_OP_JOURNAL : journal; - this.state = SessionState.STAGING; - } - - /** - * Stage a repair record for atomic commit. The record is appended to the WAL before being added - * to the in-memory list. - * - * @param record the repair record to stage - * @throws IllegalStateException if the session is not in STAGING state - */ - public void stage(RepairRecord record) { - if (state != SessionState.STAGING) { - throw new IllegalStateException( - "Cannot stage records in session " + sessionId + " with state " + state); - } - try { - journal.append(sessionId, record); - } catch (Exception e) { - throw new IllegalStateException( - "Failed to append repair record into journal for session " + sessionId, e); - } - stagedRecords.add(record); - } - - /** - * @return true if promotion succeeded - */ - public boolean promoteAtomically() { - if (state != SessionState.STAGING) { - LOGGER.warn("Cannot promote session {} in state {}", sessionId, state); - return false; - } - - try { - // Separate inserts and deletes - List inserts = new ArrayList<>(); - List deletes = new ArrayList<>(); - for (RepairRecord record : stagedRecords) { - if (record.getType() == RepairRecord.RecordType.INSERT) { - inserts.add(record); - } else { - deletes.add(record); - } - } - applier.apply( - sessionId, - partitionId, - Collections.unmodifiableList(inserts), - Collections.unmodifiableList(deletes)); - journal.markCommitted(sessionId); - - LOGGER.info( - "RepairSession {} promoted: {} inserts, {} deletes for partition {}", - sessionId, - inserts.size(), - deletes.size(), - partitionId); - - state = SessionState.COMMITTED; - return true; - } catch (Exception e) { - LOGGER.error("Failed to promote RepairSession {}: {}", sessionId, e.getMessage(), e); - state = SessionState.ABORTED; - return false; - } - } - - /** Abort and clean up the session. Staged records are discarded. */ - public void abort() { - state = SessionState.ABORTED; - stagedRecords.clear(); - journal.delete(sessionId); - } - - /** Clean up WAL resources after successful promotion. */ - public void cleanup() { - journal.delete(sessionId); - stagedRecords.clear(); - } - - public String getSessionId() { - return sessionId; - } - - public long getPartitionId() { - return partitionId; - } - - public SessionState getState() { - return state; - } - - public List getStagedRecords() { - return Collections.unmodifiableList(stagedRecords); - } - - public int getStagedCount() { - return stagedRecords.size(); - } -} diff --git a/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/consensus/iotv2/consistency/repair/RepairStrategy.java b/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/consensus/iotv2/consistency/repair/RepairStrategy.java deleted file mode 100644 index 26b6e1d0ecb9a..0000000000000 --- a/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/consensus/iotv2/consistency/repair/RepairStrategy.java +++ /dev/null @@ -1,28 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, - * software distributed under the License is distributed on an - * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY - * KIND, either express or implied. See the License for the - * specific language governing permissions and limitations - * under the License. - */ - -package org.apache.iotdb.commons.consensus.iotv2.consistency.repair; - -/** Per-TsFile repair strategy selected by the cost model. */ -public enum RepairStrategy { - /** Send entire existing TsFile from Leader to Follower. Zero packaging cost. */ - DIRECT_TSFILE_TRANSFER, - /** Stream individual diff points via RepairSession with atomic promote. */ - POINT_STREAMING -} diff --git a/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/schema/column/ColumnHeaderConstant.java b/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/schema/column/ColumnHeaderConstant.java index 7f6fedfc224f7..e8a1d1a3a9cbc 100644 --- a/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/schema/column/ColumnHeaderConstant.java +++ b/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/schema/column/ColumnHeaderConstant.java @@ -247,6 +247,20 @@ private ColumnHeaderConstant() { public static final String REGION_ID_TABLE_MODEL = "region_id"; public static final String DATANODE_ID_TABLE_MODEL = "datanode_id"; + public static final String TIME_PARTITION_TABLE_MODEL = "time_partition"; + public static final String CHECK_STATE_TABLE_MODEL = "check_state"; + public static final String REPAIR_STATE_TABLE_MODEL = "repair_state"; + public static final String LAST_CHECKED_AT_TABLE_MODEL = "last_checked_at"; + public static final String LAST_SAFE_WATERMARK_TABLE_MODEL = "last_safe_watermark"; + public static final String PARTITION_MUTATION_EPOCH_TABLE_MODEL = "partition_mutation_epoch"; + public static final String SNAPSHOT_EPOCH_TABLE_MODEL = "snapshot_epoch"; + public static final String SNAPSHOT_STATE_TABLE_MODEL = "snapshot_state"; + public static final String LAST_MISMATCH_AT_TABLE_MODEL = "last_mismatch_at"; + public static final String MISMATCH_SCOPE_REF_TABLE_MODEL = "mismatch_scope_ref"; + public static final String MISMATCH_LEAF_COUNT_TABLE_MODEL = "mismatch_leaf_count"; + public static final String REPAIR_EPOCH_TABLE_MODEL = "repair_epoch"; + public static final String LAST_ERROR_CODE_TABLE_MODEL = "last_error_code"; + public static final String LAST_ERROR_MESSAGE_TABLE_MODEL = "last_error_message"; public static final String SERIES_SLOT_NUM_TABLE_MODEL = "series_slot_num"; public static final String TIME_SLOT_NUM_TABLE_MODEL = "time_slot_num"; public static final String RPC_ADDRESS_TABLE_MODEL = "rpc_address"; diff --git a/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/schema/table/InformationSchema.java b/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/schema/table/InformationSchema.java index ac6c8ebab24f2..787ebe75a4988 100644 --- a/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/schema/table/InformationSchema.java +++ b/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/schema/table/InformationSchema.java @@ -38,6 +38,7 @@ public class InformationSchema { public static final String TABLES = "tables"; public static final String COLUMNS = "columns"; public static final String REGIONS = "regions"; + public static final String REPAIR_PROGRESS = "repair_progress"; public static final String PIPES = "pipes"; public static final String PIPE_PLUGINS = "pipe_plugins"; public static final String TOPICS = "topics"; @@ -177,6 +178,51 @@ public class InformationSchema { ColumnHeaderConstant.COMPRESSION_RATIO_TABLE_MODEL, TSDataType.DOUBLE)); schemaTables.put(REGIONS, regionTable); + final TsTable repairProgressTable = new TsTable(REPAIR_PROGRESS); + repairProgressTable.addColumnSchema( + new TagColumnSchema(ColumnHeaderConstant.REGION_ID_TABLE_MODEL, TSDataType.INT32)); + repairProgressTable.addColumnSchema( + new TagColumnSchema(ColumnHeaderConstant.TIME_PARTITION_TABLE_MODEL, TSDataType.INT64)); + repairProgressTable.addColumnSchema( + new AttributeColumnSchema(ColumnHeaderConstant.CHECK_STATE_TABLE_MODEL, TSDataType.STRING)); + repairProgressTable.addColumnSchema( + new AttributeColumnSchema( + ColumnHeaderConstant.REPAIR_STATE_TABLE_MODEL, TSDataType.STRING)); + repairProgressTable.addColumnSchema( + new AttributeColumnSchema( + ColumnHeaderConstant.LAST_CHECKED_AT_TABLE_MODEL, TSDataType.TIMESTAMP)); + repairProgressTable.addColumnSchema( + new AttributeColumnSchema( + ColumnHeaderConstant.LAST_SAFE_WATERMARK_TABLE_MODEL, TSDataType.INT64)); + repairProgressTable.addColumnSchema( + new AttributeColumnSchema( + ColumnHeaderConstant.PARTITION_MUTATION_EPOCH_TABLE_MODEL, TSDataType.INT64)); + repairProgressTable.addColumnSchema( + new AttributeColumnSchema( + ColumnHeaderConstant.SNAPSHOT_EPOCH_TABLE_MODEL, TSDataType.INT64)); + repairProgressTable.addColumnSchema( + new AttributeColumnSchema( + ColumnHeaderConstant.SNAPSHOT_STATE_TABLE_MODEL, TSDataType.STRING)); + repairProgressTable.addColumnSchema( + new AttributeColumnSchema( + ColumnHeaderConstant.LAST_MISMATCH_AT_TABLE_MODEL, TSDataType.TIMESTAMP)); + repairProgressTable.addColumnSchema( + new AttributeColumnSchema( + ColumnHeaderConstant.MISMATCH_SCOPE_REF_TABLE_MODEL, TSDataType.STRING)); + repairProgressTable.addColumnSchema( + new AttributeColumnSchema( + ColumnHeaderConstant.MISMATCH_LEAF_COUNT_TABLE_MODEL, TSDataType.INT32)); + repairProgressTable.addColumnSchema( + new AttributeColumnSchema( + ColumnHeaderConstant.REPAIR_EPOCH_TABLE_MODEL, TSDataType.STRING)); + repairProgressTable.addColumnSchema( + new AttributeColumnSchema( + ColumnHeaderConstant.LAST_ERROR_CODE_TABLE_MODEL, TSDataType.STRING)); + repairProgressTable.addColumnSchema( + new AttributeColumnSchema( + ColumnHeaderConstant.LAST_ERROR_MESSAGE_TABLE_MODEL, TSDataType.STRING)); + schemaTables.put(REPAIR_PROGRESS, repairProgressTable); + final TsTable pipeTable = new TsTable(PIPES); pipeTable.addColumnSchema( new TagColumnSchema( diff --git a/iotdb-core/node-commons/src/test/java/org/apache/iotdb/commons/consensus/iotv2/consistency/ConsistencyMerkleTreeTest.java b/iotdb-core/node-commons/src/test/java/org/apache/iotdb/commons/consensus/iotv2/consistency/ConsistencyMerkleTreeTest.java deleted file mode 100644 index c0ca0a9b0b3b5..0000000000000 --- a/iotdb-core/node-commons/src/test/java/org/apache/iotdb/commons/consensus/iotv2/consistency/ConsistencyMerkleTreeTest.java +++ /dev/null @@ -1,61 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, - * software distributed under the License is distributed on an - * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY - * KIND, either express or implied. See the License for the - * specific language governing permissions and limitations - * under the License. - */ - -package org.apache.iotdb.commons.consensus.iotv2.consistency; - -import org.junit.Assert; -import org.junit.Test; - -import java.util.Arrays; -import java.util.Collections; - -public class ConsistencyMerkleTreeTest { - - @Test - public void shouldPreserveDualDigestOnFlush() { - ConsistencyMerkleTree merkleTree = new ConsistencyMerkleTree(); - DualDigest first = new DualDigest(0x01L, 0x10L); - DualDigest second = new DualDigest(0x02L, 0x20L); - - merkleTree.onTsFileFlushed(1L, first); - merkleTree.onTsFileFlushed(1L, second); - - Assert.assertEquals( - new DualDigest(0x03L, 0x30L), merkleTree.getPartitionNode(1L).getPartitionDigest()); - Assert.assertEquals(new DualDigest(0x03L, 0x30L), merkleTree.getRegionDigest()); - } - - @Test - public void shouldPreserveDualDigestOnCompaction() { - ConsistencyMerkleTree merkleTree = new ConsistencyMerkleTree(); - DualDigest sourceOne = new DualDigest(0x01L, 0x10L); - DualDigest sourceTwo = new DualDigest(0x02L, 0x20L); - DualDigest targetOne = new DualDigest(0x04L, 0x40L); - DualDigest targetTwo = new DualDigest(0x08L, 0x80L); - - merkleTree.onTsFileFlushed(7L, sourceOne); - merkleTree.onTsFileFlushed(7L, sourceTwo); - merkleTree.onCompaction( - Collections.singletonList(sourceOne), Arrays.asList(targetOne, targetTwo), 7L); - - Assert.assertEquals( - new DualDigest(0x0EL, 0xE0L), merkleTree.getPartitionNode(7L).getPartitionDigest()); - Assert.assertEquals(new DualDigest(0x0EL, 0xE0L), merkleTree.getRegionDigest()); - } -} diff --git a/iotdb-core/node-commons/src/test/java/org/apache/iotdb/commons/consensus/iotv2/consistency/LogicalMismatchScopeTest.java b/iotdb-core/node-commons/src/test/java/org/apache/iotdb/commons/consensus/iotv2/consistency/LogicalMismatchScopeTest.java new file mode 100644 index 0000000000000..0ac65a1d3bcf3 --- /dev/null +++ b/iotdb-core/node-commons/src/test/java/org/apache/iotdb/commons/consensus/iotv2/consistency/LogicalMismatchScopeTest.java @@ -0,0 +1,98 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.iotdb.commons.consensus.iotv2.consistency; + +import org.junit.Assert; +import org.junit.Test; + +import java.util.Arrays; +import java.util.Collections; +import java.util.List; + +public class LogicalMismatchScopeTest { + + @Test + public void shouldRoundTripLegacyLeafScope() { + List scopes = + LogicalMismatchScope.deserialize("LIVE@leaf:1:0,TOMBSTONE@leaf:2:0"); + + Assert.assertEquals( + Arrays.asList( + new LogicalMismatchScope.Scope("LIVE", "leaf:1:0"), + new LogicalMismatchScope.Scope("TOMBSTONE", "leaf:2:0")), + scopes); + Assert.assertEquals("LIVE@leaf:1:0,TOMBSTONE@leaf:2:0", LogicalMismatchScope.serialize(scopes)); + } + + @Test + public void shouldPersistOptionalLogicalKeyRange() { + List scopes = + Collections.singletonList( + new LogicalMismatchScope.Scope( + "LIVE", "leaf:3:0", "root.db.d1|s1|1|INT64|1", "root.db.d1|s1|9|INT64|9")); + + String serialized = LogicalMismatchScope.serialize(scopes); + List recovered = LogicalMismatchScope.deserialize(serialized); + + Assert.assertEquals(scopes, recovered); + } + + @Test + public void shouldPersistExactLogicalKeysForMicroRepair() { + List scopes = + Collections.singletonList( + new LogicalMismatchScope.Scope( + "LIVE", + "leaf:7:0", + "root.db.d1|s1|1|INT64|1", + "root.db.d1|s1|5|INT64|5", + Arrays.asList( + "root.db.d1|s1|1|INT64|1", + "root.db.d1|s2|2|INT64|2", + "root.db.d2|s1|5|INT64|5"))); + + String serialized = LogicalMismatchScope.serialize(scopes); + List recovered = LogicalMismatchScope.deserialize(serialized); + + Assert.assertEquals(scopes, recovered); + } + + @Test + public void shouldPersistNonRepairableDirective() { + List scopes = + Collections.singletonList( + new LogicalMismatchScope.Scope( + "TOMBSTONE", + "leaf:9:0", + "root.db.d1|s1|1|INT64|1", + "root.db.d1|s1|5|INT64|5", + Collections.singletonList("root.db.d1|s1|1|INT64|1"), + LogicalMismatchScope.RepairDirective.FOLLOWER_EXTRA_TOMBSTONE)); + + String serialized = LogicalMismatchScope.serialize(scopes); + List recovered = LogicalMismatchScope.deserialize(serialized); + + Assert.assertEquals(scopes, recovered); + Assert.assertFalse(recovered.get(0).isRepairable()); + Assert.assertEquals( + LogicalMismatchScope.RepairDirective.FOLLOWER_EXTRA_TOMBSTONE, + recovered.get(0).getRepairDirective()); + } +} diff --git a/iotdb-core/node-commons/src/test/java/org/apache/iotdb/commons/consensus/iotv2/consistency/RepairProgressTableTest.java b/iotdb-core/node-commons/src/test/java/org/apache/iotdb/commons/consensus/iotv2/consistency/RepairProgressTableTest.java new file mode 100644 index 0000000000000..b4b6c9789f8a7 --- /dev/null +++ b/iotdb-core/node-commons/src/test/java/org/apache/iotdb/commons/consensus/iotv2/consistency/RepairProgressTableTest.java @@ -0,0 +1,129 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.iotdb.commons.consensus.iotv2.consistency; + +import org.apache.tsfile.utils.PublicBAOS; +import org.junit.Assert; +import org.junit.Test; + +import java.io.DataOutputStream; +import java.nio.ByteBuffer; + +public class RepairProgressTableTest { + + @Test + public void verifiedPartitionWithSameSnapshotShouldBePrunedFromNextCheck() { + RepairProgressTable table = new RepairProgressTable("DataRegion-1"); + + table.markVerified(100L, 1000L, 2000L, 3000L, 3000L, RepairProgressTable.SnapshotState.READY); + + RepairProgressTable.PartitionProgress progress = table.getPartition(100L); + Assert.assertNotNull(progress); + Assert.assertFalse(progress.shouldCheck(3000L, 3000L, RepairProgressTable.SnapshotState.READY)); + Assert.assertTrue(progress.shouldCheck(3001L, 3001L, RepairProgressTable.SnapshotState.READY)); + } + + @Test + public void replicaObservationTokenChangeShouldForceRecheck() { + RepairProgressTable table = new RepairProgressTable("DataRegion-1"); + + table.markVerified( + 100L, + 1000L, + 2000L, + 3000L, + 3000L, + RepairProgressTable.SnapshotState.READY, + "leader=4|follower=3:snapshot=3000"); + + RepairProgressTable.PartitionProgress progress = table.getPartition(100L); + Assert.assertNotNull(progress); + Assert.assertFalse( + progress.shouldCheck( + 3000L, + 3000L, + RepairProgressTable.SnapshotState.READY, + "leader=4|follower=3:snapshot=3000")); + Assert.assertTrue( + progress.shouldCheck( + 3000L, + 3000L, + RepairProgressTable.SnapshotState.READY, + "leader=4|follower=3:snapshot=0")); + } + + @Test + public void dirtyTransitionShouldClearVerifiedRepairStateAndScope() { + RepairProgressTable table = new RepairProgressTable("DataRegion-1"); + + table.markMismatch( + 100L, + 1000L, + 2000L, + 3000L, + 3000L, + RepairProgressTable.SnapshotState.READY, + "LIVE@leaf:1:0", + 1, + "leader:1:2000:3000:3000"); + table.markRepairSucceeded( + 100L, + 1100L, + 2100L, + 3001L, + 3001L, + RepairProgressTable.SnapshotState.READY, + "leader:1:2000:3000:3000"); + table.markDirty(100L); + + RepairProgressTable.PartitionProgress progress = table.getPartition(100L); + Assert.assertEquals(RepairProgressTable.CheckState.DIRTY, progress.getCheckState()); + Assert.assertEquals(RepairProgressTable.RepairState.IDLE, progress.getRepairState()); + Assert.assertNull(progress.getMismatchScopeRef()); + } + + @Test + public void serDeShouldPreserveCheckAndRepairState() throws Exception { + RepairProgressTable table = new RepairProgressTable("DataRegion-7"); + table.markMismatch( + 10L, + 111L, + 222L, + 333L, + 333L, + RepairProgressTable.SnapshotState.READY, + "TOMBSTONE@leaf:7:1", + 2, + "leader:7:222:333:333"); + table.markRepairRunning(10L, "leader:7:222:333:333"); + table.markRepairFailed(10L, "leader:7:222:333:333", "ERR_CODE", "ERR_MSG"); + + RepairProgressTable restored; + try (PublicBAOS byteArrayOutputStream = new PublicBAOS(); + DataOutputStream outputStream = new DataOutputStream(byteArrayOutputStream)) { + table.serialize(outputStream); + restored = + RepairProgressTable.deserialize( + ByteBuffer.wrap(byteArrayOutputStream.getBuf(), 0, byteArrayOutputStream.size())); + } + + Assert.assertEquals(table, restored); + } +} diff --git a/iotdb-core/node-commons/src/test/java/org/apache/iotdb/commons/consensus/iotv2/consistency/repair/DiffAttributionTest.java b/iotdb-core/node-commons/src/test/java/org/apache/iotdb/commons/consensus/iotv2/consistency/repair/DiffAttributionTest.java deleted file mode 100644 index ac738a14dce12..0000000000000 --- a/iotdb-core/node-commons/src/test/java/org/apache/iotdb/commons/consensus/iotv2/consistency/repair/DiffAttributionTest.java +++ /dev/null @@ -1,70 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, - * software distributed under the License is distributed on an - * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY - * KIND, either express or implied. See the License for the - * specific language governing permissions and limitations - * under the License. - */ - -package org.apache.iotdb.commons.consensus.iotv2.consistency.repair; - -import org.apache.iotdb.commons.consensus.iotv2.consistency.ibf.CompositeKeyCodec; -import org.apache.iotdb.commons.consensus.iotv2.consistency.ibf.DiffEntry; -import org.apache.iotdb.commons.consensus.iotv2.consistency.ibf.RowRefIndex; -import org.apache.iotdb.commons.consensus.iotv2.consistency.merkle.MerkleEntry; -import org.apache.iotdb.commons.consensus.iotv2.consistency.merkle.MerkleFileContent; - -import org.junit.Assert; -import org.junit.Test; - -import java.util.Arrays; -import java.util.Collections; -import java.util.List; -import java.util.Map; - -public class DiffAttributionTest { - - @Test - public void shouldUseActualBucketBoundariesWhenAttributingDiffs() { - DiffAttribution diffAttribution = new DiffAttribution(); - RowRefIndex rowRefIndex = - new RowRefIndex.Builder() - .addDevice("root.sg.d1", Collections.singletonList("s1")) - .setTimeBucketStart(0L) - .build(); - long compositeKey = CompositeKeyCodec.encode(0, 0, 250L, 0L, 1L); - DiffEntry diffEntry = new DiffEntry(compositeKey, 1L, DiffEntry.DiffType.LEADER_HAS); - List merkleFiles = - Arrays.asList( - new MerkleFileContent( - 1L, - 10L, - Collections.singletonList( - new MerkleEntry("root.sg.d1", "s1", 100L, 200L, 10, 101L)), - "first.tsfile"), - new MerkleFileContent( - 2L, - 20L, - Collections.singletonList( - new MerkleEntry("root.sg.d1", "s1", 200L, 300L, 10, 202L)), - "second.tsfile")); - - Map> attributed = - diffAttribution.attributeToSourceTsFiles( - Collections.singletonList(diffEntry), rowRefIndex, merkleFiles); - - Assert.assertFalse(attributed.containsKey("first.tsfile")); - Assert.assertEquals(Collections.singletonList(diffEntry), attributed.get("second.tsfile")); - } -} diff --git a/iotdb-core/node-commons/src/test/java/org/apache/iotdb/commons/consensus/iotv2/consistency/repair/RepairConflictResolverTest.java b/iotdb-core/node-commons/src/test/java/org/apache/iotdb/commons/consensus/iotv2/consistency/repair/RepairConflictResolverTest.java deleted file mode 100644 index 7683764b8d515..0000000000000 --- a/iotdb-core/node-commons/src/test/java/org/apache/iotdb/commons/consensus/iotv2/consistency/repair/RepairConflictResolverTest.java +++ /dev/null @@ -1,74 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, - * software distributed under the License is distributed on an - * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY - * KIND, either express or implied. See the License for the - * specific language governing permissions and limitations - * under the License. - */ - -package org.apache.iotdb.commons.consensus.iotv2.consistency.repair; - -import org.apache.iotdb.commons.consensus.iotv2.consistency.ibf.DataPointLocator; - -import org.junit.Assert; -import org.junit.Test; - -import java.util.Collections; - -public class RepairConflictResolverTest { - - @Test - public void leaderHasWithoutNewerFollowerDeletionShouldSendToFollower() { - RepairConflictResolver resolver = - new RepairConflictResolver(Collections.emptyList(), Collections.emptyList()); - - Assert.assertEquals( - RepairAction.SEND_TO_FOLLOWER, - resolver.resolveLeaderHas(new DataPointLocator("root.sg.d1", "s1", 100L), 10L)); - } - - @Test - public void leaderHasWithNewerFollowerDeletionShouldDeleteOnLeader() { - RepairConflictResolver resolver = - new RepairConflictResolver( - Collections.emptyList(), - Collections.singletonList(new ModEntrySummary("root.sg.d1", "s1", 0L, 200L, 20L))); - - Assert.assertEquals( - RepairAction.DELETE_ON_LEADER, - resolver.resolveLeaderHas(new DataPointLocator("root.sg.d1", "s1", 100L), 10L)); - } - - @Test - public void followerHasWithoutNewerLeaderDeletionShouldSendToLeader() { - RepairConflictResolver resolver = - new RepairConflictResolver(Collections.emptyList(), Collections.emptyList()); - - Assert.assertEquals( - RepairAction.SEND_TO_LEADER, - resolver.resolveFollowerHas(new DataPointLocator("root.sg.d1", "s1", 100L), 10L)); - } - - @Test - public void followerHasWithNewerLeaderDeletionShouldDeleteOnFollower() { - RepairConflictResolver resolver = - new RepairConflictResolver( - Collections.singletonList(new ModEntrySummary("root.sg.d1", "s1", 0L, 200L, 20L)), - Collections.emptyList()); - - Assert.assertEquals( - RepairAction.DELETE_ON_FOLLOWER, - resolver.resolveFollowerHas(new DataPointLocator("root.sg.d1", "s1", 100L), 10L)); - } -} diff --git a/iotdb-core/node-commons/src/test/java/org/apache/iotdb/commons/consensus/iotv2/consistency/repair/RepairSessionTest.java b/iotdb-core/node-commons/src/test/java/org/apache/iotdb/commons/consensus/iotv2/consistency/repair/RepairSessionTest.java deleted file mode 100644 index 2c91b46a72ff9..0000000000000 --- a/iotdb-core/node-commons/src/test/java/org/apache/iotdb/commons/consensus/iotv2/consistency/repair/RepairSessionTest.java +++ /dev/null @@ -1,80 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, - * software distributed under the License is distributed on an - * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY - * KIND, either express or implied. See the License for the - * specific language governing permissions and limitations - * under the License. - */ - -package org.apache.iotdb.commons.consensus.iotv2.consistency.repair; - -import org.apache.iotdb.commons.consensus.iotv2.consistency.ibf.DataPointLocator; - -import org.junit.Assert; -import org.junit.Test; - -import java.util.ArrayList; -import java.util.Arrays; -import java.util.Collections; -import java.util.List; - -public class RepairSessionTest { - - @Test - public void shouldApplyAndJournalRecordsAtomically() { - List journalEvents = new ArrayList<>(); - List appliedInserts = new ArrayList<>(); - List appliedDeletes = new ArrayList<>(); - RepairSession session = - new RepairSession( - 5L, - (sessionId, partitionId, inserts, deletes) -> { - appliedInserts.addAll(inserts); - appliedDeletes.addAll(deletes); - }, - new RepairSession.RepairSessionJournal() { - @Override - public void append(String sessionId, RepairRecord record) { - journalEvents.add("append:" + record.getType()); - } - - @Override - public void markCommitted(String sessionId) { - journalEvents.add("commit"); - } - - @Override - public void delete(String sessionId) { - journalEvents.add("delete"); - } - }); - RepairRecord insert = - RepairRecord.insert(new DataPointLocator("root.sg.d1", "s1", 100L), 1L, "v1", 100L); - RepairRecord delete = - RepairRecord.delete(new DataPointLocator("root.sg.d1", "s1", 101L), 2L, 101L); - - session.stage(insert); - session.stage(delete); - - Assert.assertTrue(session.promoteAtomically()); - Assert.assertEquals(RepairSession.SessionState.COMMITTED, session.getState()); - Assert.assertEquals(Collections.singletonList(insert), appliedInserts); - Assert.assertEquals(Collections.singletonList(delete), appliedDeletes); - - session.cleanup(); - - Assert.assertEquals( - Arrays.asList("append:INSERT", "append:DELETE", "commit", "delete"), journalEvents); - } -} diff --git a/iotdb-protocol/thrift-confignode/src/main/thrift/confignode.thrift b/iotdb-protocol/thrift-confignode/src/main/thrift/confignode.thrift index c34bd594f5204..29701d1ecea8b 100644 --- a/iotdb-protocol/thrift-confignode/src/main/thrift/confignode.thrift +++ b/iotdb-protocol/thrift-confignode/src/main/thrift/confignode.thrift @@ -175,6 +175,8 @@ struct TSetDataNodeStatusReq { struct TTriggerRegionConsistencyRepairReq { 1: required common.TConsensusGroupId consensusGroupId + 2: optional list partitionFilter + 3: optional string repairEpoch } // Database @@ -715,6 +717,29 @@ struct TShowConfigNodes4InformationSchemaResp { 2: optional list configNodesInfoList } +struct TRepairProgressInfo { + 1: required i32 regionId + 2: required i64 timePartition + 3: required string checkState + 4: required string repairState + 5: required i64 lastCheckedAt + 6: required i64 lastSafeWatermark + 7: required i64 partitionMutationEpoch + 8: required i64 snapshotEpoch + 9: required string snapshotState + 10: required i64 lastMismatchAt + 11: optional string mismatchScopeRef + 12: required i32 mismatchLeafCount + 13: optional string repairEpoch + 14: optional string lastErrorCode + 15: optional string lastErrorMessage +} + +struct TShowRepairProgressResp { + 1: required common.TSStatus status + 2: optional list repairProgressInfoList +} + // Show Database struct TDatabaseInfo { 1: required string name @@ -1788,6 +1813,9 @@ service IConfigNodeRPCService { /** Show cluster ConfigNodes' information for information schema */ TShowConfigNodes4InformationSchemaResp showConfigNodes4InformationSchema() + /** Show replica consistency check / repair progress for information schema */ + TShowRepairProgressResp showRepairProgress() + /** Show cluster Databases' information */ TShowDatabaseResp showDatabase(TGetDatabaseReq req) diff --git a/iotdb-protocol/thrift-datanode/src/main/thrift/datanode.thrift b/iotdb-protocol/thrift-datanode/src/main/thrift/datanode.thrift index ae67e9b9f8163..d469bfb9e56e7 100644 --- a/iotdb-protocol/thrift-datanode/src/main/thrift/datanode.thrift +++ b/iotdb-protocol/thrift-datanode/src/main/thrift/datanode.thrift @@ -428,31 +428,159 @@ struct TLoadResp { 3: optional common.TSStatus status } -struct TConsistencyMerkleFile { - 1: required string sourceTsFilePath - 2: required i64 tsFileSize - 3: required i64 fileXorHash - 4: required i64 fileAddHash +struct TConsistencyDeletionSummary { + 1: required string pathPattern + 2: required i64 timeRangeStart + 3: required i64 timeRangeEnd + 4: required binary serializedProgressIndex } -struct TTimePartitionConsistencyView { +struct TPartitionConsistencyEligibility { 1: required i64 timePartitionId - 2: required list merkleFiles + 2: required i64 partitionMutationEpoch + 3: required i64 snapshotEpoch + 4: required string snapshotState + 5: required i64 liveRootXorHash + 6: required i64 liveRootAddHash + 7: required i64 tombstoneRootXorHash + 8: required i64 tombstoneRootAddHash } -struct TDataRegionConsistencySnapshotReq { +struct TGetConsistencyEligibilityReq { 1: required common.TConsensusGroupId consensusGroupId } -struct TDataRegionConsistencySnapshotResp { +struct TGetConsistencyEligibilityResp { 1: required common.TSStatus status - 2: optional list timePartitionViews + 2: required i64 syncLag + 3: required i64 safeWatermark + 4: optional list partitions +} + +struct TSnapshotSubtreeNode { + 1: required string parentNodeHandle + 2: required string nodeHandle + 3: required string treeKind + 4: required i32 depth + 5: required bool leaf + 6: required i64 xorHash + 7: required i64 addHash + 8: required i64 itemCount + 9: optional string leafId + 10: optional string keyRangeStart + 11: optional string keyRangeEnd +} + +struct TGetSnapshotSubtreeReq { + 1: required common.TConsensusGroupId consensusGroupId + 2: required i64 timePartitionId + 3: required i64 snapshotEpoch + 4: required string treeKind + 5: required list nodeHandles +} + +struct TGetSnapshotSubtreeResp { + 1: required common.TSStatus status + 2: required i64 timePartitionId + 3: required i64 snapshotEpoch + 4: optional bool stale + 5: optional list nodes +} + +struct TLeafDiffEstimate { + 1: required i64 timePartitionId + 2: required i64 snapshotEpoch + 3: required string treeKind + 4: required string leafId + 5: required i64 rowCount + 6: required i64 tombstoneCount + 7: required i64 strataEstimate + 8: optional string keyRangeStart + 9: optional string keyRangeEnd +} + +struct TEstimateLeafDiffReq { + 1: required common.TConsensusGroupId consensusGroupId + 2: required i64 timePartitionId + 3: required i64 snapshotEpoch + 4: required string treeKind + 5: required string leafId } -struct TRepairTransferTsFileReq { +struct TEstimateLeafDiffResp { + 1: required common.TSStatus status + 2: required i64 timePartitionId + 3: required i64 snapshotEpoch + 4: optional bool stale + 5: optional TLeafDiffEstimate leafDiff +} + +struct TLeafDiffEntry { + 1: required string logicalKey + 2: required string diffType +} + +struct TDecodeLeafDiffReq { 1: required common.TConsensusGroupId consensusGroupId - 2: required string sourceTsFilePath - 3: required list targetDataNodes + 2: required i64 timePartitionId + 3: required i64 snapshotEpoch + 4: required string treeKind + 5: required string leafId +} + +struct TDecodeLeafDiffResp { + 1: required common.TSStatus status + 2: required i64 timePartitionId + 3: required i64 snapshotEpoch + 4: optional bool stale + 5: optional list diffEntries +} + +struct TLogicalRepairLeafSelector { + 1: required string treeKind + 2: required string leafId +} + +struct TLogicalRepairBatch { + 1: required string sessionId + 2: required string treeKind + 3: required string leafId + 4: required i32 seqNo + 5: required string batchKind + 6: required binary payload +} + +struct TStreamLogicalRepairReq { + 1: required common.TConsensusGroupId consensusGroupId + 2: required i64 timePartitionId + 3: required string repairEpoch + 4: required list leafSelectors +} + +struct TStreamLogicalRepairResp { + 1: required common.TSStatus status + 2: required i64 timePartitionId + 3: optional bool stale + 4: optional list batches +} + +struct TApplyLogicalRepairBatchReq { + 1: required common.TConsensusGroupId consensusGroupId + 2: required i64 timePartitionId + 3: required string repairEpoch + 4: required string sessionId + 5: required string treeKind + 6: required string leafId + 7: required i32 seqNo + 8: required string batchKind + 9: required binary payload +} + +struct TFinishLogicalRepairSessionReq { + 1: required common.TConsensusGroupId consensusGroupId + 2: required i64 timePartitionId + 3: required string repairEpoch + 4: required string sessionId } struct TConstructSchemaBlackListReq { @@ -842,10 +970,19 @@ service IDataNodeRPCService { TLoadResp sendLoadCommand(TLoadCommandReq req); - TDataRegionConsistencySnapshotResp getDataRegionConsistencySnapshot( - TDataRegionConsistencySnapshotReq req); + TGetConsistencyEligibilityResp getConsistencyEligibility(TGetConsistencyEligibilityReq req); + + TGetSnapshotSubtreeResp getSnapshotSubtree(TGetSnapshotSubtreeReq req); + + TEstimateLeafDiffResp estimateLeafDiff(TEstimateLeafDiffReq req); + + TDecodeLeafDiffResp decodeLeafDiff(TDecodeLeafDiffReq req); + + TStreamLogicalRepairResp streamLogicalRepair(TStreamLogicalRepairReq req); + + common.TSStatus applyLogicalRepairBatch(TApplyLogicalRepairBatchReq req); - common.TSStatus repairTransferTsFile(TRepairTransferTsFileReq req); + common.TSStatus finishLogicalRepairSession(TFinishLogicalRepairSessionReq req); common.TSStatus updateAttribute(TAttributeUpdateReq req); diff --git a/replica_consistency_check_repair_review.md b/replica_consistency_check_repair_review.md deleted file mode 100644 index 45dfb456a6e5a..0000000000000 --- a/replica_consistency_check_repair_review.md +++ /dev/null @@ -1,43 +0,0 @@ -# Review: Replica Consistency Check and Repair Plan - -## 主要发现(按严重级别) - -- **P0:IBF 解码结果与修复输入不闭环,当前定义下无法直接落地** - - 方案中 `Key=xxHash64(deviceId||measurement||timestamp)`,但后续流式修复直接使用 `deviceId/measurement/timestamp` 查询原始点;中间缺少“哈希键 -> 原始点位”的反查索引设计。 - - 建议:明确一种可实现路径(例如在参与 IBF 的窗口内构建临时 `key -> rowRef` 映射,或将可逆编码扩展为携带定位信息)。 - -- **P0:`last_repaired_watermark` 单一水位可能掩盖局部失败,造成永久漏检** - - 有效范围定义为 `(lastRepairedWatermark, T_safe]`,但失败仅记录在 `failed_partitions` 文本字段;如果本轮部分分区失败但全局水位仍推进,失败区间可能被跳过。 - - 建议:改为“分区级/子范围级 checkpoint”(至少 `partition_id + repaired_to`),或仅在范围内全成功时推进全局水位。 - -- **P1:XOR 聚合存在可抵消性,存在误判为一致的理论窗口** - - XOR 作为集合聚合摘要存在信息损失,不同差异组合可能相互抵消,导致误判一致。 - - 建议:至少使用双摘要(如 `xor + sum(mod 2^64)`),或改为真正树形聚合(`parent=H(left||right)`)。 - -- **P1:`Leader wins` 与弱一致/领导权切换语义可能冲突** - - 方案中多处采用“Leader 版本覆盖”,但 leader 转移与复制延迟窗口下,若不统一按 `ProgressIndex/term` 做因果裁决,可能覆盖掉更“新”的删除或写入。 - - 建议:将冲突裁决统一定义为“更高 `ProgressIndex` 获胜”,并落实到所有 repair record 的 apply 逻辑。 - -- **P1:修复应用阶段缺少明确原子提交边界** - - Procedure 有状态机和 checkpoint,但 streaming repair 的“写入可见性边界”与“重试幂等边界”尚不清晰,存在部分应用后重放副作用风险。 - - 建议:引入 repair session staging + 原子 promote,或最少定义严格幂等键、去重窗口与 WAL 持久化顺序。 - -- **P2:性能估算偏乐观,可能触发锁竞争** - - 视图构建若持有 `resourceListLock` 读锁执行重扫描/加载,可能抑制 compaction 写锁并放大尾延迟。 - - 建议:采用“短读锁抓快照 + 锁外重活”,并增加单分区最大处理时长与中断点。 - -## 测试缺口(建议作为准入门槛) - -- 故障注入:`EXECUTE_REPAIR` 中点崩溃、网络抖动、leader 切换,验证无重复/无遗漏/无回退。 -- 语义冲突:同 key 上插入/删除并发与乱序,验证 `ProgressIndex` 裁决一致。 -- 正确性对照:随机数据集做全量扫描真值对比,量化 false-negative(尤其 XOR 聚合方案)。 -- 大分歧退化:分区级缺失/整段缺失,验证快速跳过 IBF 直接全量修复。 -- 幂等回放:同一 repair session 重放多次结果不变。 - -## 结论 - -方案方向正确、工程化程度高,但建议在上线前优先收敛三点: - -1. 打通 IBF 解码到修复输入的可实现闭环; -2. 引入分区级 checkpoint,避免全局水位推进掩盖失败; -3. 将冲突裁决统一为 `ProgressIndex`,替代“Leader wins”口径。 From fcf0c81084efc83ba64de821780338b7433d2414 Mon Sep 17 00:00:00 2001 From: Peng Junzhi <201250214@smail.nju.edu.cn> Date: Fri, 20 Mar 2026 15:05:27 +0800 Subject: [PATCH 4/4] fix: improve robustness --- .../IoTDBIoTConsensusV23C3DBasicITBase.java | 379 ++++++++++++++++-- .../IoTDBIoTConsensusV2Batch3C3DBasicIT.java | 6 + .../IoTDBIoTConsensusV2Stream3C3DBasicIT.java | 6 + .../storageengine/dataregion/DataRegion.java | 2 + .../DataRegionConsistencyManager.java | 115 +++++- .../DataRegionConsistencyRepairService.java | 12 +- ...LogicalConsistencyPartitionStateStore.java | 128 ++++++ .../LogicalRepairSessionJournal.java | 21 +- .../DataRegionConsistencyManagerTest.java | 203 ++++++++-- .../LogicalRepairSessionJournalTest.java | 43 ++ 10 files changed, 836 insertions(+), 79 deletions(-) create mode 100644 iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/consistency/LogicalConsistencyPartitionStateStore.java diff --git a/integration-test/src/test/java/org/apache/iotdb/db/it/iotconsensusv2/IoTDBIoTConsensusV23C3DBasicITBase.java b/integration-test/src/test/java/org/apache/iotdb/db/it/iotconsensusv2/IoTDBIoTConsensusV23C3DBasicITBase.java index a44f1ba020ec1..711cec4ddc3ae 100644 --- a/integration-test/src/test/java/org/apache/iotdb/db/it/iotconsensusv2/IoTDBIoTConsensusV23C3DBasicITBase.java +++ b/integration-test/src/test/java/org/apache/iotdb/db/it/iotconsensusv2/IoTDBIoTConsensusV23C3DBasicITBase.java @@ -25,6 +25,7 @@ import org.apache.iotdb.common.rpc.thrift.TSStatus; import org.apache.iotdb.commons.client.ClientPoolFactory; import org.apache.iotdb.commons.client.IClientManager; +import org.apache.iotdb.commons.client.property.ThriftClientProperty; import org.apache.iotdb.commons.client.sync.SyncConfigNodeIServiceClient; import org.apache.iotdb.commons.client.sync.SyncDataNodeInternalServiceClient; import org.apache.iotdb.confignode.it.regionmigration.IoTDBRegionOperationReliabilityITFramework; @@ -37,9 +38,19 @@ import org.apache.iotdb.it.env.EnvFactory; import org.apache.iotdb.it.env.cluster.node.DataNodeWrapper; import org.apache.iotdb.itbase.env.BaseEnv; +import org.apache.iotdb.mpp.rpc.thrift.TApplyLogicalRepairBatchReq; +import org.apache.iotdb.mpp.rpc.thrift.TDataNodeHeartbeatReq; +import org.apache.iotdb.mpp.rpc.thrift.TFinishLogicalRepairSessionReq; import org.apache.iotdb.mpp.rpc.thrift.TGetConsistencyEligibilityReq; import org.apache.iotdb.mpp.rpc.thrift.TGetConsistencyEligibilityResp; +import org.apache.iotdb.mpp.rpc.thrift.TGetSnapshotSubtreeReq; +import org.apache.iotdb.mpp.rpc.thrift.TGetSnapshotSubtreeResp; +import org.apache.iotdb.mpp.rpc.thrift.TLogicalRepairBatch; +import org.apache.iotdb.mpp.rpc.thrift.TLogicalRepairLeafSelector; import org.apache.iotdb.mpp.rpc.thrift.TPartitionConsistencyEligibility; +import org.apache.iotdb.mpp.rpc.thrift.TSnapshotSubtreeNode; +import org.apache.iotdb.mpp.rpc.thrift.TStreamLogicalRepairReq; +import org.apache.iotdb.mpp.rpc.thrift.TStreamLogicalRepairResp; import org.apache.tsfile.utils.Pair; import org.awaitility.Awaitility; @@ -529,6 +540,107 @@ public void testReplicaConsistencyRepairAfterFollowerLosesSealedTsFile() throws .untilAsserted(() -> assertDataPointCountOnNode(regionReplicaSelection.followerNode, 12L)); } + /** + * The follower-side staged logical repair journal should survive a DataNode restart. This covers + * the data-plane recovery path where batches are durably staged first and only applied during the + * later finish step. The test uses an idempotent logical repair stream derived from the leader's + * current logical snapshot so the assertion stays focused on repair-session durability rather + * than on mismatch injection mechanics. + */ + public void testLogicalRepairSessionSurvivesFollowerRestart() throws Exception { + RegionReplicaSelection regionReplicaSelection; + long firstPartitionId = timePartitionId(100L); + long partitionId = timePartitionId(200L); + + try (Connection connection = makeItCloseQuietly(EnvFactory.getEnv().getConnection()); + Statement statement = makeItCloseQuietly(connection.createStatement())) { + insertAndFlushPartitionData(statement, 100L); + insertAndFlushPartitionData(statement, 200L); + + regionReplicaSelection = selectReplicatedDataRegion(statement); + waitForReplicationComplete( + regionReplicaSelection.leaderNode, regionReplicaSelection.regionId); + waitForCheckState(regionReplicaSelection.regionId, firstPartitionId, "VERIFIED"); + waitForCheckState(regionReplicaSelection.regionId, partitionId, "VERIFIED"); + } + + TPartitionConsistencyEligibility leaderPartition = + getPartitionEligibility( + regionReplicaSelection.leaderNode, regionReplicaSelection.regionId, partitionId); + Assert.assertEquals("READY", leaderPartition.getSnapshotState()); + + List leafSelectors = + getPartitionLeafSelectors( + regionReplicaSelection.leaderNode, + regionReplicaSelection.regionId, + partitionId, + leaderPartition.getSnapshotEpoch(), + "LIVE"); + Assert.assertFalse( + "Idempotent logical repair should still stream at least one live leaf", + leafSelectors.isEmpty()); + leafSelectors = Collections.singletonList(leafSelectors.get(0)); + + String repairEpoch = + buildManualRepairEpoch( + regionReplicaSelection.leaderDataNodeId, partitionId, leaderPartition); + TStreamLogicalRepairResp repairResp = + streamLogicalRepair( + regionReplicaSelection.leaderNode, + regionReplicaSelection.regionId, + partitionId, + repairEpoch, + leafSelectors); + Assert.assertEquals( + "Streaming logical repair should succeed", 200, repairResp.getStatus().getCode()); + Assert.assertFalse("Logical repair stream should not be stale", repairResp.isStale()); + Assert.assertTrue( + "Logical repair stream should contain batches", + repairResp.isSetBatches() && !repairResp.getBatches().isEmpty()); + + String sessionId = + stageLogicalRepairBatches( + regionReplicaSelection.followerNode, + regionReplicaSelection.regionId, + partitionId, + repairEpoch, + repairResp.getBatches()); + Path sessionJournalPath = + logicalRepairSessionPath(regionReplicaSelection.followerNode, sessionId); + Assert.assertTrue( + "Staged logical repair session should be persisted before restart", + Files.exists(sessionJournalPath)); + + regionReplicaSelection.followerNode.stopForcibly(); + Assert.assertFalse( + "Follower should stop between stage and finish", + regionReplicaSelection.followerNode.isAlive()); + regionReplicaSelection.followerNode.start(); + waitForNodeConnectionReady(regionReplicaSelection.followerNode); + waitForReplicationComplete(regionReplicaSelection.leaderNode, regionReplicaSelection.regionId); + Assert.assertTrue( + "Staged logical repair session should survive follower restart", + Files.exists(sessionJournalPath)); + + finishLogicalRepairSession( + regionReplicaSelection.followerNode, + regionReplicaSelection.regionId, + partitionId, + repairEpoch, + sessionId); + Awaitility.await() + .pollDelay(1, TimeUnit.SECONDS) + .pollInterval(1, TimeUnit.SECONDS) + .atMost(1, TimeUnit.MINUTES) + .until(() -> !Files.exists(sessionJournalPath)); + + assertDataPointCountOnNode(regionReplicaSelection.followerNode, 12L); + Assert.assertEquals( + "Logical repair should not regress the already-verified partition state", + "VERIFIED", + getRepairProgressRow(regionReplicaSelection.regionId, partitionId).checkState); + } + protected void insertAndFlushTestData(Statement statement) throws Exception { insertAndFlushPartitionData(statement, 100L); } @@ -828,6 +940,37 @@ protected void waitForNodeConnectionReady(DataNodeWrapper targetNode) { "DataNode " + targetNode.getId() + " is not accepting JDBC connections yet", e); } }); + waitForInternalRpcReady(targetNode); + } + + private void waitForInternalRpcReady(DataNodeWrapper targetNode) { + Awaitility.await() + .pollDelay(1, TimeUnit.SECONDS) + .pollInterval(1, TimeUnit.SECONDS) + .atMost(2, TimeUnit.MINUTES) + .untilAsserted( + () -> { + try (SyncDataNodeInternalServiceClient client = + DATA_NODE_INTERNAL_CLIENT_MANAGER.borrowClient( + new TEndPoint( + targetNode.getInternalAddress(), targetNode.getInternalPort()))) { + TDataNodeHeartbeatReq heartbeatReq = new TDataNodeHeartbeatReq(); + heartbeatReq.setHeartbeatTimestamp(System.nanoTime()); + heartbeatReq.setNeedJudgeLeader(false); + heartbeatReq.setNeedSamplingLoad(false); + heartbeatReq.setTimeSeriesQuotaRemain(0L); + heartbeatReq.setLogicalClock(0L); + Assert.assertNotNull( + "Expected an internal heartbeat response for DataNode " + targetNode.getId(), + client.getDataNodeHeartBeat(heartbeatReq)); + } catch (Exception e) { + throw new AssertionError( + "DataNode " + + targetNode.getId() + + " is not accepting internal RPC connections yet", + e); + } + }); } protected static void assertNullValue(String value) { @@ -955,6 +1098,11 @@ private void assertPartitionViewMatched(RegionReplicaSelection regionReplicaSele regionReplicaSelection.followerNode, regionReplicaSelection.regionId))); } + private Path logicalRepairSessionPath(DataNodeWrapper dataNodeWrapper, String sessionId) { + return Paths.get( + dataNodeWrapper.getSystemDir(), "consistency-repair", "sessions", sessionId + ".session"); + } + private TGetConsistencyEligibilityResp getConsistencyEligibility( DataNodeWrapper dataNodeWrapper, int regionId) throws Exception { try (SyncDataNodeInternalServiceClient client = @@ -973,6 +1121,193 @@ private TGetConsistencyEligibilityResp getConsistencyEligibility( } } + private TStreamLogicalRepairResp streamLogicalRepair( + DataNodeWrapper dataNodeWrapper, + int regionId, + long timePartitionId, + String repairEpoch, + List leafSelectors) + throws Exception { + try (SyncDataNodeInternalServiceClient client = + DATA_NODE_INTERNAL_CLIENT_MANAGER.borrowClient( + new TEndPoint( + dataNodeWrapper.getInternalAddress(), dataNodeWrapper.getInternalPort()))) { + return client.streamLogicalRepair( + new TStreamLogicalRepairReq( + new TConsensusGroupId(TConsensusGroupType.DataRegion, regionId), + timePartitionId, + repairEpoch, + leafSelectors)); + } + } + + private TPartitionConsistencyEligibility getPartitionEligibility( + DataNodeWrapper dataNodeWrapper, int regionId, long timePartitionId) throws Exception { + return getConsistencyEligibility(dataNodeWrapper, regionId).getPartitions().stream() + .filter(partition -> partition.getTimePartitionId() == timePartitionId) + .findFirst() + .orElseThrow( + () -> + new AssertionError( + "Partition " + + timePartitionId + + " is missing from consistency eligibility on DataNode " + + dataNodeWrapper.getId())); + } + + private List getPartitionLeafSelectors( + DataNodeWrapper dataNodeWrapper, + int regionId, + long timePartitionId, + long snapshotEpoch, + String treeKind) + throws Exception { + List shardNodes = + getSnapshotSubtreeNodes( + dataNodeWrapper, + regionId, + timePartitionId, + snapshotEpoch, + treeKind, + Collections.singletonList("root")); + List shardHandles = new ArrayList<>(); + for (TSnapshotSubtreeNode shardNode : shardNodes) { + if (!shardNode.isLeaf()) { + shardHandles.add(shardNode.getNodeHandle()); + } + } + if (shardHandles.isEmpty()) { + return Collections.emptyList(); + } + + List leafNodes = + getSnapshotSubtreeNodes( + dataNodeWrapper, regionId, timePartitionId, snapshotEpoch, treeKind, shardHandles); + leafNodes.sort(Comparator.comparing(TSnapshotSubtreeNode::getLeafId)); + List selectors = new ArrayList<>(); + for (TSnapshotSubtreeNode leafNode : leafNodes) { + if (!leafNode.isLeaf()) { + continue; + } + selectors.add(new TLogicalRepairLeafSelector(treeKind, leafNode.getLeafId())); + } + return selectors; + } + + private List getSnapshotSubtreeNodes( + DataNodeWrapper dataNodeWrapper, + int regionId, + long timePartitionId, + long snapshotEpoch, + String treeKind, + List nodeHandles) + throws Exception { + try (SyncDataNodeInternalServiceClient client = + DATA_NODE_INTERNAL_CLIENT_MANAGER.borrowClient( + new TEndPoint( + dataNodeWrapper.getInternalAddress(), dataNodeWrapper.getInternalPort()))) { + TGetSnapshotSubtreeResp response = + client.getSnapshotSubtree( + new TGetSnapshotSubtreeReq( + new TConsensusGroupId(TConsensusGroupType.DataRegion, regionId), + timePartitionId, + snapshotEpoch, + treeKind, + nodeHandles)); + Assert.assertEquals( + "Snapshot subtree RPC should succeed on DataNode " + dataNodeWrapper.getId(), + 200, + response.getStatus().getCode()); + Assert.assertFalse( + "Snapshot subtree should stay valid for partition " + timePartitionId, + response.isStale()); + return response.isSetNodes() ? response.getNodes() : Collections.emptyList(); + } + } + + private String buildManualRepairEpoch( + int leaderDataNodeId, + long timePartitionId, + TPartitionConsistencyEligibility partitionEligibility) { + return leaderDataNodeId + + ":" + + timePartitionId + + ":0:" + + partitionEligibility.getSnapshotEpoch() + + ":" + + partitionEligibility.getPartitionMutationEpoch() + + ":manual-it"; + } + + private String stageLogicalRepairBatches( + DataNodeWrapper followerNode, + int regionId, + long timePartitionId, + String repairEpoch, + List batches) + throws Exception { + String sessionId = null; + for (TLogicalRepairBatch batch : batches) { + if (sessionId == null) { + sessionId = batch.getSessionId(); + } else { + Assert.assertEquals( + "All staged batches should belong to one repair session", + sessionId, + batch.getSessionId()); + } + try (SyncDataNodeInternalServiceClient client = + DATA_NODE_INTERNAL_CLIENT_MANAGER.borrowClient( + new TEndPoint(followerNode.getInternalAddress(), followerNode.getInternalPort()))) { + TSStatus status = + client.applyLogicalRepairBatch( + new TApplyLogicalRepairBatchReq( + new TConsensusGroupId(TConsensusGroupType.DataRegion, regionId), + timePartitionId, + repairEpoch, + batch.getSessionId(), + batch.getTreeKind(), + batch.getLeafId(), + batch.getSeqNo(), + batch.getBatchKind(), + batch.bufferForPayload())); + Assert.assertEquals("Staging logical repair batch should succeed", 200, status.getCode()); + } + } + Assert.assertNotNull("Repair stream should have a session id", sessionId); + return sessionId; + } + + private void finishLogicalRepairSession( + DataNodeWrapper followerNode, + int regionId, + long timePartitionId, + String repairEpoch, + String sessionId) + throws Exception { + TEndPoint endPoint = + new TEndPoint(followerNode.getInternalAddress(), followerNode.getInternalPort()); + SyncDataNodeInternalServiceClient client = + new SyncDataNodeInternalServiceClient( + new ThriftClientProperty.Builder() + .setConnectionTimeoutMs((int) TimeUnit.MINUTES.toMillis(5)) + .build(), + endPoint, + null); + try { + TSStatus status = + client.finishLogicalRepairSession( + new TFinishLogicalRepairSessionReq( + new TConsensusGroupId(TConsensusGroupType.DataRegion, regionId), + timePartitionId, + repairEpoch, + sessionId)); + Assert.assertEquals("Finishing logical repair session should succeed", 200, status.getCode()); + } finally { + client.getInputProtocol().getTransport().close(); + } + } + private List partitionSnapshotSignature( TGetConsistencyEligibilityResp eligibilityResponse) { if (eligibilityResponse == null || !eligibilityResponse.isSetPartitions()) { @@ -1061,41 +1396,7 @@ private RepairProgressRow getRepairProgressRow( } private List getRepairProgressRows(int regionId) throws Exception { - try (Connection connection = - makeItCloseQuietly(EnvFactory.getEnv().getConnection(BaseEnv.TABLE_SQL_DIALECT)); - Statement statement = makeItCloseQuietly(connection.createStatement()); - ResultSet resultSet = - statement.executeQuery( - "select region_id, time_partition, check_state, repair_state, last_checked_at, " - + "last_safe_watermark, partition_mutation_epoch, snapshot_epoch, snapshot_state, " - + "last_mismatch_at, mismatch_scope_ref, mismatch_leaf_count, repair_epoch, " - + "last_error_code, last_error_message from information_schema.repair_progress " - + "where region_id = " - + regionId - + " order by time_partition")) { - List rows = new ArrayList<>(); - while (resultSet.next()) { - rows.add( - new RepairProgressRow( - resultSet.getInt(1), - resultSet.getLong(2), - resultSet.getString(3), - resultSet.getString(4), - resultSet.getLong(5), - resultSet.getLong(6), - resultSet.getLong(7), - resultSet.getLong(8), - resultSet.getString(9), - resultSet.getLong(10), - resultSet.getString(11), - resultSet.getInt(12), - resultSet.getString(13), - resultSet.getString(14), - resultSet.getString(15))); - } - rows.sort(Comparator.comparingLong(row -> row.timePartition)); - return rows; - } + return getRepairProgressRows(selectReadableDataNode(), regionId); } private List getRepairProgressRows(DataNodeWrapper targetNode, int regionId) @@ -1138,10 +1439,18 @@ private List getRepairProgressRows(DataNodeWrapper targetNode resultSet.getString(14), resultSet.getString(15))); } + rows.sort(Comparator.comparingLong(row -> row.timePartition)); return rows; } } + private DataNodeWrapper selectReadableDataNode() { + return EnvFactory.getEnv().getDataNodeWrapperList().stream() + .filter(DataNodeWrapper::isAlive) + .findFirst() + .orElseThrow(() -> new IllegalStateException("No alive DataNode is available for query")); + } + private long timePartitionId(long timestamp) { return Math.floorDiv(timestamp, TIME_PARTITION_INTERVAL); } diff --git a/integration-test/src/test/java/org/apache/iotdb/db/it/iotconsensusv2/batch/IoTDBIoTConsensusV2Batch3C3DBasicIT.java b/integration-test/src/test/java/org/apache/iotdb/db/it/iotconsensusv2/batch/IoTDBIoTConsensusV2Batch3C3DBasicIT.java index 837789d3cdf13..6ca3391e35400 100644 --- a/integration-test/src/test/java/org/apache/iotdb/db/it/iotconsensusv2/batch/IoTDBIoTConsensusV2Batch3C3DBasicIT.java +++ b/integration-test/src/test/java/org/apache/iotdb/db/it/iotconsensusv2/batch/IoTDBIoTConsensusV2Batch3C3DBasicIT.java @@ -62,6 +62,12 @@ public void testReplicaConsistencyRepairAfterFollowerLosesSealedTsFile() throws super.testReplicaConsistencyRepairAfterFollowerLosesSealedTsFile(); } + @Override + @Test + public void testLogicalRepairSessionSurvivesFollowerRestart() throws Exception { + super.testLogicalRepairSessionSurvivesFollowerRestart(); + } + @Override @Test public void testBackgroundConsistencyCheckOnlyRunsOnColdPartitions() throws Exception { diff --git a/integration-test/src/test/java/org/apache/iotdb/db/it/iotconsensusv2/stream/IoTDBIoTConsensusV2Stream3C3DBasicIT.java b/integration-test/src/test/java/org/apache/iotdb/db/it/iotconsensusv2/stream/IoTDBIoTConsensusV2Stream3C3DBasicIT.java index c188b99e6853a..0d06c71596c77 100644 --- a/integration-test/src/test/java/org/apache/iotdb/db/it/iotconsensusv2/stream/IoTDBIoTConsensusV2Stream3C3DBasicIT.java +++ b/integration-test/src/test/java/org/apache/iotdb/db/it/iotconsensusv2/stream/IoTDBIoTConsensusV2Stream3C3DBasicIT.java @@ -62,6 +62,12 @@ public void testReplicaConsistencyRepairAfterFollowerLosesSealedTsFile() throws super.testReplicaConsistencyRepairAfterFollowerLosesSealedTsFile(); } + @Override + @Test + public void testLogicalRepairSessionSurvivesFollowerRestart() throws Exception { + super.testLogicalRepairSessionSurvivesFollowerRestart(); + } + @Override @Test public void testBackgroundConsistencyCheckOnlyRunsOnColdPartitions() throws Exception { diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/DataRegion.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/DataRegion.java index 3aac7d0bee388..88189609f30db 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/DataRegion.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/DataRegion.java @@ -4000,6 +4000,8 @@ public void loadNewTsFile( } onTsFileLoaded(newTsFileResource, isFromConsensus, lastReader); + DataRegionConsistencyManager.getInstance() + .onPartitionMutation(dataRegionId.convertToTConsensusGroupId(), newFilePartitionId); logger.info("TsFile {} is successfully loaded in unsequence list.", newFileName); } catch (final DiskSpaceInsufficientException e) { logger.error( diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/consistency/DataRegionConsistencyManager.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/consistency/DataRegionConsistencyManager.java index 13e90ecfb885c..74a6ff03eb824 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/consistency/DataRegionConsistencyManager.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/consistency/DataRegionConsistencyManager.java @@ -98,6 +98,15 @@ public class DataRegionConsistencyManager { private final ConcurrentHashMap regionStates = new ConcurrentHashMap<>(); private final ThreadLocal repairMutationContext = new ThreadLocal<>(); + private final LogicalConsistencyPartitionStateStore partitionStateStore; + + DataRegionConsistencyManager() { + this(new LogicalConsistencyPartitionStateStore()); + } + + DataRegionConsistencyManager(LogicalConsistencyPartitionStateStore partitionStateStore) { + this.partitionStateStore = partitionStateStore; + } public static DataRegionConsistencyManager getInstance() { return INSTANCE; @@ -108,9 +117,11 @@ public PartitionInspection inspectPartition( DataRegion dataRegion, long partitionId, List partitionDeletionSummaries) { + String consensusGroupKey = consensusGroupId.toString(); RegionState regionState = getOrCreateRegionState(consensusGroupId); PartitionState partitionState = regionState.partitions.computeIfAbsent(partitionId, PartitionState::new); + PartitionInspection inspection; synchronized (partitionState) { if (partitionState.snapshotState != RepairProgressTable.SnapshotState.READY || partitionState.snapshotEpoch != partitionState.partitionMutationEpoch) { @@ -126,8 +137,17 @@ public PartitionInspection inspectPartition( e); } } - return partitionState.toInspection(partitionId); + inspection = partitionState.toInspection(partitionId); } + persistKnownPartitionsIfNeeded(consensusGroupKey, regionState); + return inspection; + } + + public List getKnownPartitions(TConsensusGroupId consensusGroupId) { + List partitions = + new ArrayList<>(getOrCreateRegionState(consensusGroupId).partitions.keySet()); + partitions.sort(Long::compareTo); + return partitions; } public SnapshotSubtreeResult getSnapshotSubtree( @@ -463,10 +483,11 @@ public void onDeletion(TConsensusGroupId consensusGroupId, long startTime, long } public void onPartitionMutation(TConsensusGroupId consensusGroupId, long partitionId) { - PartitionState state = - getOrCreateRegionState(consensusGroupId) - .partitions - .computeIfAbsent(partitionId, PartitionState::new); + String consensusGroupKey = consensusGroupId.toString(); + RegionState regionState = getOrCreateRegionState(consensusGroupId); + PartitionState newState = new PartitionState(partitionId); + PartitionState existingState = regionState.partitions.putIfAbsent(partitionId, newState); + PartitionState state = existingState == null ? newState : existingState; synchronized (state) { if (isRepairMutation(consensusGroupId, partitionId)) { state.snapshotState = RepairProgressTable.SnapshotState.DIRTY; @@ -481,10 +502,88 @@ public void onPartitionMutation(TConsensusGroupId consensusGroupId, long partiti state.tombstoneTree = SnapshotTree.empty(); state.lastError = null; } + if (existingState == null) { + persistKnownPartitionsIfNeeded(consensusGroupKey, regionState); + } } private RegionState getOrCreateRegionState(TConsensusGroupId consensusGroupId) { - return regionStates.computeIfAbsent(consensusGroupId.toString(), ignored -> new RegionState()); + return regionStates.computeIfAbsent( + consensusGroupId.toString(), ignored -> loadRegionState(consensusGroupId.toString())); + } + + private RegionState loadRegionState(String consensusGroupKey) { + RegionState regionState = new RegionState(); + try { + Map persistedMutationEpochs = partitionStateStore.load(consensusGroupKey); + for (Map.Entry entry : persistedMutationEpochs.entrySet()) { + PartitionState partitionState = new PartitionState(entry.getKey()); + partitionState.partitionMutationEpoch = entry.getValue(); + regionState.partitions.put(entry.getKey(), partitionState); + } + regionState.lastPersistedMutationEpochs = new TreeMap<>(persistedMutationEpochs); + } catch (IOException e) { + LOGGER.warn( + "Failed to restore logical consistency partition state for region {}", + consensusGroupKey, + e); + } + return regionState; + } + + private void persistKnownPartitionsIfNeeded(String consensusGroupKey, RegionState regionState) { + Map snapshotToPersist = snapshotPartitionMutationEpochs(regionState); + synchronized (regionState.persistMonitor) { + if (snapshotToPersist.equals(regionState.lastPersistedMutationEpochs) + || snapshotToPersist.equals(regionState.pendingPersistMutationEpochs)) { + return; + } + if (regionState.persistInFlight) { + regionState.pendingPersistMutationEpochs = snapshotToPersist; + return; + } + regionState.persistInFlight = true; + } + + while (true) { + boolean persistSucceeded = false; + try { + partitionStateStore.persist(consensusGroupKey, snapshotToPersist); + persistSucceeded = true; + } catch (IOException e) { + LOGGER.warn( + "Failed to persist logical consistency partition state for region {}", + consensusGroupKey, + e); + } + + synchronized (regionState.persistMonitor) { + if (persistSucceeded) { + regionState.lastPersistedMutationEpochs = snapshotToPersist; + } + if (regionState.pendingPersistMutationEpochs == null + || regionState.pendingPersistMutationEpochs.equals( + regionState.lastPersistedMutationEpochs) + || (!persistSucceeded + && regionState.pendingPersistMutationEpochs.equals(snapshotToPersist))) { + regionState.pendingPersistMutationEpochs = null; + regionState.persistInFlight = false; + return; + } + snapshotToPersist = regionState.pendingPersistMutationEpochs; + regionState.pendingPersistMutationEpochs = null; + } + } + } + + private Map snapshotPartitionMutationEpochs(RegionState regionState) { + Map mutationEpochs = new TreeMap<>(); + for (Map.Entry entry : regionState.partitions.entrySet()) { + synchronized (entry.getValue()) { + mutationEpochs.put(entry.getKey(), entry.getValue().partitionMutationEpoch); + } + } + return mutationEpochs; } private boolean ensureReadySnapshot( @@ -1399,6 +1498,10 @@ void accept( private static class RegionState { private final ConcurrentHashMap partitions = new ConcurrentHashMap<>(); + private final Object persistMonitor = new Object(); + private Map lastPersistedMutationEpochs = Collections.emptyMap(); + private Map pendingPersistMutationEpochs = null; + private boolean persistInFlight = false; } private static class RepairMutationContext { diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/consistency/DataRegionConsistencyRepairService.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/consistency/DataRegionConsistencyRepairService.java index 03754c787eda9..47c9b33ff6302 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/consistency/DataRegionConsistencyRepairService.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/consistency/DataRegionConsistencyRepairService.java @@ -67,7 +67,9 @@ import java.util.ArrayList; import java.util.Collections; import java.util.Comparator; +import java.util.LinkedHashSet; import java.util.List; +import java.util.Set; import java.util.stream.Collectors; /** DataNode-side logical snapshot and logical repair primitives for replica consistency repair. */ @@ -105,12 +107,14 @@ public TGetConsistencyEligibilityResp getConsistencyEligibility( : dataRegion.getDelayAnalyzer().getSafeWatermark(System.currentTimeMillis()); List regionDeletionSummaries = collectDeletionSummaries(req.getConsensusGroupId()); - List timePartitions = new ArrayList<>(dataRegion.getTimePartitions()); - augmentTimePartitionsWithDeletionRanges(timePartitions, regionDeletionSummaries); - timePartitions.sort(Long::compareTo); + Set timePartitions = new LinkedHashSet<>(dataRegion.getTimePartitions()); + timePartitions.addAll(consistencyManager.getKnownPartitions(req.getConsensusGroupId())); + List orderedTimePartitions = new ArrayList<>(timePartitions); + augmentTimePartitionsWithDeletionRanges(orderedTimePartitions, regionDeletionSummaries); + orderedTimePartitions.sort(Long::compareTo); List partitions = new ArrayList<>(); - for (Long timePartition : timePartitions) { + for (Long timePartition : orderedTimePartitions) { // Eligibility must expose follower partitions even when the follower-local DelayAnalyzer // has // not warmed up yet. ConfigNode applies cold-partition pruning from the leader view; if we diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/consistency/LogicalConsistencyPartitionStateStore.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/consistency/LogicalConsistencyPartitionStateStore.java new file mode 100644 index 0000000000000..f51fbb5ea1bbe --- /dev/null +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/consistency/LogicalConsistencyPartitionStateStore.java @@ -0,0 +1,128 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.iotdb.db.storageengine.dataregion.consistency; + +import org.apache.iotdb.db.conf.IoTDBDescriptor; + +import org.apache.tsfile.utils.ReadWriteIOUtils; + +import java.io.FileOutputStream; +import java.io.IOException; +import java.io.InputStream; +import java.io.OutputStream; +import java.nio.charset.StandardCharsets; +import java.nio.file.AtomicMoveNotSupportedException; +import java.nio.file.Files; +import java.nio.file.Path; +import java.nio.file.Paths; +import java.nio.file.StandardCopyOption; +import java.nio.file.StandardOpenOption; +import java.util.Base64; +import java.util.Collections; +import java.util.Map; +import java.util.TreeMap; + +/** + * Persists the partition mutation epochs that the logical consistency layer has already observed. + */ +class LogicalConsistencyPartitionStateStore { + + private static final int FORMAT_VERSION = 1; + + private final Path stateDirOverride; + + LogicalConsistencyPartitionStateStore() { + this(null); + } + + LogicalConsistencyPartitionStateStore(Path stateDir) { + this.stateDirOverride = stateDir; + } + + public Map load(String consensusGroupKey) throws IOException { + Path path = statePath(consensusGroupKey); + if (!Files.exists(path)) { + return Collections.emptyMap(); + } + + try (InputStream inputStream = Files.newInputStream(path, StandardOpenOption.READ)) { + int version = ReadWriteIOUtils.readInt(inputStream); + if (version != FORMAT_VERSION) { + throw new IOException("Unsupported logical consistency partition-state format " + version); + } + int partitionCount = ReadWriteIOUtils.readInt(inputStream); + Map mutationEpochs = new TreeMap<>(); + for (int i = 0; i < partitionCount; i++) { + mutationEpochs.put( + ReadWriteIOUtils.readLong(inputStream), ReadWriteIOUtils.readLong(inputStream)); + } + return mutationEpochs; + } + } + + public void persist(String consensusGroupKey, Map mutationEpochs) throws IOException { + Path stateDir = getStateDir(); + Files.createDirectories(stateDir); + Path path = statePath(consensusGroupKey); + Path tmpPath = path.resolveSibling(path.getFileName() + ".tmp"); + + try (FileOutputStream outputStream = new FileOutputStream(tmpPath.toFile())) { + serialize(mutationEpochs, outputStream); + outputStream.getFD().sync(); + } + + try { + Files.move( + tmpPath, path, StandardCopyOption.ATOMIC_MOVE, StandardCopyOption.REPLACE_EXISTING); + } catch (AtomicMoveNotSupportedException e) { + Files.move(tmpPath, path, StandardCopyOption.REPLACE_EXISTING); + } + } + + private Path statePath(String consensusGroupKey) { + return getStateDir().resolve(encodeFileName(consensusGroupKey) + ".state"); + } + + private Path getStateDir() { + if (stateDirOverride != null) { + return stateDirOverride; + } + return Paths.get( + IoTDBDescriptor.getInstance().getConfig().getSystemDir(), + "consistency-check", + "partition-state"); + } + + private String encodeFileName(String consensusGroupKey) { + return Base64.getUrlEncoder() + .withoutPadding() + .encodeToString(consensusGroupKey.getBytes(StandardCharsets.UTF_8)); + } + + private void serialize(Map mutationEpochs, OutputStream outputStream) + throws IOException { + ReadWriteIOUtils.write(FORMAT_VERSION, outputStream); + ReadWriteIOUtils.write(mutationEpochs.size(), outputStream); + for (Map.Entry entry : mutationEpochs.entrySet()) { + ReadWriteIOUtils.write(entry.getKey(), outputStream); + ReadWriteIOUtils.write(entry.getValue(), outputStream); + } + } +} diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/consistency/LogicalRepairSessionJournal.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/consistency/LogicalRepairSessionJournal.java index ea281fb5955f7..8fb78552f05b3 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/consistency/LogicalRepairSessionJournal.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/consistency/LogicalRepairSessionJournal.java @@ -48,19 +48,15 @@ class LogicalRepairSessionJournal { private static final int FORMAT_VERSION = 1; - private final Path journalDir; + private final Path journalDirOverride; private final ConcurrentHashMap sessions = new ConcurrentHashMap<>(); LogicalRepairSessionJournal() { - this( - Paths.get( - IoTDBDescriptor.getInstance().getConfig().getSystemDir(), - "consistency-repair", - "sessions")); + this(null); } LogicalRepairSessionJournal(Path journalDir) { - this.journalDir = journalDir; + this.journalDirOverride = journalDir; } public synchronized void stageBatch( @@ -151,6 +147,7 @@ private void validateSession( } private void persist(SessionState sessionState) throws IOException { + Path journalDir = getJournalDir(); Files.createDirectories(journalDir); Path sessionPath = sessionPath(sessionState.sessionId); Path tmpPath = sessionPath.resolveSibling(sessionPath.getFileName() + ".tmp"); @@ -172,7 +169,15 @@ private void persist(SessionState sessionState) throws IOException { } private Path sessionPath(String sessionId) { - return journalDir.resolve(sessionId + ".session"); + return getJournalDir().resolve(sessionId + ".session"); + } + + private Path getJournalDir() { + if (journalDirOverride != null) { + return journalDirOverride; + } + return Paths.get( + IoTDBDescriptor.getInstance().getConfig().getSystemDir(), "consistency-repair", "sessions"); } private static byte[] duplicatePayload(ByteBuffer payload) { diff --git a/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/storageengine/dataregion/consistency/DataRegionConsistencyManagerTest.java b/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/storageengine/dataregion/consistency/DataRegionConsistencyManagerTest.java index 79add27513f22..e1bc5c58efb69 100644 --- a/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/storageengine/dataregion/consistency/DataRegionConsistencyManagerTest.java +++ b/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/storageengine/dataregion/consistency/DataRegionConsistencyManagerTest.java @@ -42,18 +42,21 @@ import org.apache.tsfile.read.common.TimeRange; import org.apache.tsfile.write.schema.IMeasurementSchema; import org.apache.tsfile.write.schema.MeasurementSchema; +import org.junit.After; import org.junit.Assert; import org.junit.Test; import org.mockito.Mockito; import java.io.File; import java.io.IOException; +import java.io.UncheckedIOException; import java.lang.reflect.Field; import java.lang.reflect.Method; import java.nio.channels.ClosedChannelException; import java.nio.charset.StandardCharsets; import java.nio.file.Files; import java.nio.file.Path; +import java.util.ArrayList; import java.util.Base64; import java.util.Collections; import java.util.Comparator; @@ -62,12 +65,25 @@ import java.util.Map; import java.util.concurrent.ConcurrentHashMap; import java.util.concurrent.ConcurrentLinkedDeque; +import java.util.concurrent.CountDownLatch; +import java.util.concurrent.TimeUnit; +import java.util.concurrent.atomic.AtomicReference; public class DataRegionConsistencyManagerTest { + private final List tempStateDirs = new ArrayList<>(); + + @After + public void tearDown() throws IOException { + for (Path tempStateDir : tempStateDirs) { + deleteRecursively(tempStateDir); + } + tempStateDirs.clear(); + } + @Test public void compactionShouldNotAdvancePartitionMutationEpoch() throws Exception { - DataRegionConsistencyManager consistencyManager = DataRegionConsistencyManager.getInstance(); + DataRegionConsistencyManager consistencyManager = newTestConsistencyManager(); TConsensusGroupId consensusGroupId = new TConsensusGroupId(TConsensusGroupType.DataRegion, 901); consistencyManager.onPartitionMutation(consensusGroupId, 7L); @@ -89,7 +105,7 @@ public void compactionShouldNotAdvancePartitionMutationEpoch() throws Exception @Test public void deletionShouldMarkAffectedPartitionDirty() throws Exception { - DataRegionConsistencyManager consistencyManager = DataRegionConsistencyManager.getInstance(); + DataRegionConsistencyManager consistencyManager = newTestConsistencyManager(); TConsensusGroupId consensusGroupId = new TConsensusGroupId(TConsensusGroupType.DataRegion, 902); consistencyManager.onDeletion(consensusGroupId, 0L, 0L); @@ -102,7 +118,7 @@ public void deletionShouldMarkAffectedPartitionDirty() throws Exception { @Test public void logicalRepairMutationShouldNotAdvancePartitionMutationEpoch() throws Exception { - DataRegionConsistencyManager consistencyManager = DataRegionConsistencyManager.getInstance(); + DataRegionConsistencyManager consistencyManager = newTestConsistencyManager(); TConsensusGroupId consensusGroupId = new TConsensusGroupId(TConsensusGroupType.DataRegion, 903); consistencyManager.onPartitionMutation(consensusGroupId, 11L); @@ -233,7 +249,7 @@ public void snapshotTreeShouldTrackLogicalKeyBounds() throws Exception { @Test public void memTableSeriesDiscoveryShouldIncludeUnsealedMeasurements() throws Exception { - DataRegionConsistencyManager consistencyManager = DataRegionConsistencyManager.getInstance(); + DataRegionConsistencyManager consistencyManager = newTestConsistencyManager(); IMeasurementSchema alignedS1 = new MeasurementSchema("s1", TSDataType.INT64); IMeasurementSchema alignedS2 = new MeasurementSchema("s2", TSDataType.DOUBLE); @@ -266,7 +282,7 @@ public void memTableSeriesDiscoveryShouldIncludeUnsealedMeasurements() throws Ex @Test public void closedChannelFailuresShouldBeClassifiedAsRetryable() throws Exception { - DataRegionConsistencyManager consistencyManager = DataRegionConsistencyManager.getInstance(); + DataRegionConsistencyManager consistencyManager = newTestConsistencyManager(); Method method = DataRegionConsistencyManager.class.getDeclaredMethod( "isRetryableSnapshotReadFailure", Throwable.class); @@ -284,27 +300,9 @@ public void closedChannelFailuresShouldBeClassifiedAsRetryable() throws Exceptio @Test public void inspectPartitionShouldRebuildSnapshotEvenWhenWorkingProcessorsExist() throws Exception { - DataRegionConsistencyManager consistencyManager = DataRegionConsistencyManager.getInstance(); + DataRegionConsistencyManager consistencyManager = newTestConsistencyManager(); TConsensusGroupId consensusGroupId = new TConsensusGroupId(TConsensusGroupType.DataRegion, 904); - - DataRegion dataRegion = Mockito.mock(DataRegion.class); - TsFileManager tsFileManager = Mockito.mock(TsFileManager.class); - Mockito.when(dataRegion.getTsFileManager()).thenReturn(tsFileManager); - Mockito.when(tsFileManager.getTsFileListSnapshot(1L, true)).thenReturn(Collections.emptyList()); - Mockito.when(tsFileManager.getTsFileListSnapshot(1L, false)) - .thenReturn(Collections.emptyList()); - TsFileProcessor processor = Mockito.mock(TsFileProcessor.class); - IMemTable emptyMemTable = Mockito.mock(IMemTable.class); - Mockito.when(processor.getTimeRangeId()).thenReturn(1L); - Mockito.when(processor.tryReadLock(1_000L)).thenReturn(true); - Mockito.when(processor.getWorkMemTable()).thenReturn(emptyMemTable); - Mockito.when(processor.getFlushingMemTable()).thenReturn(new ConcurrentLinkedDeque<>()); - Mockito.doNothing().when(processor).readUnLock(); - Mockito.when(emptyMemTable.getMemTableMap()).thenReturn(Collections.emptyMap()); - Mockito.when(dataRegion.getWorkSequenceTsFileProcessors()) - .thenReturn(Collections.singletonList(processor)); - Mockito.when(dataRegion.getWorkUnsequenceTsFileProcessors()) - .thenReturn(Collections.emptyList()); + DataRegion dataRegion = createDataRegionWithEmptySnapshotInputs(1L, true); DataRegionConsistencyManager.PartitionInspection inspection = consistencyManager.inspectPartition( @@ -316,9 +314,72 @@ public void inspectPartitionShouldRebuildSnapshotEvenWhenWorkingProcessorsExist( Assert.assertEquals(0L, inspection.getSnapshotEpoch()); } + @Test + public void knownPartitionsShouldRecoverFromPersistedPartitionState() throws Exception { + Path stateDir = Files.createTempDirectory("logical-consistency-partition-state"); + try { + LogicalConsistencyPartitionStateStore store = + new LogicalConsistencyPartitionStateStore(stateDir); + DataRegionConsistencyManager writer = new DataRegionConsistencyManager(store); + TConsensusGroupId consensusGroupId = + new TConsensusGroupId(TConsensusGroupType.DataRegion, 905); + + writer.onPartitionMutation(consensusGroupId, 2L); + + DataRegionConsistencyManager recovered = new DataRegionConsistencyManager(store); + Assert.assertEquals( + Collections.singletonList(2L), recovered.getKnownPartitions(consensusGroupId)); + Assert.assertEquals(1L, getPartitionMutationEpoch(recovered, consensusGroupId, 2L)); + } finally { + deleteRecursively(stateDir); + } + } + + @Test + public void persistKnownPartitionsShouldNotBlockConcurrentMutation() throws Exception { + BlockingPartitionStateStore store = new BlockingPartitionStateStore(); + DataRegionConsistencyManager consistencyManager = new DataRegionConsistencyManager(store); + TConsensusGroupId consensusGroupId = new TConsensusGroupId(TConsensusGroupType.DataRegion, 906); + DataRegion dataRegion = createDataRegionWithEmptySnapshotInputs(1L, true); + AtomicReference inspectionFailure = new AtomicReference<>(); + + Thread inspectionThread = + new Thread( + () -> { + try { + consistencyManager.inspectPartition( + consensusGroupId, dataRegion, 1L, Collections.emptyList()); + } catch (Throwable t) { + inspectionFailure.set(t); + } + }); + inspectionThread.start(); + + Assert.assertTrue( + "The initial persistence should start", store.awaitPersistStarted(5, TimeUnit.SECONDS)); + + long mutationStartNanos = System.nanoTime(); + consistencyManager.onPartitionMutation(consensusGroupId, 2L); + long mutationElapsedMillis = + TimeUnit.NANOSECONDS.toMillis(System.nanoTime() - mutationStartNanos); + Assert.assertTrue( + "Concurrent partition mutation should not wait on slow persistence", + mutationElapsedMillis < 1_000L); + + store.releasePersist(); + inspectionThread.join(TimeUnit.SECONDS.toMillis(5)); + if (inspectionFailure.get() != null) { + throw new AssertionError("Inspection thread should succeed", inspectionFailure.get()); + } + + Map persistedState = store.load(consensusGroupId.toString()); + Assert.assertEquals(Long.valueOf(0L), persistedState.get(1L)); + Assert.assertEquals(Long.valueOf(1L), persistedState.get(2L)); + } + @Test public void collectLogicalSeriesContextsShouldNotPoisonSharedReaders() throws Exception { - DataRegionConsistencyManager consistencyManager = DataRegionConsistencyManager.getInstance(); + DataRegionConsistencyManager consistencyManager = newTestConsistencyManager(); Path tempDir = Files.createTempDirectory("consistency-manager-reader-regression"); TsFileResource resource = null; MultiTsFileDeviceIterator verificationIterator = null; @@ -378,6 +439,64 @@ private String encodeBase64(String value) { .encodeToString(value.getBytes(StandardCharsets.UTF_8)); } + private DataRegion createDataRegionWithEmptySnapshotInputs( + long partitionId, boolean includeWorkingProcessor) throws Exception { + DataRegion dataRegion = Mockito.mock(DataRegion.class); + TsFileManager tsFileManager = Mockito.mock(TsFileManager.class); + Mockito.when(dataRegion.getTsFileManager()).thenReturn(tsFileManager); + Mockito.when(tsFileManager.getTsFileListSnapshot(partitionId, true)) + .thenReturn(Collections.emptyList()); + Mockito.when(tsFileManager.getTsFileListSnapshot(partitionId, false)) + .thenReturn(Collections.emptyList()); + Mockito.when(dataRegion.getWorkUnsequenceTsFileProcessors()) + .thenReturn(Collections.emptyList()); + + if (!includeWorkingProcessor) { + Mockito.when(dataRegion.getWorkSequenceTsFileProcessors()) + .thenReturn(Collections.emptyList()); + return dataRegion; + } + + TsFileProcessor processor = Mockito.mock(TsFileProcessor.class); + IMemTable emptyMemTable = Mockito.mock(IMemTable.class); + Mockito.when(processor.getTimeRangeId()).thenReturn(partitionId); + Mockito.when(processor.tryReadLock(1_000L)).thenReturn(true); + Mockito.when(processor.getWorkMemTable()).thenReturn(emptyMemTable); + Mockito.when(processor.getFlushingMemTable()).thenReturn(new ConcurrentLinkedDeque<>()); + Mockito.doNothing().when(processor).readUnLock(); + Mockito.when(emptyMemTable.getMemTableMap()).thenReturn(Collections.emptyMap()); + Mockito.when(dataRegion.getWorkSequenceTsFileProcessors()) + .thenReturn(Collections.singletonList(processor)); + return dataRegion; + } + + private DataRegionConsistencyManager newTestConsistencyManager() throws IOException { + Path tempStateDir = Files.createTempDirectory("logical-consistency-test-state"); + tempStateDirs.add(tempStateDir); + return new DataRegionConsistencyManager( + new LogicalConsistencyPartitionStateStore(tempStateDir)); + } + + private void deleteRecursively(Path root) throws IOException { + if (root == null || !Files.exists(root)) { + return; + } + try { + Files.walk(root) + .sorted(Comparator.reverseOrder()) + .forEach( + path -> { + try { + Files.deleteIfExists(path); + } catch (IOException e) { + throw new UncheckedIOException(e); + } + }); + } catch (UncheckedIOException e) { + throw e.getCause(); + } + } + @SuppressWarnings("unchecked") private void assertDeviceSeriesContext( Object deviceSeriesContext, boolean expectedAligned, String... expectedMeasurements) @@ -441,4 +560,36 @@ private RepairProgressTable.SnapshotState getSnapshotState( snapshotStateField.setAccessible(true); return (RepairProgressTable.SnapshotState) snapshotStateField.get(partitionState); } + + private static class BlockingPartitionStateStore extends LogicalConsistencyPartitionStateStore { + private final CountDownLatch persistStarted = new CountDownLatch(1); + private final CountDownLatch allowPersist = new CountDownLatch(1); + private final ConcurrentHashMap> persisted = new ConcurrentHashMap<>(); + + @Override + public void persist(String consensusGroupKey, Map mutationEpochs) + throws IOException { + persistStarted.countDown(); + try { + allowPersist.await(); + } catch (InterruptedException e) { + Thread.currentThread().interrupt(); + throw new IOException("Interrupted while blocking persist", e); + } + persisted.put(consensusGroupKey, new HashMap<>(mutationEpochs)); + } + + @Override + public Map load(String consensusGroupKey) { + return persisted.getOrDefault(consensusGroupKey, Collections.emptyMap()); + } + + private boolean awaitPersistStarted(long timeout, TimeUnit unit) throws InterruptedException { + return persistStarted.await(timeout, unit); + } + + private void releasePersist() { + allowPersist.countDown(); + } + } } diff --git a/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/storageengine/dataregion/consistency/LogicalRepairSessionJournalTest.java b/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/storageengine/dataregion/consistency/LogicalRepairSessionJournalTest.java index a6a07f5baae2c..6f9fb25684af7 100644 --- a/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/storageengine/dataregion/consistency/LogicalRepairSessionJournalTest.java +++ b/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/storageengine/dataregion/consistency/LogicalRepairSessionJournalTest.java @@ -19,6 +19,8 @@ package org.apache.iotdb.db.storageengine.dataregion.consistency; +import org.apache.iotdb.db.conf.IoTDBDescriptor; + import org.junit.Assert; import org.junit.Test; @@ -153,6 +155,47 @@ public void completeShouldDeletePersistedSession() throws Exception { } } + @Test + public void defaultJournalShouldResolveSystemDirLazily() throws Exception { + String originalSystemDir = IoTDBDescriptor.getInstance().getConfig().getSystemDir(); + Path initialSystemDir = Files.createTempDirectory("logical-repair-journal-initial"); + Path effectiveSystemDir = Files.createTempDirectory("logical-repair-journal-effective"); + try { + IoTDBDescriptor.getInstance().getConfig().setSystemDir(initialSystemDir.toString()); + LogicalRepairSessionJournal journal = new LogicalRepairSessionJournal(); + + IoTDBDescriptor.getInstance().getConfig().setSystemDir(effectiveSystemDir.toString()); + String repairEpoch = "1:10:1000:2000:2000:1-1_2_3"; + journal.stageBatch( + "DataRegion-10", + 10L, + repairEpoch, + "session-d", + "LIVE", + "leaf:4:0", + 0, + "RESET_LEAF", + ByteBuffer.allocate(0)); + + Assert.assertTrue( + Files.exists( + effectiveSystemDir + .resolve("consistency-repair") + .resolve("sessions") + .resolve("session-d.session"))); + Assert.assertFalse( + Files.exists( + initialSystemDir + .resolve("consistency-repair") + .resolve("sessions") + .resolve("session-d.session"))); + } finally { + IoTDBDescriptor.getInstance().getConfig().setSystemDir(originalSystemDir); + deleteRecursively(initialSystemDir); + deleteRecursively(effectiveSystemDir); + } + } + private byte[] toByteArray(ByteBuffer buffer) { ByteBuffer duplicate = buffer.duplicate(); byte[] bytes = new byte[duplicate.remaining()];