diff --git a/fluss-common/src/main/java/org/apache/fluss/metrics/MetricNames.java b/fluss-common/src/main/java/org/apache/fluss/metrics/MetricNames.java
index 1b1445665e..8b787bc0bd 100644
--- a/fluss-common/src/main/java/org/apache/fluss/metrics/MetricNames.java
+++ b/fluss-common/src/main/java/org/apache/fluss/metrics/MetricNames.java
@@ -84,6 +84,14 @@ public class MetricNames {
public static final String SERVER_PHYSICAL_STORAGE_LOCAL_SIZE = "localSize";
public static final String SERVER_PHYSICAL_STORAGE_REMOTE_LOG_SIZE = "remoteLogSize";
+ /**
+ * Total physical storage size of standby replica snapshots on this tablet server.
+ *
+ *
Standby replicas maintain downloaded KV snapshots for fast leader failover. This metric
+ * tracks the total size of all KV snapshot data downloaded by standby replicas on this server.
+ */
+ public static final String SERVER_PHYSICAL_STORAGE_STANDBY_SIZE = "standbySize";
+
// --------------------------------------------------------------------------------------------
// metrics for user
// --------------------------------------------------------------------------------------------
diff --git a/fluss-rpc/src/main/proto/FlussApi.proto b/fluss-rpc/src/main/proto/FlussApi.proto
index f04b307565..312954c8d8 100644
--- a/fluss-rpc/src/main/proto/FlussApi.proto
+++ b/fluss-rpc/src/main/proto/FlussApi.proto
@@ -365,6 +365,7 @@ message NotifyKvSnapshotOffsetRequest {
required int32 bucket_id = 3;
required int32 coordinator_epoch = 4;
required int64 min_retain_offset = 5;
+ optional int64 snapshot_id = 6;
}
message NotifyKvSnapshotOffsetResponse {
@@ -900,6 +901,7 @@ message PbAdjustIsrReqForBucket {
repeated int32 new_isr = 4 [packed = true];
required int32 coordinator_epoch = 5;
required int32 bucket_epoch = 6;
+ repeated int32 standby_replicas = 7 [packed = true];
}
message PbAdjustIsrRespForTable {
@@ -917,6 +919,7 @@ message PbAdjustIsrRespForBucket {
repeated int32 isr = 7 [packed = true];
optional int32 bucket_epoch = 8;
optional int32 coordinator_epoch = 9;
+ repeated int32 standby_replicas = 10 [packed = true];
}
message PbListOffsetsRespForBucket {
@@ -937,6 +940,7 @@ message PbNotifyLeaderAndIsrReqForBucket {
repeated int32 replicas = 5 [packed = true];
repeated int32 isr = 6 [packed = true];
required int32 bucket_epoch = 7;
+ repeated int32 standby_replicas = 8 [packed = true];
}
message PbNotifyLeaderAndIsrRespForBucket {
diff --git a/fluss-server/src/main/java/org/apache/fluss/server/TabletManagerBase.java b/fluss-server/src/main/java/org/apache/fluss/server/TabletManagerBase.java
index 6525df1c45..9b0aaae705 100644
--- a/fluss-server/src/main/java/org/apache/fluss/server/TabletManagerBase.java
+++ b/fluss-server/src/main/java/org/apache/fluss/server/TabletManagerBase.java
@@ -173,7 +173,7 @@ protected int runInThreadPool(Runnable[] runnableJobs, String poolName) throws T
* @param tableBucket the table bucket
* @return the tablet directory
*/
- protected File getOrCreateTabletDir(PhysicalTablePath tablePath, TableBucket tableBucket) {
+ public File getOrCreateTabletDir(PhysicalTablePath tablePath, TableBucket tableBucket) {
File tabletDir = getTabletDir(tablePath, tableBucket);
if (tabletDir.exists()) {
return tabletDir;
diff --git a/fluss-server/src/main/java/org/apache/fluss/server/coordinator/CoordinatorEventProcessor.java b/fluss-server/src/main/java/org/apache/fluss/server/coordinator/CoordinatorEventProcessor.java
index 909fa503dc..d47d0e62db 100644
--- a/fluss-server/src/main/java/org/apache/fluss/server/coordinator/CoordinatorEventProcessor.java
+++ b/fluss-server/src/main/java/org/apache/fluss/server/coordinator/CoordinatorEventProcessor.java
@@ -1618,6 +1618,7 @@ private List tryProcessAdjustIsr(
// TODO: reject the request if there is a replica in ISR is not online,
// see KIP-841.
tryAdjustLeaderAndIsr.isr(),
+ tryAdjustLeaderAndIsr.standbyReplicas(),
coordinatorContext.getCoordinatorEpoch(),
currentLeaderAndIsr.bucketEpoch() + 1);
newLeaderAndIsrList.put(tableBucket, newLeaderAndIsr);
@@ -1740,7 +1741,9 @@ private void tryProcessCommitKvSnapshot(
completedSnapshotStore.add(completedSnapshot);
coordinatorEventManager.put(
new NotifyKvSnapshotOffsetEvent(
- tb, completedSnapshot.getLogOffset()));
+ tb,
+ completedSnapshot.getLogOffset(),
+ completedSnapshot.getSnapshotID()));
callback.complete(new CommitKvSnapshotResponse());
} catch (Exception e) {
callback.completeExceptionally(e);
@@ -1750,7 +1753,6 @@ private void tryProcessCommitKvSnapshot(
private void processNotifyKvSnapshotOffsetEvent(NotifyKvSnapshotOffsetEvent event) {
TableBucket tb = event.getTableBucket();
- long logOffset = event.getLogOffset();
coordinatorRequestBatch.newBatch();
coordinatorContext
.getBucketLeaderAndIsr(tb)
@@ -1761,7 +1763,8 @@ private void processNotifyKvSnapshotOffsetEvent(NotifyKvSnapshotOffsetEvent even
coordinatorContext.getFollowers(
tb, leaderAndIsr.leader()),
tb,
- logOffset));
+ event.getLogOffset(),
+ event.getSnapshotId()));
coordinatorRequestBatch.sendNotifyKvSnapshotOffsetRequest(
coordinatorContext.getCoordinatorEpoch());
}
diff --git a/fluss-server/src/main/java/org/apache/fluss/server/coordinator/CoordinatorRequestBatch.java b/fluss-server/src/main/java/org/apache/fluss/server/coordinator/CoordinatorRequestBatch.java
index 063ba58786..6b70149547 100644
--- a/fluss-server/src/main/java/org/apache/fluss/server/coordinator/CoordinatorRequestBatch.java
+++ b/fluss-server/src/main/java/org/apache/fluss/server/coordinator/CoordinatorRequestBatch.java
@@ -372,7 +372,10 @@ public void addNotifyRemoteLogOffsetsRequestForTabletServers(
}
public void addNotifyKvSnapshotOffsetRequestForTabletServers(
- List tabletServers, TableBucket tableBucket, long minRetainOffset) {
+ List tabletServers,
+ TableBucket tableBucket,
+ long minRetainOffset,
+ long snapshotId) {
tabletServers.stream()
.filter(s -> s >= 0)
.forEach(
@@ -380,7 +383,7 @@ public void addNotifyKvSnapshotOffsetRequestForTabletServers(
notifyKvSnapshotOffsetRequestMap.put(
id,
makeNotifyKvSnapshotOffsetRequest(
- tableBucket, minRetainOffset)));
+ tableBucket, minRetainOffset, snapshotId)));
}
public void addNotifyLakeTableOffsetRequestForTableServers(
diff --git a/fluss-server/src/main/java/org/apache/fluss/server/coordinator/event/NotifyKvSnapshotOffsetEvent.java b/fluss-server/src/main/java/org/apache/fluss/server/coordinator/event/NotifyKvSnapshotOffsetEvent.java
index da6a420676..2875d05ff0 100644
--- a/fluss-server/src/main/java/org/apache/fluss/server/coordinator/event/NotifyKvSnapshotOffsetEvent.java
+++ b/fluss-server/src/main/java/org/apache/fluss/server/coordinator/event/NotifyKvSnapshotOffsetEvent.java
@@ -25,10 +25,12 @@ public class NotifyKvSnapshotOffsetEvent implements CoordinatorEvent {
private final TableBucket tableBucket;
private final long logOffset;
+ private final long snapshotId;
- public NotifyKvSnapshotOffsetEvent(TableBucket tableBucket, long logOffset) {
+ public NotifyKvSnapshotOffsetEvent(TableBucket tableBucket, long logOffset, long snapshotId) {
this.tableBucket = tableBucket;
this.logOffset = logOffset;
+ this.snapshotId = snapshotId;
}
public TableBucket getTableBucket() {
@@ -38,4 +40,8 @@ public TableBucket getTableBucket() {
public long getLogOffset() {
return logOffset;
}
+
+ public long getSnapshotId() {
+ return snapshotId;
+ }
}
diff --git a/fluss-server/src/main/java/org/apache/fluss/server/coordinator/statemachine/ReplicaLeaderElection.java b/fluss-server/src/main/java/org/apache/fluss/server/coordinator/statemachine/ReplicaLeaderElection.java
index e9b0d33cf6..34a6679fe8 100644
--- a/fluss-server/src/main/java/org/apache/fluss/server/coordinator/statemachine/ReplicaLeaderElection.java
+++ b/fluss-server/src/main/java/org/apache/fluss/server/coordinator/statemachine/ReplicaLeaderElection.java
@@ -21,6 +21,7 @@
import org.apache.fluss.server.zk.data.LeaderAndIsr;
import java.util.ArrayList;
+import java.util.Collections;
import java.util.HashSet;
import java.util.List;
import java.util.Optional;
@@ -41,23 +42,25 @@ public static class DefaultLeaderElection extends ReplicaLeaderElection {
* @param assignments the assignments
* @param aliveReplicas the alive replicas
* @param leaderAndIsr the original leaderAndIsr
+ * @param isPrimaryKeyTable whether this table bucket is primary key table
* @return the election result
*/
public Optional leaderElection(
- List assignments, List aliveReplicas, LeaderAndIsr leaderAndIsr) {
- // currently, we always use the first replica in assignment, which also in aliveReplicas
- // and
- // isr as the leader replica.
+ List assignments,
+ List aliveReplicas,
+ LeaderAndIsr leaderAndIsr,
+ boolean isPrimaryKeyTable) {
List isr = leaderAndIsr.isr();
- for (int assignment : assignments) {
- if (aliveReplicas.contains(assignment) && isr.contains(assignment)) {
- return Optional.of(
- new TableBucketStateMachine.ElectionResult(
- aliveReplicas, leaderAndIsr.newLeaderAndIsr(assignment, isr)));
- }
- }
-
- return Optional.empty();
+ // First we will filter out the assignment list to only contain the alive replicas and
+ // isr.
+ List availableReplicas =
+ assignments.stream()
+ .filter(
+ replica ->
+ aliveReplicas.contains(replica)
+ && isr.contains(replica))
+ .collect(Collectors.toList());
+ return electLeader(availableReplicas, aliveReplicas, leaderAndIsr, isPrimaryKeyTable);
}
}
@@ -70,32 +73,66 @@ public static class ControlledShutdownLeaderElection extends ReplicaLeaderElecti
* @param aliveReplicas the alive replicas
* @param leaderAndIsr the original leaderAndIsr
* @param shutdownTabletServers the shutdown tabletServers
+ * @param isPrimaryKeyTable whether this table bucket is primary key table
* @return the election result
*/
public Optional leaderElection(
List assignments,
List aliveReplicas,
LeaderAndIsr leaderAndIsr,
- Set shutdownTabletServers) {
+ Set shutdownTabletServers,
+ boolean isPrimaryKeyTable) {
List originIsr = leaderAndIsr.isr();
Set isrSet = new HashSet<>(originIsr);
- for (Integer id : assignments) {
- if (aliveReplicas.contains(id)
- && isrSet.contains(id)
- && !shutdownTabletServers.contains(id)) {
- Set newAliveReplicas = new HashSet<>(aliveReplicas);
- newAliveReplicas.removeAll(shutdownTabletServers);
- List newIsr =
- originIsr.stream()
- .filter(replica -> !shutdownTabletServers.contains(replica))
- .collect(Collectors.toList());
- return Optional.of(
- new ElectionResult(
- new ArrayList<>(newAliveReplicas),
- leaderAndIsr.newLeaderAndIsr(id, newIsr)));
- }
+ // Filter out available replicas: alive, in ISR, and not shutting down.
+ List availableReplicas =
+ assignments.stream()
+ .filter(
+ replica ->
+ aliveReplicas.contains(replica)
+ && isrSet.contains(replica)
+ && !shutdownTabletServers.contains(replica))
+ .collect(Collectors.toList());
+
+ if (availableReplicas.isEmpty()) {
+ return Optional.empty();
}
- return Optional.empty();
+
+ // For log table, simply use the first available replica as leader.
+ if (!isPrimaryKeyTable) {
+ List newIsr =
+ originIsr.stream()
+ .filter(replica -> !shutdownTabletServers.contains(replica))
+ .collect(Collectors.toList());
+ Set newAliveReplicas = new HashSet<>(aliveReplicas);
+ newAliveReplicas.removeAll(shutdownTabletServers);
+ return Optional.of(
+ new ElectionResult(
+ new ArrayList<>(newAliveReplicas),
+ leaderAndIsr.newLeaderAndIsr(
+ availableReplicas.get(0),
+ newIsr,
+ Collections.emptyList())));
+ }
+
+ // For PK table, elect leader and standby.
+ LeaderAndStandby leaderAndStandby =
+ electLeaderAndStandbyForPkTable(availableReplicas, leaderAndIsr);
+
+ Set newAliveReplicas = new HashSet<>(aliveReplicas);
+ newAliveReplicas.removeAll(shutdownTabletServers);
+ List newIsr =
+ originIsr.stream()
+ .filter(replica -> !shutdownTabletServers.contains(replica))
+ .collect(Collectors.toList());
+
+ return Optional.of(
+ new ElectionResult(
+ new ArrayList<>(newAliveReplicas),
+ leaderAndIsr.newLeaderAndIsr(
+ leaderAndStandby.leader,
+ newIsr,
+ leaderAndStandby.standbyReplicas)));
}
}
@@ -108,20 +145,109 @@ public ReassignmentLeaderElection(List newReplicas) {
}
public Optional leaderElection(
- List liveReplicas, LeaderAndIsr leaderAndIsr) {
- // currently, we always use the first replica in targetReplicas, which also in
+ List liveReplicas, LeaderAndIsr leaderAndIsr, boolean isPrimaryKeyTable) {
+ // Currently, we always use the first replica in targetReplicas, which also in
// liveReplicas and isr as the leader replica. For bucket reassignment, the first
// replica is the target leader replica.
List isr = leaderAndIsr.isr();
- for (int assignment : newReplicas) {
- if (liveReplicas.contains(assignment) && isr.contains(assignment)) {
- return Optional.of(
- new ElectionResult(
- liveReplicas, leaderAndIsr.newLeaderAndIsr(assignment, isr)));
- }
- }
+ List availableReplicas =
+ newReplicas.stream()
+ .filter(
+ replica ->
+ liveReplicas.contains(replica) && isr.contains(replica))
+ .collect(Collectors.toList());
+ return electLeader(availableReplicas, liveReplicas, leaderAndIsr, isPrimaryKeyTable);
+ }
+ }
+
+ // ------------------------------------------------------------------------
+ // Common election logic
+ // ------------------------------------------------------------------------
+ private static Optional electLeader(
+ List availableReplicas,
+ List aliveReplicas,
+ LeaderAndIsr leaderAndIsr,
+ boolean isPrimaryKeyTable) {
+ if (availableReplicas.isEmpty()) {
return Optional.empty();
}
+
+ List isr = leaderAndIsr.isr();
+
+ // For log table, simply use the first available replica as leader.
+ if (!isPrimaryKeyTable) {
+ return Optional.of(
+ new ElectionResult(
+ aliveReplicas,
+ leaderAndIsr.newLeaderAndIsr(
+ availableReplicas.get(0), isr, Collections.emptyList())));
+ }
+
+ // For PK table, elect leader and standby.
+ LeaderAndStandby leaderAndStandby =
+ electLeaderAndStandbyForPkTable(availableReplicas, leaderAndIsr);
+ return Optional.of(
+ new ElectionResult(
+ aliveReplicas,
+ leaderAndIsr.newLeaderAndIsr(
+ leaderAndStandby.leader, isr, leaderAndStandby.standbyReplicas)));
+ }
+
+ /**
+ * Elect leader and standby for PK table.
+ *
+ * Election strategy:
+ *
+ *
+ * - If current standby exists and is available, promote it to leader
+ *
- Otherwise, use the first available replica as leader
+ *
- Select new standby from remaining available replicas (if any)
+ *
+ */
+ private static LeaderAndStandby electLeaderAndStandbyForPkTable(
+ List availableReplicas, LeaderAndIsr leaderAndIsr) {
+ int currentStandby = getCurrentStandby(leaderAndIsr);
+ int newLeader;
+
+ if (currentStandby != -1 && availableReplicas.contains(currentStandby)) {
+ // Promote current standby to leader.
+ newLeader = currentStandby;
+ } else {
+ // Use first available replica as leader.
+ newLeader = availableReplicas.get(0);
+ }
+
+ // Find new standby from remaining replicas.
+ List standbyReplicas = findNewStandby(availableReplicas, newLeader);
+ return new LeaderAndStandby(newLeader, standbyReplicas);
+ }
+
+ /** Get current standby replica ID, returns -1 if no standby exists. */
+ private static int getCurrentStandby(LeaderAndIsr leaderAndIsr) {
+ return leaderAndIsr.standbyReplicas().isEmpty()
+ ? -1
+ : leaderAndIsr.standbyReplicas().get(0);
+ }
+
+ /** Find new standby from available replicas, excluding the leader. */
+ private static List findNewStandby(
+ List availableReplicas, int excludeLeader) {
+ return availableReplicas.stream()
+ .filter(replica -> replica != excludeLeader)
+ .findFirst()
+ .map(Collections::singletonList)
+ .orElse(Collections.emptyList());
+ }
+
+ /** Internal class to hold leader and standby election result. */
+ private static class LeaderAndStandby {
+ final int leader;
+ final List standbyReplicas;
+
+ LeaderAndStandby(int leader, List standbyReplicas) {
+ this.leader = leader;
+ this.standbyReplicas = standbyReplicas;
+ }
}
}
diff --git a/fluss-server/src/main/java/org/apache/fluss/server/coordinator/statemachine/ReplicaStateMachine.java b/fluss-server/src/main/java/org/apache/fluss/server/coordinator/statemachine/ReplicaStateMachine.java
index 091c2cbc9c..7d5bc37e41 100644
--- a/fluss-server/src/main/java/org/apache/fluss/server/coordinator/statemachine/ReplicaStateMachine.java
+++ b/fluss-server/src/main/java/org/apache/fluss/server/coordinator/statemachine/ReplicaStateMachine.java
@@ -481,7 +481,8 @@ private Map doRemoveReplicaFromIsr(
.collect(Collectors.toList());
LeaderAndIsr adjustLeaderAndIsr =
newLeader == LeaderAndIsr.NO_LEADER
- ? leaderAndIsr.newLeaderAndIsr(newLeader, newIsr)
+ ? leaderAndIsr.newLeaderAndIsr(
+ newLeader, newIsr, leaderAndIsr.standbyReplicas())
: leaderAndIsr.newLeaderAndIsr(newIsr);
adjustedLeaderAndIsr.put(tableBucketReplica, adjustLeaderAndIsr);
toUpdateLeaderAndIsrList.put(tableBucket, adjustLeaderAndIsr);
diff --git a/fluss-server/src/main/java/org/apache/fluss/server/coordinator/statemachine/TableBucketStateMachine.java b/fluss-server/src/main/java/org/apache/fluss/server/coordinator/statemachine/TableBucketStateMachine.java
index 85dcc434f4..deeba0502c 100644
--- a/fluss-server/src/main/java/org/apache/fluss/server/coordinator/statemachine/TableBucketStateMachine.java
+++ b/fluss-server/src/main/java/org/apache/fluss/server/coordinator/statemachine/TableBucketStateMachine.java
@@ -439,7 +439,12 @@ private Optional doInitElectionForBucket(
// servers as inSyncReplica set.
Optional resultOpt =
initReplicaLeaderElection(
- assignedServers, liveServers, coordinatorContext.getCoordinatorEpoch());
+ assignedServers,
+ liveServers,
+ coordinatorContext.getCoordinatorEpoch(),
+ coordinatorContext
+ .getTableInfoById(tableBucket.getTableId())
+ .hasPrimaryKey());
if (!resultOpt.isPresent()) {
LOG.error(
"The leader election for table bucket {} is empty.",
@@ -613,10 +618,12 @@ private Optional electLeader(
}
Optional resultOpt = Optional.empty();
+ boolean isPkTable =
+ coordinatorContext.getTableInfoById(tableBucket.getTableId()).hasPrimaryKey();
if (electionStrategy instanceof DefaultLeaderElection) {
resultOpt =
((DefaultLeaderElection) electionStrategy)
- .leaderElection(assignment, liveReplicas, leaderAndIsr);
+ .leaderElection(assignment, liveReplicas, leaderAndIsr, isPkTable);
} else if (electionStrategy instanceof ControlledShutdownLeaderElection) {
Set shuttingDownTabletServers = coordinatorContext.shuttingDownTabletServers();
resultOpt =
@@ -625,11 +632,12 @@ private Optional electLeader(
assignment,
liveReplicas,
leaderAndIsr,
- shuttingDownTabletServers);
+ shuttingDownTabletServers,
+ isPkTable);
} else if (electionStrategy instanceof ReassignmentLeaderElection) {
resultOpt =
((ReassignmentLeaderElection) electionStrategy)
- .leaderElection(liveReplicas, leaderAndIsr);
+ .leaderElection(liveReplicas, leaderAndIsr, isPkTable);
}
if (!resultOpt.isPresent()) {
@@ -670,23 +678,40 @@ public LeaderAndIsr getLeaderAndIsr() {
* @param assignments the assignments
* @param aliveReplicas the alive replicas
* @param coordinatorEpoch the coordinator epoch
+ * @param isPrimaryKeyTable whether this table bucket is primary key table
* @return the election result
*/
@VisibleForTesting
public static Optional initReplicaLeaderElection(
- List assignments, List aliveReplicas, int coordinatorEpoch) {
- // currently, we always use the first replica in assignment, which also in aliveReplicas and
- // isr as the leader replica.
- for (int assignment : assignments) {
- if (aliveReplicas.contains(assignment)) {
- return Optional.of(
- new ElectionResult(
- aliveReplicas,
- new LeaderAndIsr(
- assignment, 0, aliveReplicas, coordinatorEpoch, 0)));
+ List assignments,
+ List aliveReplicas,
+ int coordinatorEpoch,
+ boolean isPrimaryKeyTable) {
+ // First we will filter out the assignment list to only contain the alive replicas.
+ List availableReplicas =
+ assignments.stream().filter(aliveReplicas::contains).collect(Collectors.toList());
+
+ // If the assignment list is empty, we return empty.
+ if (availableReplicas.isEmpty()) {
+ return Optional.empty();
+ }
+
+ // Then we will use the first replica in assignment as the leader replica.
+ int leader = availableReplicas.get(0);
+
+ // If this table is primaryKey table, we will use the second replica in assignment as the
+ // standby if exists.
+ List standbyReplicas = new ArrayList<>();
+ if (isPrimaryKeyTable) {
+ if (availableReplicas.size() > 1) {
+ standbyReplicas.add(availableReplicas.get(1));
}
}
- return Optional.empty();
+ return Optional.of(
+ new ElectionResult(
+ aliveReplicas,
+ new LeaderAndIsr(
+ leader, 0, aliveReplicas, standbyReplicas, coordinatorEpoch, 0)));
}
}
diff --git a/fluss-server/src/main/java/org/apache/fluss/server/entity/NotifyKvSnapshotOffsetData.java b/fluss-server/src/main/java/org/apache/fluss/server/entity/NotifyKvSnapshotOffsetData.java
index f6f18b1439..b31c3786a8 100644
--- a/fluss-server/src/main/java/org/apache/fluss/server/entity/NotifyKvSnapshotOffsetData.java
+++ b/fluss-server/src/main/java/org/apache/fluss/server/entity/NotifyKvSnapshotOffsetData.java
@@ -18,19 +18,25 @@
package org.apache.fluss.server.entity;
import org.apache.fluss.metadata.TableBucket;
-import org.apache.fluss.rpc.messages.NotifyRemoteLogOffsetsRequest;
-/** The data for request {@link NotifyRemoteLogOffsetsRequest}. */
+import javax.annotation.Nullable;
+
+/** The data for NotifyKvSnapshotOffset request. */
public class NotifyKvSnapshotOffsetData {
private final TableBucket tableBucket;
private final long minRetainOffset;
private final int coordinatorEpoch;
+ private final @Nullable Long snapshotId;
public NotifyKvSnapshotOffsetData(
- TableBucket tableBucket, long minRetainOffset, int coordinatorEpoch) {
+ TableBucket tableBucket,
+ long minRetainOffset,
+ int coordinatorEpoch,
+ @Nullable Long snapshotId) {
this.tableBucket = tableBucket;
this.minRetainOffset = minRetainOffset;
this.coordinatorEpoch = coordinatorEpoch;
+ this.snapshotId = snapshotId;
}
public TableBucket getTableBucket() {
@@ -45,15 +51,22 @@ public int getCoordinatorEpoch() {
return coordinatorEpoch;
}
+ @Nullable
+ public Long getSnapshotId() {
+ return snapshotId;
+ }
+
@Override
public String toString() {
- return "NotifyRemoteLogOffsetsData{"
+ return "NotifyKvSnapshotOffsetData{"
+ "tableBucket="
+ tableBucket
+ ", minRetainOffset="
+ minRetainOffset
+ ", coordinatorEpoch="
+ coordinatorEpoch
+ + ", snapshotId="
+ + snapshotId
+ '}';
}
}
diff --git a/fluss-server/src/main/java/org/apache/fluss/server/entity/NotifyLeaderAndIsrData.java b/fluss-server/src/main/java/org/apache/fluss/server/entity/NotifyLeaderAndIsrData.java
index 8ba0002084..e6702cdcaf 100644
--- a/fluss-server/src/main/java/org/apache/fluss/server/entity/NotifyLeaderAndIsrData.java
+++ b/fluss-server/src/main/java/org/apache/fluss/server/entity/NotifyLeaderAndIsrData.java
@@ -85,4 +85,12 @@ public int[] getIsrArray() {
public LeaderAndIsr getLeaderAndIsr() {
return leaderAndIsr;
}
+
+ public List getStandbyReplicas() {
+ return leaderAndIsr.standbyReplicas();
+ }
+
+ public int[] getStandbyReplicasArray() {
+ return leaderAndIsr.standbyReplicas().stream().mapToInt(Integer::intValue).toArray();
+ }
}
diff --git a/fluss-server/src/main/java/org/apache/fluss/server/kv/KvManager.java b/fluss-server/src/main/java/org/apache/fluss/server/kv/KvManager.java
index 1f0c9c2689..5cf2b1258e 100644
--- a/fluss-server/src/main/java/org/apache/fluss/server/kv/KvManager.java
+++ b/fluss-server/src/main/java/org/apache/fluss/server/kv/KvManager.java
@@ -307,39 +307,50 @@ public Optional getKv(TableBucket tableBucket) {
return Optional.ofNullable(currentKvs.get(tableBucket));
}
- public void dropKv(TableBucket tableBucket) {
- KvTablet dropKvTablet =
+ public void closeOrDropKv(TableBucket tableBucket, boolean needDrop) {
+ KvTablet removeTablet =
inLock(tabletCreationOrDeletionLock, () -> currentKvs.remove(tableBucket));
- if (dropKvTablet != null) {
- TablePath tablePath = dropKvTablet.getTablePath();
+ if (removeTablet != null) {
+ TablePath tablePath = removeTablet.getTablePath();
try {
- dropKvTablet.drop();
- if (dropKvTablet.getPartitionName() == null) {
- LOG.info(
- "Deleted kv bucket {} for table {} in file path {}.",
- tableBucket.getBucket(),
- tablePath,
- dropKvTablet.getKvTabletDir().getAbsolutePath());
+ if (needDrop) {
+ removeTablet.drop();
+ if (removeTablet.getPartitionName() == null) {
+ LOG.info(
+ "Deleted kv bucket {} for table {} in file path {}.",
+ tableBucket.getBucket(),
+ tablePath,
+ removeTablet.getKvTabletDir().getAbsolutePath());
+ } else {
+ LOG.info(
+ "Deleted kv bucket {} for the partition {} of table {} in file path {}.",
+ tableBucket.getBucket(),
+ removeTablet.getPartitionName(),
+ tablePath,
+ removeTablet.getKvTabletDir().getAbsolutePath());
+ }
} else {
+ removeTablet.close();
LOG.info(
- "Deleted kv bucket {} for the partition {} of table {} in file path {}.",
- tableBucket.getBucket(),
- dropKvTablet.getPartitionName(),
- tablePath,
- dropKvTablet.getKvTabletDir().getAbsolutePath());
+ "Closed kvTablet and released RocksDB lock for bucket {} without deleting files",
+ tableBucket);
}
} catch (Exception e) {
throw new KvStorageException(
String.format(
- "Exception while deleting kv for table %s, bucket %s in dir %s.",
+ "Exception while deleting or closing kv for table %s, bucket %s in dir %s, needDrop: %s.",
tablePath,
tableBucket.getBucket(),
- dropKvTablet.getKvTabletDir().getAbsolutePath()),
+ removeTablet.getKvTabletDir().getAbsolutePath(),
+ needDrop),
e);
}
} else {
- LOG.warn("Fail to delete kv bucket {}.", tableBucket.getBucket());
+ LOG.warn(
+ "Fail to delete or close kv bucket {}, needDrop: {}.",
+ tableBucket.getBucket(),
+ needDrop);
}
}
diff --git a/fluss-server/src/main/java/org/apache/fluss/server/kv/KvSnapshotResource.java b/fluss-server/src/main/java/org/apache/fluss/server/kv/KvSnapshotResource.java
index 85ee1ebcf6..1dff6a7e76 100644
--- a/fluss-server/src/main/java/org/apache/fluss/server/kv/KvSnapshotResource.java
+++ b/fluss-server/src/main/java/org/apache/fluss/server/kv/KvSnapshotResource.java
@@ -17,6 +17,7 @@
package org.apache.fluss.server.kv;
+import org.apache.fluss.annotation.VisibleForTesting;
import org.apache.fluss.config.ConfigOptions;
import org.apache.fluss.config.Configuration;
import org.apache.fluss.server.kv.snapshot.KvSnapshotDataDownloader;
@@ -52,7 +53,8 @@ public class KvSnapshotResource {
/** A downloader to download snapshot data. */
private final KvSnapshotDataDownloader kvSnapshotDataDownloader;
- private KvSnapshotResource(
+ @VisibleForTesting
+ public KvSnapshotResource(
ScheduledExecutorService kvSnapshotScheduler,
KvSnapshotDataUploader kvSnapshotDataUploader,
KvSnapshotDataDownloader kvSnapshotDataDownloader,
diff --git a/fluss-server/src/main/java/org/apache/fluss/server/kv/snapshot/CompletedSnapshot.java b/fluss-server/src/main/java/org/apache/fluss/server/kv/snapshot/CompletedSnapshot.java
index 86cd25858b..f14733253b 100644
--- a/fluss-server/src/main/java/org/apache/fluss/server/kv/snapshot/CompletedSnapshot.java
+++ b/fluss-server/src/main/java/org/apache/fluss/server/kv/snapshot/CompletedSnapshot.java
@@ -114,6 +114,17 @@ public CompletedSnapshot(
this(tableBucket, snapshotID, snapshotLocation, kvSnapshotHandle, 0, null, null);
}
+ public CompletedSnapshot getIncrementalSnapshot(KvSnapshotHandle newKvSnapshotHandle) {
+ return new CompletedSnapshot(
+ tableBucket,
+ snapshotID,
+ snapshotLocation,
+ newKvSnapshotHandle,
+ logOffset,
+ rowCount,
+ autoIncIDRanges);
+ }
+
public long getSnapshotID() {
return snapshotID;
}
@@ -184,6 +195,10 @@ private void disposeSnapshotStorage() throws IOException {
}
}
+ public CompletedSnapshotHandle getCompletedSnapshotHandle() {
+ return new CompletedSnapshotHandle(snapshotID, getMetadataFilePath(), logOffset);
+ }
+
/**
* Return the metadata file path that stores all the information that describes the snapshot.
*/
diff --git a/fluss-server/src/main/java/org/apache/fluss/server/kv/snapshot/CompletedSnapshotHandleStore.java b/fluss-server/src/main/java/org/apache/fluss/server/kv/snapshot/CompletedSnapshotHandleStore.java
index 31f2424e3c..0e2d796c77 100644
--- a/fluss-server/src/main/java/org/apache/fluss/server/kv/snapshot/CompletedSnapshotHandleStore.java
+++ b/fluss-server/src/main/java/org/apache/fluss/server/kv/snapshot/CompletedSnapshotHandleStore.java
@@ -82,4 +82,15 @@ List getAllCompletedSnapshotHandles(TableBucket tableBu
*/
Optional getLatestCompletedSnapshotHandle(TableBucket tableBucket)
throws Exception;
+
+ /**
+ * Get the completed snapshot handle for the given snapshot id of the given table bucket.
+ *
+ * @param tableBucket the table bucket the snapshot belongs to
+ * @param snapshotId the snapshot id
+ * @return the completed snapshot handle, empty if no any snapshot
+ * @throws Exception if getting failed
+ */
+ Optional getCompletedSnapshotHandle(
+ TableBucket tableBucket, long snapshotId) throws Exception;
}
diff --git a/fluss-server/src/main/java/org/apache/fluss/server/kv/snapshot/DefaultSnapshotContext.java b/fluss-server/src/main/java/org/apache/fluss/server/kv/snapshot/DefaultSnapshotContext.java
index 9bec8977cc..ec13369097 100644
--- a/fluss-server/src/main/java/org/apache/fluss/server/kv/snapshot/DefaultSnapshotContext.java
+++ b/fluss-server/src/main/java/org/apache/fluss/server/kv/snapshot/DefaultSnapshotContext.java
@@ -158,6 +158,18 @@ public FsPath getRemoteKvDir() {
};
}
+ @Override
+ public CompletedSnapshot getCompletedSnapshotProvider(TableBucket tableBucket, long snapshotId)
+ throws Exception {
+ Optional optSnapshotHandle =
+ completedSnapshotHandleStore.getCompletedSnapshotHandle(tableBucket, snapshotId);
+ if (optSnapshotHandle.isPresent()) {
+ return optSnapshotHandle.get().retrieveCompleteSnapshot();
+ } else {
+ return null;
+ }
+ }
+
@Override
public int maxFetchLogSizeInRecoverKv() {
return maxFetchLogSizeInRecoverKv;
diff --git a/fluss-server/src/main/java/org/apache/fluss/server/kv/snapshot/KvSnapshotManager.java b/fluss-server/src/main/java/org/apache/fluss/server/kv/snapshot/KvSnapshotManager.java
new file mode 100644
index 0000000000..d52a2696af
--- /dev/null
+++ b/fluss-server/src/main/java/org/apache/fluss/server/kv/snapshot/KvSnapshotManager.java
@@ -0,0 +1,1011 @@
+/*
+ * 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.fluss.server.kv.snapshot;
+
+import org.apache.fluss.annotation.VisibleForTesting;
+import org.apache.fluss.fs.FileSystemSafetyNet;
+import org.apache.fluss.fs.FsPath;
+import org.apache.fluss.metadata.TableBucket;
+import org.apache.fluss.server.kv.rocksdb.RocksDBKvBuilder;
+import org.apache.fluss.utils.CloseableRegistry;
+import org.apache.fluss.utils.FileUtils;
+import org.apache.fluss.utils.IOUtils;
+import org.apache.fluss.utils.MathUtils;
+import org.apache.fluss.utils.clock.Clock;
+import org.apache.fluss.utils.concurrent.FutureUtils;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import javax.annotation.Nullable;
+import javax.annotation.concurrent.NotThreadSafe;
+
+import java.io.Closeable;
+import java.io.File;
+import java.io.IOException;
+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.util.ArrayList;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Optional;
+import java.util.Set;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.Executor;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.RunnableFuture;
+import java.util.concurrent.ScheduledExecutorService;
+import java.util.concurrent.ScheduledFuture;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicInteger;
+import java.util.function.LongSupplier;
+import java.util.stream.Collectors;
+
+import static org.apache.fluss.server.kv.snapshot.RocksIncrementalSnapshot.SST_FILE_SUFFIX;
+import static org.apache.fluss.utils.Preconditions.checkNotNull;
+
+/* This file is based on source code of Apache Flink Project (https://flink.apache.org/), licensed by the Apache
+ * Software Foundation (ASF) under the Apache License, Version 2.0. See the NOTICE file distributed with this work for
+ * additional information regarding copyright ownership. */
+
+/**
+ * For a leader replica of PrimaryKey Table, it is a stateless snapshot manager which will trigger
+ * upload kv snapshot periodically. It'll use a {@link ScheduledExecutorService} to schedule the
+ * snapshot initialization and a {@link ExecutorService} to complete async phase of snapshot.
+ *
+ * For a standby replica of PrimaryKey Table, it will trigger by the
+ * NotifyKvSnapshotOffsetRequest to incremental download sst files for remote to keep the data up to
+ * the latest kv snapshot.
+ *
+ *
For a follower replica of PrimaryKey Table, it will do nothing.
+ */
+public class KvSnapshotManager implements Closeable {
+
+ private static final Logger LOG = LoggerFactory.getLogger(KvSnapshotManager.class);
+
+ /** Number of consecutive snapshot failures. */
+ private final AtomicInteger numberOfConsecutiveFailures;
+
+ /** Whether upload snapshot is started. */
+ private volatile boolean isLeader = false;
+
+ /** Whether the replica is a standby replica. */
+ private volatile boolean isStandby = false;
+
+ /** Whether the standby replica is initializing. */
+ private volatile boolean standbyInitializing = false;
+
+ /**
+ * A supplier to get the current snapshot delay. This allows dynamic reconfiguration of the
+ * snapshot interval at runtime.
+ */
+ private final LongSupplier snapshotIntervalSupplier;
+
+ private final long initialDelay;
+ /** The table bucket that the snapshot manager is for. */
+ private final TableBucket tableBucket;
+
+ private final File tabletDir;
+
+ private final SnapshotContext snapshotContext;
+ private final Clock clock;
+
+ /** The target on which the snapshot will be done. */
+ private @Nullable UploadSnapshotTarget uploadSnapshotTarget;
+
+ /**
+ * The scheduled snapshot task.
+ *
+ *
Since all reads and writes of {@code scheduledTask} are protected by synchronized, the
+ * volatile modifier is not necessary here.
+ */
+ private ScheduledFuture> scheduledTask = null;
+
+ /** The sst files downloaded for standby replicas. */
+ private @Nullable Set downloadedSstFiles;
+
+ private @Nullable Set downloadedMiscFiles;
+ private long standbySnapshotSize;
+
+ protected KvSnapshotManager(
+ TableBucket tableBucket, File tabletDir, SnapshotContext snapshotContext, Clock clock) {
+ this.tableBucket = tableBucket;
+ this.tabletDir = tabletDir;
+ this.snapshotContext = snapshotContext;
+ this.numberOfConsecutiveFailures = new AtomicInteger(0);
+ this.snapshotIntervalSupplier = snapshotContext::getSnapshotIntervalMs;
+ long periodicSnapshotDelay = snapshotIntervalSupplier.getAsLong();
+ this.initialDelay =
+ periodicSnapshotDelay > 0
+ ? MathUtils.murmurHash(tableBucket.hashCode()) % periodicSnapshotDelay
+ : 0;
+ this.clock = clock;
+ this.uploadSnapshotTarget = null;
+ this.downloadedSstFiles = null;
+ this.downloadedMiscFiles = null;
+ this.standbySnapshotSize = 0;
+ }
+
+ public static KvSnapshotManager create(
+ TableBucket tableBucket, File tabletDir, SnapshotContext snapshotContext, Clock clock) {
+ return new KvSnapshotManager(tableBucket, tabletDir, snapshotContext, clock);
+ }
+
+ public void becomeLeader() {
+ isLeader = true;
+ isStandby = false;
+ // Clear standby download cache when leaving standby role
+ clearStandbyDownloadCache();
+ ensureDbDirectoryExists();
+ }
+
+ public void becomeFollower() {
+ isLeader = false;
+ isStandby = false;
+ // Clear standby download cache when leaving standby role
+ clearStandbyDownloadCache();
+ }
+
+ public void becomeStandby() {
+ isLeader = false;
+ isStandby = true;
+ // Clear standby download cache when new added to standby role
+ clearStandbyDownloadCache();
+ ensureDbDirectoryExists();
+ }
+
+ @VisibleForTesting
+ public @Nullable Set getDownloadedSstFiles() {
+ return downloadedSstFiles;
+ }
+
+ @VisibleForTesting
+ public @Nullable Set getDownloadedMiscFiles() {
+ return downloadedMiscFiles;
+ }
+
+ /**
+ * Clear the standby download cache.
+ *
+ * This method should be called when a replica leaves the standby role (becomes a regular
+ * follower or leader). It clears the cached state of downloaded SST files, misc files, and
+ * snapshot size. This ensures that if the replica becomes standby again later, it will perform
+ * a fresh download based on the actual local files, rather than reusing stale cache that
+ * references deleted files.
+ */
+ private void clearStandbyDownloadCache() {
+ downloadedSstFiles = null;
+ downloadedMiscFiles = null;
+ standbySnapshotSize = 0;
+ LOG.info(
+ "Cleared standby download cache for table bucket {}, will reload from local files on next standby promotion",
+ tableBucket);
+ }
+
+ /** Ensure the RocksDB data directory exists. */
+ private void ensureDbDirectoryExists() {
+ Path kvDbPath = tabletDir.toPath().resolve(RocksDBKvBuilder.DB_INSTANCE_DIR_STRING);
+ if (!kvDbPath.toFile().exists()) {
+ kvDbPath.toFile().mkdirs();
+ }
+ }
+
+ /**
+ * The guardedExecutor is an executor that uses to trigger upload snapshot.
+ *
+ *
It's expected to be passed with a guarded executor to prevent any concurrent modification
+ * to KvTablet during trigger snapshotting.
+ */
+ public void startPeriodicUploadSnapshot(
+ Executor guardedExecutor, UploadSnapshotTarget uploadSnapshotTarget) {
+ this.uploadSnapshotTarget = uploadSnapshotTarget;
+
+ // disable periodic snapshot when periodicMaterializeDelay is not positive
+ if (initialDelay > 0) {
+ LOG.info("TableBucket {} starts periodic snapshot", tableBucket);
+ scheduleNextSnapshot(initialDelay, guardedExecutor);
+ }
+ }
+
+ public void downloadSnapshot(long snapshotId) throws Exception {
+ if (standbyInitializing || !isStandby) {
+ LOG.info(
+ "Skip downloading snapshot {} for bucket {} because replica is no "
+ + "longer standby or is initializing. is standby: {}, is initializing: {}",
+ snapshotId,
+ tableBucket,
+ isStandby,
+ standbyInitializing);
+ return;
+ }
+ CompletedSnapshot completedSnapshot =
+ snapshotContext.getCompletedSnapshotProvider(tableBucket, snapshotId);
+ if (completedSnapshot == null) {
+ LOG.warn(
+ "Snapshot {} not found for bucket {}, skip downloading.",
+ snapshotId,
+ tableBucket);
+ return;
+ }
+ long startTime = clock.milliseconds();
+ incrementalDownloadSnapshot(completedSnapshot);
+ standbySnapshotSize = completedSnapshot.getSnapshotSize();
+ LOG.info(
+ "Standby download snapshot {} for bucket {} completed in {} ms, snapshot size: {}.",
+ snapshotId,
+ tableBucket,
+ clock.milliseconds() - startTime,
+ standbySnapshotSize);
+ }
+
+ /**
+ * download the latest snapshot.
+ *
+ *
For a standby replica, it will download the latest snapshot to keep the data up to the
+ * latest kv snapshot.
+ *
+ * @return the latest snapshot
+ */
+ public Optional downloadLatestSnapshot() throws Exception {
+ // standbyInitializing is used to prevent concurrent download via
+ // downloadSnapshot(snapshotId).
+ standbyInitializing = true;
+ try {
+ // Note: no isStandby check here. This method is called from both:
+ // 1. initKvTablet() during leader initialization - isStandby is already false
+ // 2. becomeStandbyAsync() during standby initialization - isStandby is true
+ // The isStandby guard is only needed in downloadSnapshot(snapshotId) which is
+ // called from the notification path exclusively for standby replicas.
+ Optional latestSnapshot = getLatestSnapshot();
+ if (latestSnapshot.isPresent()) {
+ CompletedSnapshot completedSnapshot = latestSnapshot.get();
+ long startTime = clock.milliseconds();
+ incrementalDownloadSnapshot(completedSnapshot);
+ standbySnapshotSize = completedSnapshot.getSnapshotSize();
+ LOG.info(
+ "Downloaded latest snapshot {} for bucket {} in {} ms, "
+ + "snapshot log offset: {}, snapshot size: {}.",
+ completedSnapshot.getSnapshotID(),
+ tableBucket,
+ clock.milliseconds() - startTime,
+ completedSnapshot.getLogOffset(),
+ standbySnapshotSize);
+ } else {
+ LOG.info(
+ "No snapshot found for bucket {}, will init kv from scratch.", tableBucket);
+ }
+
+ return latestSnapshot;
+ } finally {
+ standbyInitializing = false;
+ }
+ }
+
+ private void incrementalDownloadSnapshot(CompletedSnapshot completedSnapshot) throws Exception {
+ if (downloadedSstFiles == null || downloadedMiscFiles == null) {
+ // first try to load all ready exists sst files.
+ downloadedSstFiles = new HashSet<>();
+ downloadedMiscFiles = new HashSet<>();
+ loadKvLocalFiles(downloadedSstFiles, downloadedMiscFiles);
+ }
+
+ Set sstFilesToDelete = new HashSet<>();
+ KvSnapshotHandle incrementalKvSnapshotHandle =
+ getIncrementalKvSnapshotHandle(
+ completedSnapshot, downloadedSstFiles, sstFilesToDelete);
+ CompletedSnapshot incrementalSnapshot =
+ completedSnapshot.getIncrementalSnapshot(incrementalKvSnapshotHandle);
+
+ // Use atomic download: download to temp dir first, then atomically move to final location
+ atomicDownloadSnapshot(incrementalSnapshot, sstFilesToDelete);
+
+ // Update cached file sets with absolute paths (resolved against the db directory)
+ // to stay consistent with loadKvLocalFiles() which returns absolute paths.
+ Path kvDbPath = tabletDir.toPath().resolve(RocksDBKvBuilder.DB_INSTANCE_DIR_STRING);
+ KvSnapshotHandle kvSnapshotHandle = completedSnapshot.getKvSnapshotHandle();
+ downloadedSstFiles =
+ kvSnapshotHandle.getSharedKvFileHandles().stream()
+ .map(handler -> kvDbPath.resolve(handler.getLocalPath()))
+ .collect(Collectors.toSet());
+ downloadedMiscFiles =
+ kvSnapshotHandle.getPrivateFileHandles().stream()
+ .map(handler -> kvDbPath.resolve(handler.getLocalPath()))
+ .collect(Collectors.toSet());
+ }
+
+ public long getSnapshotSize() {
+ if (uploadSnapshotTarget != null) {
+ return uploadSnapshotTarget.getSnapshotSize();
+ } else {
+ return 0L;
+ }
+ }
+
+ public long getStandbySnapshotSize() {
+ return standbySnapshotSize;
+ }
+
+ public File getTabletDir() {
+ return tabletDir;
+ }
+
+ public Optional getLatestSnapshot() {
+ try {
+ return Optional.ofNullable(
+ snapshotContext.getLatestCompletedSnapshotProvider().apply(tableBucket));
+ } catch (Exception e) {
+ LOG.warn("Get latest completed snapshot for {} failed.", tableBucket, e);
+ }
+ return Optional.empty();
+ }
+
+ // schedule thread and asyncOperationsThreadPool can access this method
+ private synchronized void scheduleNextSnapshot(long delay, Executor guardedExecutor) {
+ ScheduledExecutorService snapshotScheduler = snapshotContext.getSnapshotScheduler();
+ if (isLeader && !snapshotScheduler.isShutdown()) {
+ LOG.debug(
+ "TableBucket {} schedules the next snapshot in {} seconds",
+ tableBucket,
+ delay / 1000);
+ scheduledTask =
+ snapshotScheduler.schedule(
+ () -> triggerUploadSnapshot(guardedExecutor),
+ delay,
+ TimeUnit.MILLISECONDS);
+ }
+ }
+
+ @VisibleForTesting
+ public long currentSnapshotId() {
+ return uploadSnapshotTarget.currentSnapshotId();
+ }
+
+ /** Trigger upload local snapshot to remote storage. */
+ public void triggerUploadSnapshot(Executor guardedExecutor) {
+ // todo: consider shrink the scope
+ // of using guardedExecutor
+ guardedExecutor.execute(
+ () -> {
+ if (isLeader) {
+ LOG.debug("TableBucket {} triggers snapshot.", tableBucket);
+ long triggerTime = System.currentTimeMillis();
+
+ Optional snapshotRunnableOptional;
+ try {
+ checkNotNull(uploadSnapshotTarget);
+ snapshotRunnableOptional = uploadSnapshotTarget.initSnapshot();
+ } catch (Exception e) {
+ LOG.error("Fail to init snapshot during triggering snapshot.", e);
+ return;
+ }
+ if (snapshotRunnableOptional.isPresent()) {
+ SnapshotRunnable runnable = snapshotRunnableOptional.get();
+ snapshotContext
+ .getAsyncOperationsThreadPool()
+ .execute(
+ () ->
+ asyncUploadSnapshotPhase(
+ triggerTime,
+ runnable.getSnapshotId(),
+ runnable.getCoordinatorEpoch(),
+ runnable.getBucketLeaderEpoch(),
+ runnable.getSnapshotLocation(),
+ runnable.getSnapshotRunnable(),
+ guardedExecutor));
+ } else {
+ scheduleNextSnapshot(guardedExecutor);
+ LOG.debug(
+ "TableBucket {} has no data updates since last snapshot, "
+ + "skip this one and schedule the next one in {} seconds",
+ tableBucket,
+ snapshotIntervalSupplier.getAsLong() / 1000);
+ }
+ }
+ });
+ }
+
+ private void asyncUploadSnapshotPhase(
+ long triggerTime,
+ long snapshotId,
+ int coordinatorEpoch,
+ int bucketLeaderEpoch,
+ SnapshotLocation snapshotLocation,
+ RunnableFuture snapshotedRunnableFuture,
+ Executor guardedExecutor) {
+ uploadSnapshot(snapshotedRunnableFuture)
+ .whenComplete(
+ (snapshotResult, throwable) -> {
+ // if succeed
+ if (throwable == null) {
+ numberOfConsecutiveFailures.set(0);
+
+ try {
+ checkNotNull(uploadSnapshotTarget);
+ uploadSnapshotTarget.handleSnapshotResult(
+ snapshotId,
+ coordinatorEpoch,
+ bucketLeaderEpoch,
+ snapshotLocation,
+ snapshotResult);
+ LOG.info(
+ "TableBucket {} snapshot {} finished successfully, cost {} ms.",
+ tableBucket,
+ snapshotId,
+ System.currentTimeMillis() - triggerTime);
+ } catch (Throwable t) {
+ LOG.warn(
+ "Fail to handle snapshot result during snapshot of TableBucket {}",
+ tableBucket,
+ t);
+ }
+ scheduleNextSnapshot(guardedExecutor);
+ } else {
+ // if failed
+ notifyFailureOrCancellation(
+ snapshotId, snapshotLocation, throwable);
+ int retryTime = numberOfConsecutiveFailures.incrementAndGet();
+ LOG.info(
+ "TableBucket {} asynchronous part of snapshot is not completed for the {} time.",
+ tableBucket,
+ retryTime,
+ throwable);
+
+ scheduleNextSnapshot(guardedExecutor);
+ }
+ });
+ }
+
+ private void notifyFailureOrCancellation(
+ long snapshot, SnapshotLocation snapshotLocation, Throwable cause) {
+ LOG.warn("TableBucket {} snapshot {} failed.", tableBucket, snapshot, cause);
+ checkNotNull(uploadSnapshotTarget);
+ uploadSnapshotTarget.handleSnapshotFailure(snapshot, snapshotLocation, cause);
+ }
+
+ private CompletableFuture uploadSnapshot(
+ RunnableFuture snapshotedRunnableFuture) {
+
+ FileSystemSafetyNet.initializeSafetyNetForThread();
+ CompletableFuture result = new CompletableFuture<>();
+ try {
+ FutureUtils.runIfNotDoneAndGet(snapshotedRunnableFuture);
+
+ LOG.debug("TableBucket {} finishes asynchronous part of snapshot.", tableBucket);
+
+ result.complete(snapshotedRunnableFuture.get());
+ } catch (Exception e) {
+ result.completeExceptionally(e);
+ discardFailedUploads(snapshotedRunnableFuture);
+ } finally {
+ FileSystemSafetyNet.closeSafetyNetAndGuardedResourcesForThread();
+ }
+
+ return result;
+ }
+
+ private void discardFailedUploads(RunnableFuture snapshotedRunnableFuture) {
+ LOG.info("TableBucket {} cleanup asynchronous runnable for snapshot.", tableBucket);
+
+ if (snapshotedRunnableFuture != null) {
+ // snapshot has started
+ if (!snapshotedRunnableFuture.cancel(true)) {
+ try {
+ SnapshotResult snapshotResult = snapshotedRunnableFuture.get();
+ if (snapshotResult != null) {
+ snapshotResult.getKvSnapshotHandle().discard();
+ FsPath remoteSnapshotPath = snapshotResult.getSnapshotPath();
+ remoteSnapshotPath.getFileSystem().delete(remoteSnapshotPath, true);
+ }
+ } catch (Exception ex) {
+ LOG.debug(
+ "TableBucket {} cancelled execution of snapshot future runnable. Cancellation produced the following exception, which is expected and can be ignored.",
+ tableBucket,
+ ex);
+ }
+ }
+ }
+ }
+
+ private void scheduleNextSnapshot(Executor guardedExecutor) {
+ scheduleNextSnapshot(snapshotIntervalSupplier.getAsLong(), guardedExecutor);
+ }
+
+ private void loadKvLocalFiles(Set downloadedSstFiles, Set downloadedMiscFiles)
+ throws Exception {
+ if (tabletDir.exists()) {
+ Path kvDbPath = tabletDir.toPath().resolve(RocksDBKvBuilder.DB_INSTANCE_DIR_STRING);
+ Path[] files = FileUtils.listDirectory(kvDbPath);
+ for (Path filePath : files) {
+ final String fileName = filePath.getFileName().toString();
+ if (fileName.endsWith(SST_FILE_SUFFIX)) {
+ downloadedSstFiles.add(filePath);
+ } else {
+ downloadedMiscFiles.add(filePath);
+ }
+ }
+ LOG.info(
+ "Loaded {} local sst files and {} misc files from {} for bucket {}.",
+ downloadedSstFiles.size(),
+ downloadedMiscFiles.size(),
+ kvDbPath,
+ tableBucket);
+ } else {
+ LOG.info("No local tablet directory found for bucket {}, starting fresh.", tableBucket);
+ }
+ }
+
+ private KvSnapshotHandle getIncrementalKvSnapshotHandle(
+ CompletedSnapshot completedSnapshot,
+ Set downloadedSstFiles,
+ Set sstFilesToDelete) {
+ // get downloaded sst files name to path.
+ Map downloadedSstFilesMap = new HashMap<>();
+ for (Path sstPath : downloadedSstFiles) {
+ downloadedSstFilesMap.put(sstPath.getFileName().toString(), sstPath);
+ }
+
+ KvSnapshotHandle completedSnapshotHandler = completedSnapshot.getKvSnapshotHandle();
+ List sstFileHandles =
+ completedSnapshotHandler.getSharedKvFileHandles();
+ List privateFileHandles =
+ completedSnapshotHandler.getPrivateFileHandles();
+
+ List incrementalSstFileHandles = new ArrayList<>();
+ Set downloadedSstFileNames = downloadedSstFilesMap.keySet();
+ for (KvFileHandleAndLocalPath sstFileHandle : sstFileHandles) {
+ String remoteName = sstFileHandle.getLocalPath();
+ if (!downloadedSstFileNames.contains(remoteName)) {
+ // File doesn't exist locally, needs download.
+ incrementalSstFileHandles.add(sstFileHandle);
+ } else {
+ // File exists locally with the same name. Verify the size matches to detect
+ // files from different RocksDB instances that happen to share the same file
+ // number (e.g., after leader re-election to a different server).
+ Path localPath = downloadedSstFilesMap.get(remoteName);
+ long remoteSize = sstFileHandle.getKvFileHandle().getSize();
+ try {
+ long localSize = Files.size(localPath);
+ if (localSize != remoteSize) {
+ LOG.info(
+ "Local SST file {} has size {} but remote expects size {}, "
+ + "will re-download for bucket {}",
+ remoteName,
+ localSize,
+ remoteSize,
+ tableBucket);
+ incrementalSstFileHandles.add(sstFileHandle);
+ sstFilesToDelete.add(localPath);
+ }
+ } catch (IOException e) {
+ LOG.warn(
+ "Failed to read size of local SST file {}, will re-download for bucket {}",
+ localPath,
+ tableBucket,
+ e);
+ incrementalSstFileHandles.add(sstFileHandle);
+ sstFilesToDelete.add(localPath);
+ }
+ }
+ }
+
+ Set newSstFileNames =
+ completedSnapshotHandler.getSharedKvFileHandles().stream()
+ .map(KvFileHandleAndLocalPath::getLocalPath)
+ .collect(Collectors.toSet());
+ for (String sstFileName : downloadedSstFileNames) {
+ if (!newSstFileNames.contains(sstFileName)) {
+ sstFilesToDelete.add(downloadedSstFilesMap.get(sstFileName));
+ }
+ }
+
+ long totalSnapshotSize =
+ sstFileHandles.stream().mapToLong(f -> f.getKvFileHandle().getSize()).sum()
+ + privateFileHandles.stream()
+ .mapToLong(f -> f.getKvFileHandle().getSize())
+ .sum();
+ long incrementalSnapshotSize =
+ incrementalSstFileHandles.stream()
+ .mapToLong(f -> f.getKvFileHandle().getSize())
+ .sum()
+ + privateFileHandles.stream()
+ .mapToLong(f -> f.getKvFileHandle().getSize())
+ .sum();
+ int reusedSstFiles = sstFileHandles.size() - incrementalSstFileHandles.size();
+ LOG.info(
+ "Incremental snapshot for bucket {}: remote has {} sst files (total size: {}), "
+ + "{} reused locally, {} to download, {} to delete, "
+ + "incremental download size: {}",
+ tableBucket,
+ sstFileHandles.size(),
+ totalSnapshotSize,
+ reusedSstFiles,
+ incrementalSstFileHandles.size(),
+ sstFilesToDelete.size(),
+ incrementalSnapshotSize);
+ return new KvSnapshotHandle(
+ incrementalSstFileHandles, privateFileHandles, incrementalSnapshotSize);
+ }
+
+ /**
+ * Atomically download snapshot files to ensure consistency.
+ *
+ * This method implements atomic snapshot download by:
+ *
+ *
+ * - Downloading all files to a temporary directory
+ *
- Verifying the download completeness
+ *
- Deleting obsolete files from the final directory
+ *
- Atomically moving new files from temp to final directory
+ *
- Cleaning up the temp directory
+ *
+ *
+ * If any step fails before step 3 (deleting obsolete files), the final directory remains in
+ * its original consistent state. Note that if a crash occurs between step 3 (deleting obsolete
+ * files) and step 4 (moving new files), the final directory may be in an incomplete state. This
+ * is acceptable for standby replicas because they can re-download the full snapshot on restart.
+ *
+ * @param incrementalSnapshot the incremental snapshot to download
+ * @param sstFilesToDelete SST files that should be deleted from the final directory
+ * @throws IOException if download or file operations fail
+ */
+ private void atomicDownloadSnapshot(
+ CompletedSnapshot incrementalSnapshot, Set sstFilesToDelete) throws IOException {
+ Path kvTabletDir = tabletDir.toPath();
+ Path kvDbPath = kvTabletDir.resolve(RocksDBKvBuilder.DB_INSTANCE_DIR_STRING);
+ Path tempDownloadDir =
+ kvTabletDir.resolve(".tmp_snapshot_" + incrementalSnapshot.getSnapshotID());
+
+ boolean downloadSuccessful = false;
+ CloseableRegistry closeableRegistry = new CloseableRegistry();
+ try {
+ // Step 1: Create temporary download directory
+ Files.createDirectories(tempDownloadDir);
+ LOG.debug(
+ "Created temporary snapshot download directory {} for bucket {}",
+ tempDownloadDir,
+ tableBucket);
+
+ // Step 2: Download all snapshot files to temporary directory
+ KvSnapshotDownloadSpec downloadSpec =
+ new KvSnapshotDownloadSpec(
+ incrementalSnapshot.getKvSnapshotHandle(), tempDownloadDir);
+ long start = clock.milliseconds();
+ LOG.info(
+ "Start to download kv snapshot {} to temporary directory {}.",
+ incrementalSnapshot,
+ tempDownloadDir);
+
+ KvSnapshotDataDownloader kvSnapshotDataDownloader =
+ snapshotContext.getSnapshotDataDownloader();
+ try {
+ kvSnapshotDataDownloader.transferAllDataToDirectory(
+ downloadSpec, closeableRegistry);
+ } catch (Exception e) {
+ if (e.getMessage()
+ .contains(CompletedSnapshot.SNAPSHOT_DATA_NOT_EXISTS_ERROR_MESSAGE)) {
+ try {
+ snapshotContext.handleSnapshotBroken(incrementalSnapshot);
+ } catch (Exception t) {
+ LOG.error("Handle broken snapshot {} failed.", incrementalSnapshot, t);
+ }
+ }
+ throw new IOException("Fail to download kv snapshot to temporary directory.", e);
+ }
+
+ long downloadTime = clock.milliseconds() - start;
+ LOG.info(
+ "Downloaded incremental kv snapshot {} to temp directory in {} ms for bucket {}.",
+ incrementalSnapshot.getSnapshotID(),
+ downloadTime,
+ tableBucket);
+
+ // Step 3: Verify download completeness
+ verifySnapshotCompleteness(incrementalSnapshot, tempDownloadDir);
+
+ downloadSuccessful = true;
+
+ // Step 4: Delete obsolete SST files from final directory
+ for (Path sstFileToDelete : sstFilesToDelete) {
+ try {
+ FileUtils.deleteFileOrDirectory(sstFileToDelete.toFile());
+ LOG.debug(
+ "Deleted obsolete SST file {} for bucket {}",
+ sstFileToDelete,
+ tableBucket);
+ } catch (IOException e) {
+ LOG.warn(
+ "Failed to delete obsolete SST file {} for bucket {}",
+ sstFileToDelete,
+ tableBucket,
+ e);
+ // Continue deletion even if one file fails
+ }
+ }
+
+ // Step 5: Delete obsolete misc files from final directory
+ // Use local reference to avoid NPE if clearStandbyDownloadCache() runs concurrently
+ Set miscFilesToClean = downloadedMiscFiles;
+ if (miscFilesToClean != null) {
+ for (Path miscFileToDelete : miscFilesToClean) {
+ try {
+ FileUtils.deleteFileOrDirectory(miscFileToDelete.toFile());
+ LOG.debug(
+ "Deleted obsolete misc file {} for bucket {}",
+ miscFileToDelete,
+ tableBucket);
+ } catch (IOException e) {
+ LOG.warn(
+ "Failed to delete obsolete misc file {} for bucket {}",
+ miscFileToDelete,
+ tableBucket,
+ e);
+ // Continue deletion even if one file fails
+ }
+ }
+ }
+
+ // Step 6: Atomically move downloaded files from temp to final directory
+ moveSnapshotFilesToFinalDirectory(tempDownloadDir, kvDbPath);
+
+ long totalTime = clock.milliseconds() - start;
+ LOG.info(
+ "Applied kv snapshot {} for bucket {} in {} ms (download: {} ms, verify+move: {} ms).",
+ incrementalSnapshot.getSnapshotID(),
+ tableBucket,
+ totalTime,
+ downloadTime,
+ totalTime - downloadTime);
+
+ } finally {
+ // Step 7: Clean up closeable registry
+ IOUtils.closeQuietly(closeableRegistry);
+
+ // Step 8: Clean up temporary directory
+ if (tempDownloadDir.toFile().exists()) {
+ try {
+ FileUtils.deleteDirectory(tempDownloadDir.toFile());
+ if (downloadSuccessful) {
+ LOG.debug(
+ "Cleaned up temporary snapshot directory {} for bucket {}",
+ tempDownloadDir,
+ tableBucket);
+ } else {
+ LOG.warn(
+ "Cleaned up temporary snapshot directory {} after failed download for bucket {}",
+ tempDownloadDir,
+ tableBucket);
+ }
+ } catch (IOException e) {
+ LOG.warn(
+ "Failed to clean up temporary snapshot directory {} for bucket {}",
+ tempDownloadDir,
+ tableBucket,
+ e);
+ }
+ }
+ }
+ }
+
+ /**
+ * Verify that all expected snapshot files have been downloaded to the temporary directory.
+ *
+ * @param snapshot the snapshot being verified
+ * @param tempDir the temporary directory containing downloaded files
+ * @throws IOException if verification fails
+ */
+ private void verifySnapshotCompleteness(CompletedSnapshot snapshot, Path tempDir)
+ throws IOException {
+ KvSnapshotHandle handle = snapshot.getKvSnapshotHandle();
+ List allFiles = new ArrayList<>();
+ allFiles.addAll(handle.getSharedKvFileHandles());
+ allFiles.addAll(handle.getPrivateFileHandles());
+
+ for (KvFileHandleAndLocalPath fileHandle : allFiles) {
+ String fileName = Paths.get(fileHandle.getLocalPath()).getFileName().toString();
+ Path expectedFile = tempDir.resolve(fileName);
+
+ if (!Files.exists(expectedFile)) {
+ throw new IOException(
+ String.format(
+ "Snapshot verification failed for bucket %s: expected file %s not found in temp directory %s",
+ tableBucket, fileName, tempDir));
+ }
+
+ long expectedSize = fileHandle.getKvFileHandle().getSize();
+ long actualSize = Files.size(expectedFile);
+ if (expectedSize != actualSize) {
+ throw new IOException(
+ String.format(
+ "Snapshot verification failed for bucket %s: file %s size mismatch (expected: %d, actual: %d)",
+ tableBucket, fileName, expectedSize, actualSize));
+ }
+ }
+
+ LOG.info(
+ "Verified completeness of snapshot {} for bucket {}: {} files, total size {} bytes",
+ snapshot.getSnapshotID(),
+ tableBucket,
+ allFiles.size(),
+ allFiles.stream().mapToLong(f -> f.getKvFileHandle().getSize()).sum());
+ }
+
+ /**
+ * Move snapshot files from temporary directory to final RocksDB directory.
+ *
+ * This method attempts atomic moves when possible (same filesystem), falling back to
+ * copy-then-delete if atomic move is not supported.
+ *
+ * @param tempDir the temporary directory containing downloaded files
+ * @param finalDir the final RocksDB db directory
+ * @throws IOException if file move operations fail
+ */
+ private void moveSnapshotFilesToFinalDirectory(Path tempDir, Path finalDir) throws IOException {
+ File[] files = tempDir.toFile().listFiles();
+ if (files == null || files.length == 0) {
+ LOG.debug(
+ "No files to move from temp directory {} to final directory {} for bucket {}",
+ tempDir,
+ finalDir,
+ tableBucket);
+ return;
+ }
+
+ int movedCount = 0;
+ for (File file : files) {
+ Path sourcePath = file.toPath();
+ Path targetPath = finalDir.resolve(file.getName());
+
+ try {
+ // Try atomic move first (rename on same filesystem)
+ Files.move(
+ sourcePath,
+ targetPath,
+ StandardCopyOption.REPLACE_EXISTING,
+ StandardCopyOption.ATOMIC_MOVE);
+ movedCount++;
+ LOG.debug(
+ "Atomically moved file {} to {} for bucket {}",
+ sourcePath.getFileName(),
+ targetPath,
+ tableBucket);
+ } catch (AtomicMoveNotSupportedException e) {
+ // Fallback to copy + delete if atomic move not supported
+ LOG.debug(
+ "Atomic move not supported for {}, using copy+delete",
+ sourcePath.getFileName());
+ Files.copy(sourcePath, targetPath, StandardCopyOption.REPLACE_EXISTING);
+ Files.delete(sourcePath);
+ movedCount++;
+ } catch (IOException e) {
+ throw new IOException(
+ String.format(
+ "Failed to move file %s to final directory %s for bucket %s",
+ sourcePath.getFileName(), finalDir, tableBucket),
+ e);
+ }
+ }
+
+ LOG.debug(
+ "Moved {} files from temp directory {} to final directory {} for bucket {}",
+ movedCount,
+ tempDir,
+ finalDir,
+ tableBucket);
+ }
+
+ /** {@link SnapshotRunnable} provider and consumer. */
+ @NotThreadSafe
+ public interface UploadSnapshotTarget {
+
+ /** Gets current snapshot id. */
+ long currentSnapshotId();
+
+ /**
+ * Initialize kv snapshot.
+ *
+ * @return a tuple of - future snapshot result from the underlying KV.
+ */
+ Optional initSnapshot() throws Exception;
+
+ /**
+ * Implementations should not trigger snapshot until the previous one has been confirmed or
+ * failed.
+ *
+ * @param snapshotId the snapshot id
+ * @param coordinatorEpoch the coordinator epoch
+ * @param bucketLeaderEpoch the leader epoch of the bucket when the snapshot is triggered
+ * @param snapshotLocation the location where the snapshot files stores
+ * @param snapshotResult the snapshot result
+ */
+ void handleSnapshotResult(
+ long snapshotId,
+ int coordinatorEpoch,
+ int bucketLeaderEpoch,
+ SnapshotLocation snapshotLocation,
+ SnapshotResult snapshotResult)
+ throws Throwable;
+
+ /** Called when the snapshot is fail. */
+ void handleSnapshotFailure(
+ long snapshotId, SnapshotLocation snapshotLocation, Throwable cause);
+
+ /** Get the total size of the snapshot. */
+ long getSnapshotSize();
+ }
+
+ @Override
+ public void close() {
+ synchronized (this) {
+ // do-nothing, please make the periodicExecutor will be closed by external
+ isLeader = false;
+ isStandby = false;
+ // cancel the scheduled task if not completed yet
+ if (scheduledTask != null && !scheduledTask.isDone()) {
+ scheduledTask.cancel(true);
+ }
+ }
+ }
+
+ /** A {@link Runnable} representing the snapshot and the associated metadata. */
+ public static class SnapshotRunnable {
+ private final RunnableFuture snapshotRunnable;
+
+ private final long snapshotId;
+ private final int coordinatorEpoch;
+ private final int bucketLeaderEpoch;
+ private final SnapshotLocation snapshotLocation;
+
+ public SnapshotRunnable(
+ RunnableFuture snapshotRunnable,
+ long snapshotId,
+ int coordinatorEpoch,
+ int bucketLeaderEpoch,
+ SnapshotLocation snapshotLocation) {
+ this.snapshotRunnable = snapshotRunnable;
+ this.snapshotId = snapshotId;
+ this.coordinatorEpoch = coordinatorEpoch;
+ this.bucketLeaderEpoch = bucketLeaderEpoch;
+ this.snapshotLocation = snapshotLocation;
+ }
+
+ RunnableFuture getSnapshotRunnable() {
+ return snapshotRunnable;
+ }
+
+ public long getSnapshotId() {
+ return snapshotId;
+ }
+
+ public SnapshotLocation getSnapshotLocation() {
+ return snapshotLocation;
+ }
+
+ public int getCoordinatorEpoch() {
+ return coordinatorEpoch;
+ }
+
+ public int getBucketLeaderEpoch() {
+ return bucketLeaderEpoch;
+ }
+ }
+}
diff --git a/fluss-server/src/main/java/org/apache/fluss/server/kv/snapshot/KvTabletSnapshotTarget.java b/fluss-server/src/main/java/org/apache/fluss/server/kv/snapshot/KvTabletUploadSnapshotTarget.java
similarity index 96%
rename from fluss-server/src/main/java/org/apache/fluss/server/kv/snapshot/KvTabletSnapshotTarget.java
rename to fluss-server/src/main/java/org/apache/fluss/server/kv/snapshot/KvTabletUploadSnapshotTarget.java
index e3e8ede435..6fe108cda8 100644
--- a/fluss-server/src/main/java/org/apache/fluss/server/kv/snapshot/KvTabletSnapshotTarget.java
+++ b/fluss-server/src/main/java/org/apache/fluss/server/kv/snapshot/KvTabletUploadSnapshotTarget.java
@@ -43,7 +43,7 @@
import java.util.function.Supplier;
/**
- * A {@link PeriodicSnapshotManager.SnapshotTarget} for a kv tablet. It'll first initiate a
+ * A {@link KvSnapshotManager.UploadSnapshotTarget} for a kv tablet. It'll first initiate a
* snapshot, then handle the snapshot result or failure.
*
* Note: it's not thread safe, {@link #initSnapshot()}}, {@link #handleSnapshotResult(long, int,
@@ -51,9 +51,9 @@
* Throwable)} may be called by different threads.
*/
@NotThreadSafe
-public class KvTabletSnapshotTarget implements PeriodicSnapshotManager.SnapshotTarget {
+public class KvTabletUploadSnapshotTarget implements KvSnapshotManager.UploadSnapshotTarget {
- private static final Logger LOG = LoggerFactory.getLogger(KvTabletSnapshotTarget.class);
+ private static final Logger LOG = LoggerFactory.getLogger(KvTabletUploadSnapshotTarget.class);
private final TableBucket tableBucket;
@@ -85,7 +85,7 @@ public class KvTabletSnapshotTarget implements PeriodicSnapshotManager.SnapshotT
private volatile long snapshotSize;
@VisibleForTesting
- KvTabletSnapshotTarget(
+ KvTabletUploadSnapshotTarget(
TableBucket tableBucket,
CompletedKvSnapshotCommitter completedKvSnapshotCommitter,
ZooKeeperClient zooKeeperClient,
@@ -119,7 +119,7 @@ public class KvTabletSnapshotTarget implements PeriodicSnapshotManager.SnapshotT
snapshotSize);
}
- public KvTabletSnapshotTarget(
+ public KvTabletUploadSnapshotTarget(
TableBucket tableBucket,
CompletedKvSnapshotCommitter completedKvSnapshotCommitter,
@Nonnull ZooKeeperClient zooKeeperClient,
@@ -167,7 +167,7 @@ public long currentSnapshotId() {
}
@Override
- public Optional initSnapshot() throws Exception {
+ public Optional initSnapshot() throws Exception {
TabletState tabletState = tabletStateSupplier.get();
long logOffset = tabletState.getFlushedLogOffset();
if (logOffset <= logOffsetOfLatestSnapshot) {
@@ -186,8 +186,8 @@ public Optional initSnapshot() throws
int coordinatorEpoch = coordinatorEpochSupplier.get();
SnapshotLocation snapshotLocation = initSnapshotLocation(currentSnapshotId);
try {
- PeriodicSnapshotManager.SnapshotRunnable snapshotRunnable =
- new PeriodicSnapshotManager.SnapshotRunnable(
+ KvSnapshotManager.SnapshotRunnable snapshotRunnable =
+ new KvSnapshotManager.SnapshotRunnable(
snapshotRunner.snapshot(
currentSnapshotId, tabletState, snapshotLocation),
currentSnapshotId,
diff --git a/fluss-server/src/main/java/org/apache/fluss/server/kv/snapshot/PeriodicSnapshotManager.java b/fluss-server/src/main/java/org/apache/fluss/server/kv/snapshot/PeriodicSnapshotManager.java
deleted file mode 100644
index 566d6ab107..0000000000
--- a/fluss-server/src/main/java/org/apache/fluss/server/kv/snapshot/PeriodicSnapshotManager.java
+++ /dev/null
@@ -1,439 +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.fluss.server.kv.snapshot;
-
-import org.apache.fluss.annotation.VisibleForTesting;
-import org.apache.fluss.fs.FileSystemSafetyNet;
-import org.apache.fluss.fs.FsPath;
-import org.apache.fluss.metadata.TableBucket;
-import org.apache.fluss.utils.MathUtils;
-import org.apache.fluss.utils.concurrent.Executors;
-import org.apache.fluss.utils.concurrent.FutureUtils;
-
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-import javax.annotation.concurrent.NotThreadSafe;
-
-import java.io.Closeable;
-import java.util.Optional;
-import java.util.concurrent.CompletableFuture;
-import java.util.concurrent.Executor;
-import java.util.concurrent.ExecutorService;
-import java.util.concurrent.RunnableFuture;
-import java.util.concurrent.ScheduledExecutorService;
-import java.util.concurrent.ScheduledFuture;
-import java.util.concurrent.TimeUnit;
-import java.util.concurrent.atomic.AtomicInteger;
-import java.util.function.LongSupplier;
-
-/* This file is based on source code of Apache Flink Project (https://flink.apache.org/), licensed by the Apache
- * Software Foundation (ASF) under the Apache License, Version 2.0. See the NOTICE file distributed with this work for
- * additional information regarding copyright ownership. */
-
-/**
- * Stateless snapshot manager which will trigger snapshot periodically. It'll use a {@link
- * ScheduledExecutorService} to schedule the snapshot initialization and a {@link ExecutorService}
- * to complete async phase of snapshot.
- */
-public class PeriodicSnapshotManager implements Closeable {
-
- private static final Logger LOG = LoggerFactory.getLogger(PeriodicSnapshotManager.class);
-
- /**
- * An executor that uses to trigger snapshot.
- *
- * It's expected to be passed with a guarded executor to prevent any concurrent modification
- * to KvTablet during trigger snapshotting.
- */
- private final Executor guardedExecutor;
-
- /** scheduled executor, periodically trigger snapshot. */
- private final ScheduledExecutorService periodicExecutor;
-
- /** Async thread pool, to complete async phase of snapshot. */
- private final ExecutorService asyncOperationsThreadPool;
-
- /**
- * A supplier to get the current snapshot delay. This allows dynamic reconfiguration of the
- * snapshot interval at runtime.
- */
- private final LongSupplier snapshotIntervalSupplier;
-
- /** Number of consecutive snapshot failures. */
- private final AtomicInteger numberOfConsecutiveFailures;
-
- /** The target on which the snapshot will be done. */
- private final SnapshotTarget target;
-
- /** Whether snapshot is started. */
- private volatile boolean started = false;
-
- /**
- * The scheduled snapshot task.
- *
- *
Since all reads and writes of {@code scheduledTask} are protected by synchronized, the
- * volatile modifier is not necessary here.
- */
- private ScheduledFuture> scheduledTask = null;
-
- private final long initialDelay;
- /** The table bucket that the snapshot manager is for. */
- private final TableBucket tableBucket;
-
- @VisibleForTesting
- protected PeriodicSnapshotManager(
- TableBucket tableBucket,
- SnapshotTarget target,
- long periodicSnapshotDelay,
- ExecutorService asyncOperationsThreadPool,
- ScheduledExecutorService periodicExecutor) {
- this(
- tableBucket,
- target,
- () -> periodicSnapshotDelay,
- asyncOperationsThreadPool,
- periodicExecutor,
- Executors.directExecutor());
- }
-
- @VisibleForTesting
- protected PeriodicSnapshotManager(
- TableBucket tableBucket,
- SnapshotTarget target,
- LongSupplier snapshotIntervalSupplier,
- ExecutorService asyncOperationsThreadPool,
- ScheduledExecutorService periodicExecutor) {
- this(
- tableBucket,
- target,
- snapshotIntervalSupplier,
- asyncOperationsThreadPool,
- periodicExecutor,
- Executors.directExecutor());
- }
-
- private PeriodicSnapshotManager(
- TableBucket tableBucket,
- SnapshotTarget target,
- LongSupplier snapshotIntervalSupplier,
- ExecutorService asyncOperationsThreadPool,
- ScheduledExecutorService periodicExecutor,
- Executor guardedExecutor) {
- this.tableBucket = tableBucket;
- this.target = target;
- this.snapshotIntervalSupplier = snapshotIntervalSupplier;
-
- this.numberOfConsecutiveFailures = new AtomicInteger(0);
- this.periodicExecutor = periodicExecutor;
- this.guardedExecutor = guardedExecutor;
- this.asyncOperationsThreadPool = asyncOperationsThreadPool;
- long periodicSnapshotDelay = snapshotIntervalSupplier.getAsLong();
- this.initialDelay =
- periodicSnapshotDelay > 0
- ? MathUtils.murmurHash(tableBucket.hashCode()) % periodicSnapshotDelay
- : 0;
- }
-
- public static PeriodicSnapshotManager create(
- TableBucket tableBucket,
- SnapshotTarget target,
- SnapshotContext snapshotContext,
- Executor guardedExecutor) {
- return new PeriodicSnapshotManager(
- tableBucket,
- target,
- snapshotContext::getSnapshotIntervalMs,
- snapshotContext.getAsyncOperationsThreadPool(),
- snapshotContext.getSnapshotScheduler(),
- guardedExecutor);
- }
-
- public void start() {
- // disable periodic snapshot when periodicMaterializeDelay is not positive
- if (!started && initialDelay > 0) {
-
- started = true;
-
- LOG.info("TableBucket {} starts periodic snapshot", tableBucket);
-
- scheduleNextSnapshot(initialDelay);
- }
- }
-
- public long getSnapshotSize() {
- return target.getSnapshotSize();
- }
-
- // schedule thread and asyncOperationsThreadPool can access this method
- private synchronized void scheduleNextSnapshot(long delay) {
- if (started && !periodicExecutor.isShutdown()) {
-
- LOG.debug(
- "TableBucket {} schedules the next snapshot in {} seconds",
- tableBucket,
- delay / 1000);
- scheduledTask =
- periodicExecutor.schedule(this::triggerSnapshot, delay, TimeUnit.MILLISECONDS);
- }
- }
-
- @VisibleForTesting
- public long currentSnapshotId() {
- return target.currentSnapshotId();
- }
-
- public void triggerSnapshot() {
- // todo: consider shrink the scope
- // of using guardedExecutor
- guardedExecutor.execute(
- () -> {
- if (started) {
- LOG.debug("TableBucket {} triggers snapshot.", tableBucket);
- long triggerTime = System.currentTimeMillis();
-
- Optional snapshotRunnableOptional;
- try {
- snapshotRunnableOptional = target.initSnapshot();
- } catch (Exception e) {
- LOG.error("Fail to init snapshot during triggering snapshot.", e);
- return;
- }
- if (snapshotRunnableOptional.isPresent()) {
- SnapshotRunnable runnable = snapshotRunnableOptional.get();
- asyncOperationsThreadPool.execute(
- () ->
- asyncSnapshotPhase(
- triggerTime,
- runnable.getSnapshotId(),
- runnable.getCoordinatorEpoch(),
- runnable.getBucketLeaderEpoch(),
- runnable.getSnapshotLocation(),
- runnable.getSnapshotRunnable()));
- } else {
- scheduleNextSnapshot();
- LOG.debug(
- "TableBucket {} has no data updates since last snapshot, "
- + "skip this one and schedule the next one in {} seconds",
- tableBucket,
- snapshotIntervalSupplier.getAsLong() / 1000);
- }
- }
- });
- }
-
- private void asyncSnapshotPhase(
- long triggerTime,
- long snapshotId,
- int coordinatorEpoch,
- int bucketLeaderEpoch,
- SnapshotLocation snapshotLocation,
- RunnableFuture snapshotedRunnableFuture) {
- uploadSnapshot(snapshotedRunnableFuture)
- .whenComplete(
- (snapshotResult, throwable) -> {
- // if succeed
- if (throwable == null) {
- numberOfConsecutiveFailures.set(0);
-
- try {
- target.handleSnapshotResult(
- snapshotId,
- coordinatorEpoch,
- bucketLeaderEpoch,
- snapshotLocation,
- snapshotResult);
- LOG.info(
- "TableBucket {} snapshot {} finished successfully, full size: {}, incremental size: {}, cost: {} ms.",
- tableBucket,
- snapshotId,
- snapshotResult.getSnapshotSize(),
- snapshotResult.getIncrementalSize(),
- System.currentTimeMillis() - triggerTime);
- } catch (Throwable t) {
- LOG.warn(
- "Fail to handle snapshot result during snapshot of TableBucket {}",
- tableBucket,
- t);
- }
- scheduleNextSnapshot();
- } else {
- // if failed
- notifyFailureOrCancellation(
- snapshotId, snapshotLocation, throwable);
- int retryTime = numberOfConsecutiveFailures.incrementAndGet();
- LOG.info(
- "TableBucket {} asynchronous part of snapshot is not completed for the {} time.",
- tableBucket,
- retryTime,
- throwable);
-
- scheduleNextSnapshot();
- }
- });
- }
-
- private void notifyFailureOrCancellation(
- long snapshot, SnapshotLocation snapshotLocation, Throwable cause) {
- LOG.warn("TableBucket {} snapshot {} failed.", tableBucket, snapshot, cause);
- target.handleSnapshotFailure(snapshot, snapshotLocation, cause);
- }
-
- private CompletableFuture uploadSnapshot(
- RunnableFuture snapshotedRunnableFuture) {
-
- FileSystemSafetyNet.initializeSafetyNetForThread();
- CompletableFuture result = new CompletableFuture<>();
- try {
- FutureUtils.runIfNotDoneAndGet(snapshotedRunnableFuture);
-
- LOG.debug("TableBucket {} finishes asynchronous part of snapshot.", tableBucket);
-
- result.complete(snapshotedRunnableFuture.get());
- } catch (Exception e) {
- result.completeExceptionally(e);
- discardFailedUploads(snapshotedRunnableFuture);
- } finally {
- FileSystemSafetyNet.closeSafetyNetAndGuardedResourcesForThread();
- }
-
- return result;
- }
-
- private void discardFailedUploads(RunnableFuture snapshotedRunnableFuture) {
- LOG.info("TableBucket {} cleanup asynchronous runnable for snapshot.", tableBucket);
-
- if (snapshotedRunnableFuture != null) {
- // snapshot has started
- if (!snapshotedRunnableFuture.cancel(true)) {
- try {
- SnapshotResult snapshotResult = snapshotedRunnableFuture.get();
- if (snapshotResult != null) {
- snapshotResult.getKvSnapshotHandle().discard();
- FsPath remoteSnapshotPath = snapshotResult.getSnapshotPath();
- remoteSnapshotPath.getFileSystem().delete(remoteSnapshotPath, true);
- }
- } catch (Exception ex) {
- LOG.debug(
- "TableBucket {} cancelled execution of snapshot future runnable. Cancellation produced the following exception, which is expected and can be ignored.",
- tableBucket,
- ex);
- }
- }
- }
- }
-
- private void scheduleNextSnapshot() {
- scheduleNextSnapshot(snapshotIntervalSupplier.getAsLong());
- }
-
- /** {@link SnapshotRunnable} provider and consumer. */
- @NotThreadSafe
- public interface SnapshotTarget {
-
- /** Gets current snapshot id. */
- long currentSnapshotId();
-
- /**
- * Initialize kv snapshot.
- *
- * @return a tuple of - future snapshot result from the underlying KV.
- */
- Optional initSnapshot() throws Exception;
-
- /**
- * Implementations should not trigger snapshot until the previous one has been confirmed or
- * failed.
- *
- * @param snapshotId the snapshot id
- * @param coordinatorEpoch the coordinator epoch
- * @param bucketLeaderEpoch the leader epoch of the bucket when the snapshot is triggered
- * @param snapshotLocation the location where the snapshot files stores
- * @param snapshotResult the snapshot result
- */
- void handleSnapshotResult(
- long snapshotId,
- int coordinatorEpoch,
- int bucketLeaderEpoch,
- SnapshotLocation snapshotLocation,
- SnapshotResult snapshotResult)
- throws Throwable;
-
- /** Called when the snapshot is fail. */
- void handleSnapshotFailure(
- long snapshotId, SnapshotLocation snapshotLocation, Throwable cause);
-
- /** Get the total size of the snapshot. */
- long getSnapshotSize();
- }
-
- @Override
- public void close() {
- synchronized (this) {
- // do-nothing, please make the periodicExecutor will be closed by external
- started = false;
- // cancel the scheduled task if not completed yet
- if (scheduledTask != null && !scheduledTask.isDone()) {
- scheduledTask.cancel(true);
- }
- }
- }
-
- /** A {@link Runnable} representing the snapshot and the associated metadata. */
- public static class SnapshotRunnable {
- private final RunnableFuture snapshotRunnable;
-
- private final long snapshotId;
- private final int coordinatorEpoch;
- private final int bucketLeaderEpoch;
- private final SnapshotLocation snapshotLocation;
-
- public SnapshotRunnable(
- RunnableFuture snapshotRunnable,
- long snapshotId,
- int coordinatorEpoch,
- int bucketLeaderEpoch,
- SnapshotLocation snapshotLocation) {
- this.snapshotRunnable = snapshotRunnable;
- this.snapshotId = snapshotId;
- this.coordinatorEpoch = coordinatorEpoch;
- this.bucketLeaderEpoch = bucketLeaderEpoch;
- this.snapshotLocation = snapshotLocation;
- }
-
- RunnableFuture getSnapshotRunnable() {
- return snapshotRunnable;
- }
-
- public long getSnapshotId() {
- return snapshotId;
- }
-
- public SnapshotLocation getSnapshotLocation() {
- return snapshotLocation;
- }
-
- public int getCoordinatorEpoch() {
- return coordinatorEpoch;
- }
-
- public int getBucketLeaderEpoch() {
- return bucketLeaderEpoch;
- }
- }
-}
diff --git a/fluss-server/src/main/java/org/apache/fluss/server/kv/snapshot/SnapshotContext.java b/fluss-server/src/main/java/org/apache/fluss/server/kv/snapshot/SnapshotContext.java
index 8f528bc473..9e0f47b0f6 100644
--- a/fluss-server/src/main/java/org/apache/fluss/server/kv/snapshot/SnapshotContext.java
+++ b/fluss-server/src/main/java/org/apache/fluss/server/kv/snapshot/SnapshotContext.java
@@ -64,6 +64,10 @@ public interface SnapshotContext {
FunctionWithException
getLatestCompletedSnapshotProvider();
+ /** Get the provider of a completed snapshot for a table bucket. */
+ CompletedSnapshot getCompletedSnapshotProvider(TableBucket tableBucket, long snapshotId)
+ throws Exception;
+
/**
* Handles broken snapshots.
*
diff --git a/fluss-server/src/main/java/org/apache/fluss/server/kv/snapshot/ZooKeeperCompletedSnapshotHandleStore.java b/fluss-server/src/main/java/org/apache/fluss/server/kv/snapshot/ZooKeeperCompletedSnapshotHandleStore.java
index 9753209ad6..5e45fd67a5 100644
--- a/fluss-server/src/main/java/org/apache/fluss/server/kv/snapshot/ZooKeeperCompletedSnapshotHandleStore.java
+++ b/fluss-server/src/main/java/org/apache/fluss/server/kv/snapshot/ZooKeeperCompletedSnapshotHandleStore.java
@@ -121,4 +121,11 @@ public Optional getLatestCompletedSnapshotHandle(
return client.getTableBucketLatestSnapshot(tableBucket)
.map(BucketSnapshot::toCompletedSnapshotHandle);
}
+
+ @Override
+ public Optional getCompletedSnapshotHandle(
+ TableBucket tableBucket, long snapshotId) throws Exception {
+ return client.getTableBucketSnapshot(tableBucket, snapshotId)
+ .map(BucketSnapshot::toCompletedSnapshotHandle);
+ }
}
diff --git a/fluss-server/src/main/java/org/apache/fluss/server/replica/IsrState.java b/fluss-server/src/main/java/org/apache/fluss/server/replica/IsrState.java
index aac5df385d..14051ed422 100644
--- a/fluss-server/src/main/java/org/apache/fluss/server/replica/IsrState.java
+++ b/fluss-server/src/main/java/org/apache/fluss/server/replica/IsrState.java
@@ -40,15 +40,19 @@ public interface IsrState {
*/
List maximalIsr();
+ List standbyReplicas();
+
/** Indicates if we have an AdjustIsr request inflight. */
boolean isInflight();
/** Class to represent the committed isr state of a {@link TableBucket}. */
class CommittedIsrState implements IsrState {
private final List isr;
+ private final List standbyReplicas;
- public CommittedIsrState(List isr) {
+ public CommittedIsrState(List isr, List standbyReplicas) {
this.isr = isr;
+ this.standbyReplicas = standbyReplicas;
}
@Override
@@ -56,6 +60,11 @@ public List isr() {
return isr;
}
+ @Override
+ public List standbyReplicas() {
+ return standbyReplicas;
+ }
+
@Override
public List maximalIsr() {
return isr;
@@ -135,6 +144,11 @@ public List isr() {
return lastCommittedState.isr();
}
+ @Override
+ public List standbyReplicas() {
+ return lastCommittedState.standbyReplicas();
+ }
+
@Override
public List maximalIsr() {
ArrayList maximalIsr = new ArrayList<>(lastCommittedState.isr());
@@ -205,6 +219,11 @@ public List isr() {
return lastCommittedState.isr();
}
+ @Override
+ public List standbyReplicas() {
+ return lastCommittedState.standbyReplicas();
+ }
+
@Override
public List maximalIsr() {
return lastCommittedState.isr();
diff --git a/fluss-server/src/main/java/org/apache/fluss/server/replica/Replica.java b/fluss-server/src/main/java/org/apache/fluss/server/replica/Replica.java
index dd83c58b1d..9015ae25a5 100644
--- a/fluss-server/src/main/java/org/apache/fluss/server/replica/Replica.java
+++ b/fluss-server/src/main/java/org/apache/fluss/server/replica/Replica.java
@@ -54,14 +54,11 @@
import org.apache.fluss.server.kv.KvRecoverHelper;
import org.apache.fluss.server.kv.KvTablet;
import org.apache.fluss.server.kv.autoinc.AutoIncIDRange;
-import org.apache.fluss.server.kv.rocksdb.RocksDBKvBuilder;
import org.apache.fluss.server.kv.snapshot.CompletedKvSnapshotCommitter;
import org.apache.fluss.server.kv.snapshot.CompletedSnapshot;
import org.apache.fluss.server.kv.snapshot.KvFileHandleAndLocalPath;
-import org.apache.fluss.server.kv.snapshot.KvSnapshotDataDownloader;
-import org.apache.fluss.server.kv.snapshot.KvSnapshotDownloadSpec;
-import org.apache.fluss.server.kv.snapshot.KvTabletSnapshotTarget;
-import org.apache.fluss.server.kv.snapshot.PeriodicSnapshotManager;
+import org.apache.fluss.server.kv.snapshot.KvSnapshotManager;
+import org.apache.fluss.server.kv.snapshot.KvTabletUploadSnapshotTarget;
import org.apache.fluss.server.kv.snapshot.RocksIncrementalSnapshot;
import org.apache.fluss.server.kv.snapshot.SnapshotContext;
import org.apache.fluss.server.log.FetchDataInfo;
@@ -91,6 +88,7 @@
import org.apache.fluss.server.zk.data.LeaderAndIsr;
import org.apache.fluss.server.zk.data.ZkData;
import org.apache.fluss.utils.CloseableRegistry;
+import org.apache.fluss.utils.FileUtils;
import org.apache.fluss.utils.FlussPaths;
import org.apache.fluss.utils.IOUtils;
import org.apache.fluss.utils.MapUtils;
@@ -190,15 +188,17 @@ public final class Replica {
private final Map followerReplicasMap =
MapUtils.newConcurrentHashMap();
- private volatile IsrState isrState = new IsrState.CommittedIsrState(Collections.emptyList());
+ private volatile IsrState isrState =
+ new IsrState.CommittedIsrState(Collections.emptyList(), Collections.emptyList());
private volatile int leaderEpoch = LeaderAndIsr.INITIAL_LEADER_EPOCH - 1;
private volatile int bucketEpoch = LeaderAndIsr.INITIAL_BUCKET_EPOCH;
private volatile int coordinatorEpoch = CoordinatorContext.INITIAL_COORDINATOR_EPOCH;
+ private volatile boolean isStandbyReplica = false;
// null if table without pk or haven't become leader
private volatile @Nullable KvTablet kvTablet;
private volatile @Nullable CloseableRegistry closeableRegistryForKv;
- private @Nullable PeriodicSnapshotManager kvSnapshotManager;
+ private @Nullable KvSnapshotManager kvSnapshotManager;
// ------- metrics
private Counter isrShrinks;
@@ -253,6 +253,18 @@ public Replica(
// create a closeable registry for the replica
this.closeableRegistry = new CloseableRegistry();
+ if (kvManager != null) {
+ // Create KvSnapshotManager for all replicas (leader/follower/standby).
+ // The tablet directory is always created for KV tables,
+ // even if the replica is a follower with no KV data.
+ this.kvSnapshotManager =
+ KvSnapshotManager.create(
+ tableBucket,
+ kvManager.getOrCreateTabletDir(physicalPath, tableBucket),
+ snapshotContext,
+ clock);
+ }
+
this.logTablet = createLog(lazyHighWatermarkCheckpoint);
this.logTablet.updateIsDataLakeEnabled(tableConfig.isDataLakeEnabled());
this.clock = clock;
@@ -402,7 +414,11 @@ public void makeLeader(NotifyLeaderAndIsrData data) throws IOException {
long currentTimeMs = clock.milliseconds();
// Updating the assignment and ISR state is safe if the bucket epoch is
// larger or equal to the current bucket epoch.
- updateAssignmentAndIsr(data.getReplicas(), true, data.getIsr());
+ updateAssignmentAndIsr(
+ data.getReplicas(),
+ true,
+ data.getIsr(),
+ data.getStandbyReplicas());
int requestLeaderEpoch = data.getLeaderEpoch();
if (requestLeaderEpoch > leaderEpoch) {
@@ -453,7 +469,11 @@ public boolean makeFollower(NotifyLeaderAndIsrData data) {
coordinatorEpoch = data.getCoordinatorEpoch();
- updateAssignmentAndIsr(Collections.emptyList(), false, Collections.emptyList());
+ updateAssignmentAndIsr(
+ Collections.emptyList(),
+ false,
+ Collections.emptyList(),
+ Collections.emptyList());
int requestLeaderEpoch = data.getLeaderEpoch();
boolean isNewLeaderEpoch = requestLeaderEpoch > leaderEpoch;
@@ -463,7 +483,10 @@ public boolean makeFollower(NotifyLeaderAndIsrData data) {
tableBucket,
requestLeaderEpoch,
logTablet.localLogEndOffset());
- onBecomeNewFollower();
+ // Currently, we only support one standby replica.
+ List standbyReplicas = data.getStandbyReplicas();
+ onBecomeNewFollower(
+ standbyReplicas.isEmpty() ? -1 : standbyReplicas.get(0));
} else if (requestLeaderEpoch == leaderEpoch) {
LOG.info(
"Skipped the become-follower state change for bucket {} since "
@@ -520,6 +543,13 @@ public LogOffsetSnapshot fetchOffsetSnapshot(boolean fetchOnlyFromLeader) throws
});
}
+ public void downloadSnapshot(long snapshotId) throws Exception {
+ if (isStandbyReplica) {
+ checkNotNull(kvSnapshotManager);
+ kvSnapshotManager.downloadSnapshot(snapshotId);
+ }
+ }
+
// -------------------------------------------------------------------------------------------
private void onBecomeNewLeader() {
@@ -530,11 +560,35 @@ private void onBecomeNewLeader() {
}
if (isKvTable()) {
- // if it's become new leader, we must
- // first destroy the old kv tablet
- // if exist. Otherwise, it'll use still the old kv tablet which will cause data loss
- dropKv();
- // now, we can create a new kv tablet
+ // Record previous role for logging before resetting flags.
+ String previousRole =
+ isStandbyReplica ? "standby" : (kvTablet != null ? "leader" : "follower");
+
+ // Reset standby flag first - the replica is now a leader, not a standby.
+ // This must be done before becomeLeader() to ensure consistent state.
+ isStandbyReplica = false;
+
+ // If kvTablet already exists (e.g., Leader -> Leader scenario),
+ // we need to close it first to release RocksDB lock before re-initializing.
+ // Set need drop as false to avoid deleting files.
+ if (kvTablet != null) {
+ LOG.info(
+ "Closing existing kvTablet before re-initializing as leader for bucket {}",
+ tableBucket);
+ kvManager.closeOrDropKv(tableBucket, false);
+ kvTablet = null;
+ }
+
+ LOG.info(
+ "Bucket {} transitioning from {} to leader, starting kv initialization.",
+ tableBucket,
+ previousRole);
+
+ // 1. If there is no sst files in local, download the latest kv snapshot and apply log.
+ // 2. If there is already sst files in local, check the diff with the latest snapshot
+ // and download the diff and delete the deleted sst files. And then apply log.
+ checkNotNull(kvSnapshotManager);
+ kvSnapshotManager.becomeLeader();
createKv();
}
}
@@ -555,11 +609,39 @@ private void registerLakeTieringMetrics() {
: logTablet.localMaxTimestamp() - logTablet.getLakeMaxTimestamp());
}
- private void onBecomeNewFollower() {
+ private void onBecomeNewFollower(int standbyReplica) {
if (isKvTable()) {
- // it should be from leader to follower, we need to destroy the kv tablet
- dropKv();
+ boolean isNowStandby = (standbyReplica == localTabletServerId);
+ boolean wasLeader = isLeader();
+ boolean wasStandby = this.isStandbyReplica;
+ String previousRole = wasLeader ? "leader" : (wasStandby ? "standby" : "follower");
+ String newRole = isNowStandby ? "standby" : "follower";
+
+ LOG.info("Bucket {} transitioning from {} to {}.", tableBucket, previousRole, newRole);
+
+ checkNotNull(kvSnapshotManager);
+ if (isNowStandby) {
+ kvSnapshotManager.becomeStandby();
+ // Mark as standby immediately to ensure coordinator's state is consistent.
+ // The snapshot download is done asynchronously to avoid blocking makeFollower.
+ isStandbyReplica = true;
+ becomeStandbyAsync();
+ } else {
+ // to be new follower.
+ kvSnapshotManager.becomeFollower();
+ if (wasStandby || wasLeader) {
+ // standby -> follower or leader -> follower
+ // Clear standby flag before dropKv() so logging is accurate
+ if (wasStandby) {
+ isStandbyReplica = false;
+ }
+ dropKv();
+ }
+
+ // follower -> follower: do nothing.
+ }
}
+
if (lakeTieringMetricGroup != null) {
lakeTieringMetricGroup.close();
}
@@ -617,6 +699,41 @@ public void updateTieredLogLocalSegments(int tieredLogLocalSegments) {
tieredLogLocalSegments);
}
+ /**
+ * Asynchronously download the latest snapshot for standby replica.
+ *
+ * This method submits the snapshot download task to an async thread pool to avoid blocking
+ * the makeFollower operation. The download can be retried later via
+ * NotifyKvSnapshotOffsetRequest if it fails.
+ */
+ private void becomeStandbyAsync() {
+ checkNotNull(snapshotContext);
+ snapshotContext
+ .getAsyncOperationsThreadPool()
+ .execute(
+ () -> {
+ try {
+ checkNotNull(kvSnapshotManager);
+ kvSnapshotManager.downloadLatestSnapshot();
+ LOG.info(
+ "TabletServer {} successfully downloaded snapshot and becomes standby for bucket {}",
+ localTabletServerId,
+ tableBucket);
+ } catch (Exception e) {
+ // The snapshot download can be retried via
+ // NotifyKvSnapshotOffsetRequest.
+ LOG.warn(
+ "Failed to download snapshot when becoming standby replica for bucket {}. Will retry later.",
+ tableBucket,
+ e);
+ }
+ });
+ LOG.info(
+ "TabletServer {} is becoming standby for bucket {}, snapshot download started asynchronously",
+ localTabletServerId,
+ tableBucket);
+ }
+
private void createKv() {
try {
// create a closeable registry for the closable related to kv
@@ -644,8 +761,8 @@ private void createKv() {
e);
}
}
- // start periodic kv snapshot
- startPeriodicKvSnapshot(snapshotUsed.orElse(null));
+ // start periodic upload kv snapshot
+ startPeriodicUploadKvSnapshot(snapshotUsed.orElse(null));
}
private void dropKv() {
@@ -660,8 +777,29 @@ private void dropKv() {
// drop the kv tablet
checkNotNull(kvManager);
- kvManager.dropKv(tableBucket);
+ kvManager.closeOrDropKv(tableBucket, true);
kvTablet = null;
+ } else {
+ // For standby replicas, kvTablet is null, so we need to manually delete the
+ // downloaded snapshot files and the tablet directory.
+ // For follower replicas, we need to delete the empty directory.
+ checkNotNull(kvSnapshotManager);
+ File tabletDir = kvSnapshotManager.getTabletDir();
+ try {
+ FileUtils.deleteDirectory(tabletDir);
+ LOG.info(
+ "Deleted {} tablet directory {} for table bucket {}",
+ isStandbyReplica ? "standby replica" : "follower",
+ tabletDir,
+ tableBucket);
+ } catch (IOException e) {
+ LOG.warn(
+ "Failed to delete {} tablet directory {} for table bucket {}",
+ isStandbyReplica ? "standby replica" : "follower",
+ tabletDir,
+ tableBucket,
+ e);
+ }
}
}
@@ -679,55 +817,32 @@ private void mayFlushKv(long newHighWatermark) {
*/
private Optional initKvTablet() {
checkNotNull(kvManager);
+ checkNotNull(kvSnapshotManager);
long startTime = clock.milliseconds();
LOG.info("Start to init kv tablet for {} of table {}.", tableBucket, physicalPath);
- // todo: we may need to handle the following cases:
- // case1: no kv files in local, restore from remote snapshot; and apply
- // the log;
- // case2: kv files in local
- // - if no remote snapshot, restore from local and apply the log known to the local
- // files.
- // - have snapshot, if the known offset to the local files is much less than(maybe
- // some value configured)
- // the remote snapshot; restore from remote snapshot;
-
- // currently for simplicity, we'll always download the snapshot files and restore from
- // the snapshots as kv files won't exist in our current implementation for
- // when replica become follower, we'll always delete the kv files.
-
- // get the offset from which, we should restore from. default is 0
- long restoreStartOffset = 0;
- Optional optCompletedSnapshot = getLatestSnapshot(tableBucket);
+ Optional optCompletedSnapshot;
try {
+ optCompletedSnapshot = kvSnapshotManager.downloadLatestSnapshot();
+ long restoreStartOffset = 0;
Long rowCount;
AutoIncIDRange autoIncIDRange;
if (optCompletedSnapshot.isPresent()) {
LOG.info(
- "Use snapshot {} to restore kv tablet for {} of table {}.",
- optCompletedSnapshot.get(),
+ "Init kv tablet for download latest snapshot {} of {} finish, cost {} ms.",
+ physicalPath,
tableBucket,
- physicalPath);
- CompletedSnapshot completedSnapshot = optCompletedSnapshot.get();
- // always create a new dir for the kv tablet
- File tabletDir = kvManager.createTabletDir(physicalPath, tableBucket);
- // down the snapshot to target tablet dir
- downloadKvSnapshots(completedSnapshot, tabletDir.toPath());
-
+ clock.milliseconds() - startTime);
// as we have downloaded kv files into the tablet dir, now, we can load it
- kvTablet = kvManager.loadKv(tabletDir, schemaGetter);
+ kvTablet = kvManager.loadKv(kvSnapshotManager.getTabletDir(), schemaGetter);
checkNotNull(kvTablet, "kv tablet should not be null.");
+ CompletedSnapshot completedSnapshot = optCompletedSnapshot.get();
restoreStartOffset = completedSnapshot.getLogOffset();
rowCount = completedSnapshot.getRowCount();
// currently, we only support one auto-increment column.
autoIncIDRange = completedSnapshot.getFirstAutoIncIDRange();
} else {
- LOG.info(
- "No snapshot found for {} of {}, restore from log.",
- tableBucket,
- physicalPath);
-
// actually, kv manager always create a kv tablet since we will drop the kv
// if it exists before init kv tablet
kvTablet =
@@ -774,49 +889,6 @@ private Optional initKvTablet() {
return optCompletedSnapshot;
}
- private void downloadKvSnapshots(CompletedSnapshot completedSnapshot, Path kvTabletDir)
- throws IOException {
- Path kvDbPath = kvTabletDir.resolve(RocksDBKvBuilder.DB_INSTANCE_DIR_STRING);
- KvSnapshotDownloadSpec downloadSpec =
- new KvSnapshotDownloadSpec(completedSnapshot.getKvSnapshotHandle(), kvDbPath);
- long start = clock.milliseconds();
- LOG.info("Start to download kv snapshot {} to directory {}.", completedSnapshot, kvDbPath);
- KvSnapshotDataDownloader kvSnapshotDataDownloader =
- snapshotContext.getSnapshotDataDownloader();
- try {
- kvSnapshotDataDownloader.transferAllDataToDirectory(downloadSpec, closeableRegistry);
- } catch (Exception e) {
- if (e.getMessage().contains(CompletedSnapshot.SNAPSHOT_DATA_NOT_EXISTS_ERROR_MESSAGE)) {
- try {
- snapshotContext.handleSnapshotBroken(completedSnapshot);
- } catch (Exception t) {
- LOG.error("Handle broken snapshot {} failed.", completedSnapshot, t);
- }
- }
- throw new IOException("Fail to download kv snapshot.", e);
- }
- long end = clock.milliseconds();
- LOG.info(
- "Download kv snapshot {} to directory {} finish, cost {} ms.",
- completedSnapshot,
- kvDbPath,
- end - start);
- }
-
- private Optional getLatestSnapshot(TableBucket tableBucket) {
- try {
- return Optional.ofNullable(
- snapshotContext.getLatestCompletedSnapshotProvider().apply(tableBucket));
- } catch (Exception e) {
- LOG.warn(
- "Get latest completed snapshot for {} of table {} failed.",
- tableBucket,
- physicalPath,
- e);
- }
- return Optional.empty();
- }
-
private void recoverKvTablet(
long startRecoverLogOffset,
@Nullable Long rowCount,
@@ -857,9 +929,11 @@ private void recoverKvTablet(
end - start);
}
- private void startPeriodicKvSnapshot(@Nullable CompletedSnapshot completedSnapshot) {
+ private void startPeriodicUploadKvSnapshot(@Nullable CompletedSnapshot completedSnapshot) {
checkNotNull(kvTablet);
- KvTabletSnapshotTarget kvTabletSnapshotTarget;
+ checkNotNull(kvSnapshotManager);
+ checkNotNull(closeableRegistryForKv);
+ KvTabletUploadSnapshotTarget kvTabletSnapshotTarget;
try {
// get the snapshot reporter to report the completed snapshot
CompletedKvSnapshotCommitter completedKvSnapshotCommitter =
@@ -911,7 +985,7 @@ private void startPeriodicKvSnapshot(@Nullable CompletedSnapshot completedSnapsh
snapshotContext.getRemoteKvDir(), physicalPath, tableBucket);
kvTabletSnapshotTarget =
- new KvTabletSnapshotTarget(
+ new KvTabletUploadSnapshotTarget(
tableBucket,
completedKvSnapshotCommitter,
snapshotContext.getZooKeeperClient(),
@@ -927,13 +1001,8 @@ private void startPeriodicKvSnapshot(@Nullable CompletedSnapshot completedSnapsh
coordinatorEpochSupplier,
lastCompletedSnapshotLogOffset,
snapshotSize);
- this.kvSnapshotManager =
- PeriodicSnapshotManager.create(
- tableBucket,
- kvTabletSnapshotTarget,
- snapshotContext,
- kvTablet.getGuardedExecutor());
- kvSnapshotManager.start();
+ kvSnapshotManager.startPeriodicUploadSnapshot(
+ kvTablet.getGuardedExecutor(), kvTabletSnapshotTarget);
closeableRegistryForKv.registerCloseable(kvSnapshotManager);
} catch (Exception e) {
LOG.error("init kv periodic snapshot for {} failed.", tableBucket, e);
@@ -948,6 +1017,14 @@ public long getLatestKvSnapshotSize() {
}
}
+ public long getStandbySnapshotSize() {
+ if (kvSnapshotManager == null || !isStandby()) {
+ return 0L;
+ } else {
+ return kvSnapshotManager.getStandbySnapshotSize();
+ }
+ }
+
public long getLeaderEndOffsetSnapshot() {
return logTablet.getLeaderEndOffsetSnapshot();
}
@@ -1134,7 +1211,10 @@ private boolean shouldWaitForReplicaToJoinIsr(
}
private void updateAssignmentAndIsr(
- List replicas, boolean isLeader, List isr) {
+ List replicas,
+ boolean isLeader,
+ List isr,
+ List standbyReplicas) {
if (isLeader) {
List followers =
replicas.stream()
@@ -1157,7 +1237,7 @@ private void updateAssignmentAndIsr(
}
// update isr info.
- isrState = new IsrState.CommittedIsrState(isr);
+ isrState = new IsrState.CommittedIsrState(isr, standbyReplicas);
}
private void updateFollowerFetchState(
@@ -1626,7 +1706,12 @@ private IsrState.PendingExpandIsrState prepareIsrExpand(
LeaderAndIsr newLeaderAndIsr =
new LeaderAndIsr(
- localTabletServerId, leaderEpoch, isrToSend, coordinatorEpoch, bucketEpoch);
+ localTabletServerId,
+ leaderEpoch,
+ isrToSend,
+ currentState.standbyReplicas(),
+ coordinatorEpoch,
+ bucketEpoch);
IsrState.PendingExpandIsrState updatedState =
new IsrState.PendingExpandIsrState(
@@ -1648,7 +1733,12 @@ IsrState.PendingShrinkIsrState prepareIsrShrink(
LeaderAndIsr newLeaderAndIsr =
new LeaderAndIsr(
- localTabletServerId, leaderEpoch, isrToSend, coordinatorEpoch, bucketEpoch);
+ localTabletServerId,
+ leaderEpoch,
+ isrToSend,
+ currentState.standbyReplicas(),
+ coordinatorEpoch,
+ bucketEpoch);
IsrState.PendingShrinkIsrState updatedState =
new IsrState.PendingShrinkIsrState(
outOfSyncFollowerReplicas, newLeaderAndIsr, currentState);
@@ -1743,7 +1833,9 @@ private boolean handleAdjustIsrUpdate(
// proposed and actual state are the same.
// In both cases, we want to move from Pending to Committed state to ensure new updates
// are processed.
- isrState = new IsrState.CommittedIsrState(leaderAndIsr.isr());
+ isrState =
+ new IsrState.CommittedIsrState(
+ leaderAndIsr.isr(), leaderAndIsr.standbyReplicas());
bucketEpoch = leaderAndIsr.bucketEpoch();
LOG.info(
"ISR updated to {} and bucket epoch updated to {} for bucket {}",
@@ -1939,6 +2031,11 @@ public boolean isLeader() {
return leaderReplicaId != null && leaderReplicaId.equals(localTabletServerId);
}
+ @VisibleForTesting
+ public boolean isStandby() {
+ return isStandbyReplica;
+ }
+
private LogTablet createLog(
OffsetCheckpointFile.LazyOffsetCheckpoints lazyHighWatermarkCheckpoint)
throws Exception {
@@ -2027,7 +2124,7 @@ public SchemaGetter getSchemaGetter() {
@VisibleForTesting
@Nullable
- public PeriodicSnapshotManager getKvSnapshotManager() {
+ public KvSnapshotManager getKvSnapshotManager() {
return kvSnapshotManager;
}
}
diff --git a/fluss-server/src/main/java/org/apache/fluss/server/replica/ReplicaManager.java b/fluss-server/src/main/java/org/apache/fluss/server/replica/ReplicaManager.java
index c57f1c6fb7..2f59d00fe1 100644
--- a/fluss-server/src/main/java/org/apache/fluss/server/replica/ReplicaManager.java
+++ b/fluss-server/src/main/java/org/apache/fluss/server/replica/ReplicaManager.java
@@ -396,6 +396,8 @@ private void registerMetrics() {
physicalStorage.gauge(
MetricNames.SERVER_PHYSICAL_STORAGE_REMOTE_LOG_SIZE,
this::physicalStorageRemoteLogSize);
+ physicalStorage.gauge(
+ MetricNames.SERVER_PHYSICAL_STORAGE_STANDBY_SIZE, this::physicalStorageStandbySize);
}
@VisibleForTesting
@@ -448,13 +450,21 @@ private long physicalStorageLocalSize() {
replica -> {
long size = replica.getLogTablet().logSize();
if (replica.isKvTable()) {
- size += replica.getLatestKvSnapshotSize();
+ if (replica.isLeader()) {
+ size += replica.getLatestKvSnapshotSize();
+ } else if (replica.isStandby()) {
+ size += replica.getStandbySnapshotSize();
+ }
}
return size;
})
.reduce(0L, Long::sum);
}
+ private long physicalStorageStandbySize() {
+ return onlineReplicas().map(Replica::getStandbySnapshotSize).reduce(0L, Long::sum);
+ }
+
private long physicalStorageRemoteLogSize() {
return remoteLogManager.getRemoteLogSize();
}
@@ -1016,20 +1026,57 @@ public void notifyRemoteLogOffsets(
public void notifyKvSnapshotOffset(
NotifyKvSnapshotOffsetData notifyKvSnapshotOffsetData,
Consumer responseCallback) {
- inLock(
- replicaStateChangeLock,
- () -> {
- // check or apply coordinator epoch.
- validateAndApplyCoordinatorEpoch(
- notifyKvSnapshotOffsetData.getCoordinatorEpoch(),
- "notifyKvSnapshotOffset");
- // update the snapshot offset.
- TableBucket tb = notifyKvSnapshotOffsetData.getTableBucket();
- LogTablet logTablet = getReplicaOrException(tb).getLogTablet();
- logTablet.updateMinRetainOffset(
- notifyKvSnapshotOffsetData.getMinRetainOffset());
- responseCallback.accept(new NotifyKvSnapshotOffsetResponse());
- });
+ TableBucket tb = notifyKvSnapshotOffsetData.getTableBucket();
+ Long snapshotId = notifyKvSnapshotOffsetData.getSnapshotId();
+ long minRetainOffset = notifyKvSnapshotOffsetData.getMinRetainOffset();
+
+ // Validate and get replica under lock
+ Replica replica =
+ inLock(
+ replicaStateChangeLock,
+ () -> {
+ // check or apply coordinator epoch.
+ validateAndApplyCoordinatorEpoch(
+ notifyKvSnapshotOffsetData.getCoordinatorEpoch(),
+ "notifyKvSnapshotOffset");
+ return getReplicaOrException(tb);
+ });
+
+ // Respond immediately to avoid blocking coordinator
+ responseCallback.accept(new NotifyKvSnapshotOffsetResponse());
+
+ // Download snapshot asynchronously outside the lock to avoid blocking
+ // leader/follower transitions and other state changes
+ if (snapshotId != null) {
+ ioExecutor.execute(
+ () -> {
+ try {
+ replica.downloadSnapshot(snapshotId);
+ LOG.debug(
+ "Successfully downloaded snapshot {} for standby replica {}.",
+ snapshotId,
+ tb);
+ } catch (Exception e) {
+ LOG.error(
+ "Error downloading snapshot id {} for standby replica {}.",
+ snapshotId,
+ tb,
+ e);
+ } finally {
+ // Always update minRetainOffset regardless of download success/failure.
+ // If we skip this, log segments may never be cleaned up when download
+ // keeps failing.
+ updateMinRetainOffset(replica, minRetainOffset);
+ }
+ });
+ } else {
+ updateMinRetainOffset(replica, minRetainOffset);
+ }
+ }
+
+ private void updateMinRetainOffset(Replica replica, long minRetainOffset) {
+ LogTablet logTablet = replica.getLogTablet();
+ logTablet.updateMinRetainOffset(minRetainOffset);
}
public void notifyLakeTableOffset(
diff --git a/fluss-server/src/main/java/org/apache/fluss/server/utils/ServerRpcMessageUtils.java b/fluss-server/src/main/java/org/apache/fluss/server/utils/ServerRpcMessageUtils.java
index 6f5672ff46..db021de3f8 100644
--- a/fluss-server/src/main/java/org/apache/fluss/server/utils/ServerRpcMessageUtils.java
+++ b/fluss-server/src/main/java/org/apache/fluss/server/utils/ServerRpcMessageUtils.java
@@ -632,7 +632,8 @@ public static PbNotifyLeaderAndIsrReqForBucket makeNotifyBucketLeaderAndIsr(
new PbNotifyLeaderAndIsrReqForBucket()
.setLeader(notifyLeaderAndIsrData.getLeader())
.setLeaderEpoch(notifyLeaderAndIsrData.getLeaderEpoch())
- .setBucketEpoch(notifyLeaderAndIsrData.getBucketEpoch());
+ .setBucketEpoch(notifyLeaderAndIsrData.getBucketEpoch())
+ .setStandbyReplicas(notifyLeaderAndIsrData.getStandbyReplicasArray());
TableBucket tb = notifyLeaderAndIsrData.getTableBucket();
PbTableBucket pbTableBucket =
@@ -668,6 +669,11 @@ public static List getNotifyLeaderAndIsrRequestData(
isr.add(reqForBucket.getIsrAt(i));
}
+ List standbyReplicas = new ArrayList<>();
+ for (int i = 0; i < reqForBucket.getStandbyReplicasCount(); i++) {
+ standbyReplicas.add(reqForBucket.getStandbyReplicaAt(i));
+ }
+
PbTableBucket pbTableBucket = reqForBucket.getTableBucket();
notifyLeaderAndIsrDataList.add(
new NotifyLeaderAndIsrData(
@@ -678,6 +684,7 @@ public static List getNotifyLeaderAndIsrRequestData(
reqForBucket.getLeader(),
reqForBucket.getLeaderEpoch(),
isr,
+ standbyReplicas,
request.getCoordinatorEpoch(),
reqForBucket.getBucketEpoch())));
}
@@ -1212,6 +1219,7 @@ public static AdjustIsrRequest makeAdjustIsrRequest(
reqForBucket.setPartitionId(tb.getPartitionId());
}
leaderAndIsr.isr().forEach(reqForBucket::addNewIsr);
+ leaderAndIsr.standbyReplicas().forEach(reqForBucket::addStandbyReplica);
if (reqForBucketByTableId.containsKey(tb.getTableId())) {
reqForBucketByTableId.get(tb.getTableId()).add(reqForBucket);
} else {
@@ -1253,12 +1261,18 @@ public static Map getAdjustIsrData(AdjustIsrRequest r
for (int i = 0; i < reqForBucket.getNewIsrsCount(); i++) {
newIsr.add(reqForBucket.getNewIsrAt(i));
}
+ List standbyReplica = new ArrayList<>();
+ for (int i = 0; i < reqForBucket.getStandbyReplicasCount(); i++) {
+ standbyReplica.add(reqForBucket.getStandbyReplicaAt(i));
+ }
+
leaderAndIsrMap.put(
tb,
new LeaderAndIsr(
leaderId,
reqForBucket.getLeaderEpoch(),
newIsr,
+ standbyReplica,
reqForBucket.getCoordinatorEpoch(),
reqForBucket.getBucketEpoch()));
}
@@ -1285,7 +1299,8 @@ public static AdjustIsrResponse makeAdjustIsrResponse(
.setLeaderEpoch(leaderAndIsr.leaderEpoch())
.setCoordinatorEpoch(leaderAndIsr.coordinatorEpoch())
.setBucketEpoch(leaderAndIsr.bucketEpoch())
- .setIsrs(leaderAndIsr.isrArray());
+ .setIsrs(leaderAndIsr.isrArray())
+ .setStandbyReplicas(leaderAndIsr.standbyReplicasArray());
}
if (respMap.containsKey(tb.getTableId())) {
@@ -1334,6 +1349,10 @@ public static Map getAdjustIsrResponseDat
for (int i = 0; i < respForBucket.getIsrsCount(); i++) {
isr.add(respForBucket.getIsrAt(i));
}
+ List standbyReplica = new ArrayList<>();
+ for (int i = 0; i < respForBucket.getStandbyReplicasCount(); i++) {
+ standbyReplica.add(respForBucket.getStandbyReplicaAt(i));
+ }
adjustIsrResult.put(
tb,
new AdjustIsrResultForBucket(
@@ -1342,6 +1361,7 @@ public static Map getAdjustIsrResponseDat
respForBucket.getLeaderId(),
respForBucket.getLeaderEpoch(),
isr,
+ standbyReplica,
respForBucket.getCoordinatorEpoch(),
respForBucket.getBucketEpoch())));
}
@@ -1554,18 +1574,20 @@ public static NotifyKvSnapshotOffsetData getNotifySnapshotOffsetData(
request.hasPartitionId() ? request.getPartitionId() : null,
request.getBucketId()),
request.getMinRetainOffset(),
- request.getCoordinatorEpoch());
+ request.getCoordinatorEpoch(),
+ request.hasSnapshotId() ? request.getSnapshotId() : null);
}
public static NotifyKvSnapshotOffsetRequest makeNotifyKvSnapshotOffsetRequest(
- TableBucket tableBucket, long minRetainOffset) {
+ TableBucket tableBucket, long minRetainOffset, long snapshotId) {
NotifyKvSnapshotOffsetRequest request = new NotifyKvSnapshotOffsetRequest();
if (tableBucket.getPartitionId() != null) {
request.setPartitionId(tableBucket.getPartitionId());
}
request.setTableId(tableBucket.getTableId())
.setBucketId(tableBucket.getBucket())
- .setMinRetainOffset(minRetainOffset);
+ .setMinRetainOffset(minRetainOffset)
+ .setSnapshotId(snapshotId);
return request;
}
diff --git a/fluss-server/src/main/java/org/apache/fluss/server/zk/data/LeaderAndIsr.java b/fluss-server/src/main/java/org/apache/fluss/server/zk/data/LeaderAndIsr.java
index 2512481389..0206017169 100644
--- a/fluss-server/src/main/java/org/apache/fluss/server/zk/data/LeaderAndIsr.java
+++ b/fluss-server/src/main/java/org/apache/fluss/server/zk/data/LeaderAndIsr.java
@@ -43,6 +43,8 @@ public class LeaderAndIsr {
/** The latest inSyncReplica collection. */
private final List isr;
+ private final List standbyReplicas;
+
/** The coordinator epoch. */
private final int coordinatorEpoch;
@@ -57,15 +59,22 @@ public LeaderAndIsr(int leader, int coordinatorEpoch) {
leader,
INITIAL_LEADER_EPOCH,
new ArrayList<>(),
+ new ArrayList<>(),
coordinatorEpoch,
INITIAL_BUCKET_EPOCH);
}
public LeaderAndIsr(
- int leader, int leaderEpoch, List isr, int coordinatorEpoch, int bucketEpoch) {
+ int leader,
+ int leaderEpoch,
+ List isr,
+ List standbyReplicas,
+ int coordinatorEpoch,
+ int bucketEpoch) {
this.leader = leader;
this.leaderEpoch = leaderEpoch;
this.isr = checkNotNull(isr);
+ this.standbyReplicas = checkNotNull(standbyReplicas);
this.coordinatorEpoch = coordinatorEpoch;
this.bucketEpoch = bucketEpoch;
}
@@ -77,9 +86,15 @@ public LeaderAndIsr(
* @param newIsr the new isr
* @return the new LeaderAndIsr
*/
- public LeaderAndIsr newLeaderAndIsr(int newLeader, List newIsr) {
+ public LeaderAndIsr newLeaderAndIsr(
+ int newLeader, List newIsr, List standbyReplicas) {
return new LeaderAndIsr(
- newLeader, leaderEpoch + 1, newIsr, coordinatorEpoch, bucketEpoch + 1);
+ newLeader,
+ leaderEpoch + 1,
+ newIsr,
+ standbyReplicas,
+ coordinatorEpoch,
+ bucketEpoch + 1);
}
/**
@@ -90,7 +105,8 @@ public LeaderAndIsr newLeaderAndIsr(int newLeader, List newIsr) {
* @return the new LeaderAndIsr
*/
public LeaderAndIsr newLeaderAndIsr(List newIsr) {
- return new LeaderAndIsr(leader, leaderEpoch, newIsr, coordinatorEpoch, bucketEpoch + 1);
+ return new LeaderAndIsr(
+ leader, leaderEpoch, newIsr, standbyReplicas, coordinatorEpoch, bucketEpoch + 1);
}
public int leader() {
@@ -109,10 +125,18 @@ public List isr() {
return isr;
}
+ public List standbyReplicas() {
+ return standbyReplicas;
+ }
+
public int[] isrArray() {
return isr.stream().mapToInt(Integer::intValue).toArray();
}
+ public int[] standbyReplicasArray() {
+ return standbyReplicas.stream().mapToInt(Integer::intValue).toArray();
+ }
+
public int bucketEpoch() {
return bucketEpoch;
}
@@ -130,12 +154,14 @@ public boolean equals(Object o) {
&& leaderEpoch == that.leaderEpoch
&& coordinatorEpoch == that.coordinatorEpoch
&& bucketEpoch == that.bucketEpoch
- && Objects.equals(isr, that.isr);
+ && Objects.equals(isr, that.isr)
+ && Objects.equals(standbyReplicas, that.standbyReplicas);
}
@Override
public int hashCode() {
- return Objects.hash(leader, leaderEpoch, isr, coordinatorEpoch, bucketEpoch);
+ return Objects.hash(
+ leader, leaderEpoch, isr, standbyReplicas, coordinatorEpoch, bucketEpoch);
}
@Override
@@ -147,6 +173,8 @@ public String toString() {
+ leaderEpoch
+ ", isr="
+ isr
+ + ", standbyReplicas="
+ + standbyReplicas
+ ", coordinatorEpoch="
+ coordinatorEpoch
+ ", bucketEpoch="
diff --git a/fluss-server/src/main/java/org/apache/fluss/server/zk/data/LeaderAndIsrJsonSerde.java b/fluss-server/src/main/java/org/apache/fluss/server/zk/data/LeaderAndIsrJsonSerde.java
index 9d76330f20..0f650b4dc7 100644
--- a/fluss-server/src/main/java/org/apache/fluss/server/zk/data/LeaderAndIsrJsonSerde.java
+++ b/fluss-server/src/main/java/org/apache/fluss/server/zk/data/LeaderAndIsrJsonSerde.java
@@ -35,13 +35,14 @@ public class LeaderAndIsrJsonSerde
public static final LeaderAndIsrJsonSerde INSTANCE = new LeaderAndIsrJsonSerde();
private static final String VERSION_KEY = "version";
- private static final int VERSION = 1;
+ private static final int VERSION = 2;
private static final String LEADER = "leader";
private static final String LEADER_EPOCH = "leader_epoch";
private static final String ISR = "isr";
private static final String COORDINATOR_EPOCH = "coordinator_epoch";
private static final String BUCKET_EPOCH = "bucket_epoch";
+ private static final String STANDBY_REPLICA = "standby_replicas";
@Override
public void serialize(LeaderAndIsr leaderAndIsr, JsonGenerator generator) throws IOException {
@@ -57,11 +58,19 @@ public void serialize(LeaderAndIsr leaderAndIsr, JsonGenerator generator) throws
generator.writeNumberField(COORDINATOR_EPOCH, leaderAndIsr.coordinatorEpoch());
generator.writeNumberField(BUCKET_EPOCH, leaderAndIsr.bucketEpoch());
+ // write standby_replicas
+ generator.writeArrayFieldStart(STANDBY_REPLICA);
+ for (Integer replica : leaderAndIsr.standbyReplicas()) {
+ generator.writeNumber(replica);
+ }
+ generator.writeEndArray();
+
generator.writeEndObject();
}
@Override
public LeaderAndIsr deserialize(JsonNode node) {
+ int version = node.get(VERSION_KEY).asInt();
int leader = node.get(LEADER).asInt();
int leaderEpoch = node.get(LEADER_EPOCH).asInt();
int coordinatorEpoch = node.get(COORDINATOR_EPOCH).asInt();
@@ -71,6 +80,16 @@ public LeaderAndIsr deserialize(JsonNode node) {
while (isrNodes.hasNext()) {
isr.add(isrNodes.next().asInt());
}
- return new LeaderAndIsr(leader, leaderEpoch, isr, coordinatorEpoch, bucketEpoch);
+
+ List standbyList = new ArrayList<>();
+ if (version > 1) {
+ Iterator hotStandbyListNodes = node.get(STANDBY_REPLICA).elements();
+ while (hotStandbyListNodes.hasNext()) {
+ standbyList.add(hotStandbyListNodes.next().asInt());
+ }
+ }
+
+ return new LeaderAndIsr(
+ leader, leaderEpoch, isr, standbyList, coordinatorEpoch, bucketEpoch);
}
}
diff --git a/fluss-server/src/test/java/org/apache/fluss/server/coordinator/CompletedSnapshotStoreManagerTest.java b/fluss-server/src/test/java/org/apache/fluss/server/coordinator/CompletedSnapshotStoreManagerTest.java
index 2558be96f9..c70bbd3043 100644
--- a/fluss-server/src/test/java/org/apache/fluss/server/coordinator/CompletedSnapshotStoreManagerTest.java
+++ b/fluss-server/src/test/java/org/apache/fluss/server/coordinator/CompletedSnapshotStoreManagerTest.java
@@ -335,5 +335,12 @@ public Optional getLatestCompletedSnapshotHandle(
return new ArrayList<>(snapshotHandleMap.get(tableBucket).values())
.stream().max(Comparator.comparingLong(CompletedSnapshotHandle::getSnapshotId));
}
+
+ @Override
+ public Optional getCompletedSnapshotHandle(
+ TableBucket tableBucket, long snapshotId) throws Exception {
+ return Optional.ofNullable(snapshotHandleMap.get(tableBucket))
+ .map(map -> map.get(snapshotId));
+ }
}
}
diff --git a/fluss-server/src/test/java/org/apache/fluss/server/coordinator/CoordinatorEventProcessorTest.java b/fluss-server/src/test/java/org/apache/fluss/server/coordinator/CoordinatorEventProcessorTest.java
index 74fd01c5e9..8c7f7cce45 100644
--- a/fluss-server/src/test/java/org/apache/fluss/server/coordinator/CoordinatorEventProcessorTest.java
+++ b/fluss-server/src/test/java/org/apache/fluss/server/coordinator/CoordinatorEventProcessorTest.java
@@ -769,6 +769,7 @@ void testNotifyOffsetsWithShrinkISR(@TempDir Path tempDir) throws Exception {
leader,
leaderAndIsr.leaderEpoch(),
newIsr,
+ Collections.emptyList(),
coordinatorEpoch,
bucketLeaderEpoch));
return null;
@@ -1027,7 +1028,9 @@ void testDoBucketReassignment() throws Exception {
CompletableFuture respCallback = new CompletableFuture<>();
// This isr list equals originReplicas + addingReplicas. the bucket epoch is 1.
- leaderAndIsrMap.put(tb0, new LeaderAndIsr(0, 0, Arrays.asList(0, 1, 2, 3), 0, 1));
+ leaderAndIsrMap.put(
+ tb0,
+ new LeaderAndIsr(0, 0, Arrays.asList(0, 1, 2, 3), Collections.emptyList(), 0, 1));
eventProcessor
.getCoordinatorEventManager()
.put(new AdjustIsrReceivedEvent(leaderAndIsrMap, respCallback));
diff --git a/fluss-server/src/test/java/org/apache/fluss/server/coordinator/TestCoordinatorGateway.java b/fluss-server/src/test/java/org/apache/fluss/server/coordinator/TestCoordinatorGateway.java
index 6aeee7137c..96930dbe22 100644
--- a/fluss-server/src/test/java/org/apache/fluss/server/coordinator/TestCoordinatorGateway.java
+++ b/fluss-server/src/test/java/org/apache/fluss/server/coordinator/TestCoordinatorGateway.java
@@ -312,6 +312,7 @@ public CompletableFuture adjustIsr(AdjustIsrRequest request)
leaderAndIsr.leader(),
currentLeaderEpoch,
leaderAndIsr.isr(),
+ leaderAndIsr.standbyReplicas(),
leaderAndIsr.coordinatorEpoch(),
leaderAndIsr.bucketEpoch() + 1));
}
diff --git a/fluss-server/src/test/java/org/apache/fluss/server/coordinator/statemachine/ReplicaLeaderElectionTest.java b/fluss-server/src/test/java/org/apache/fluss/server/coordinator/statemachine/ReplicaLeaderElectionTest.java
index d1dc77d61a..5fbfaff477 100644
--- a/fluss-server/src/test/java/org/apache/fluss/server/coordinator/statemachine/ReplicaLeaderElectionTest.java
+++ b/fluss-server/src/test/java/org/apache/fluss/server/coordinator/statemachine/ReplicaLeaderElectionTest.java
@@ -41,30 +41,90 @@ public class ReplicaLeaderElectionTest {
void testDefaultReplicaLeaderElection() {
List assignments = Arrays.asList(2, 4);
List liveReplicas = Arrays.asList(2, 4);
- LeaderAndIsr originLeaderAndIsr = new LeaderAndIsr(4, 0, Arrays.asList(2, 4), 0, 0);
+ LeaderAndIsr originLeaderAndIsr =
+ new LeaderAndIsr(4, 0, Arrays.asList(2, 4), Collections.emptyList(), 0, 0);
DefaultLeaderElection defaultLeaderElection = new DefaultLeaderElection();
Optional leaderElectionResultOpt =
- defaultLeaderElection.leaderElection(assignments, liveReplicas, originLeaderAndIsr);
+ defaultLeaderElection.leaderElection(
+ assignments, liveReplicas, originLeaderAndIsr, false);
assertThat(leaderElectionResultOpt.isPresent()).isTrue();
ElectionResult leaderElectionResult = leaderElectionResultOpt.get();
assertThat(leaderElectionResult.getLiveReplicas()).containsExactlyInAnyOrder(2, 4);
assertThat(leaderElectionResult.getLeaderAndIsr().leader()).isEqualTo(2);
assertThat(leaderElectionResult.getLeaderAndIsr().isr()).containsExactlyInAnyOrder(2, 4);
+ assertThat(leaderElectionResult.getLeaderAndIsr().standbyReplicas()).isEmpty();
+ }
+
+ @Test
+ void testDefaultReplicaLeaderElectionForPkTable() {
+ List assignments = Arrays.asList(2, 3, 4);
+ List liveReplicas = Arrays.asList(3, 4);
+ LeaderAndIsr originLeaderAndIsr =
+ new LeaderAndIsr(2, 0, Arrays.asList(2, 3, 4), Collections.emptyList(), 0, 0);
+
+ // first, test origin leaderAndIsr don't have standby replica.
+ DefaultLeaderElection defaultLeaderElection = new DefaultLeaderElection();
+ Optional leaderElectionResultOpt =
+ defaultLeaderElection.leaderElection(
+ assignments, liveReplicas, originLeaderAndIsr, true);
+ assertThat(leaderElectionResultOpt.isPresent()).isTrue();
+ ElectionResult leaderElectionResult = leaderElectionResultOpt.get();
+ assertThat(leaderElectionResult.getLiveReplicas()).containsExactlyInAnyOrder(3, 4);
+ assertThat(leaderElectionResult.getLeaderAndIsr().leader()).isEqualTo(3);
+ assertThat(leaderElectionResult.getLeaderAndIsr().isr()).containsExactlyInAnyOrder(2, 3, 4);
+ assertThat(leaderElectionResult.getLeaderAndIsr().standbyReplicas())
+ .containsExactlyInAnyOrder(4);
+
+ // second. test origin leaderAndIsr has standby replica.
+ originLeaderAndIsr =
+ new LeaderAndIsr(2, 0, Arrays.asList(2, 3, 4), Collections.singletonList(4), 0, 0);
+ defaultLeaderElection = new DefaultLeaderElection();
+ leaderElectionResultOpt =
+ defaultLeaderElection.leaderElection(
+ assignments, liveReplicas, originLeaderAndIsr, true);
+ assertThat(leaderElectionResultOpt.isPresent()).isTrue();
+ leaderElectionResult = leaderElectionResultOpt.get();
+ assertThat(leaderElectionResult.getLiveReplicas()).containsExactlyInAnyOrder(3, 4);
+ assertThat(leaderElectionResult.getLeaderAndIsr().leader()).isEqualTo(4);
+ assertThat(leaderElectionResult.getLeaderAndIsr().isr()).containsExactlyInAnyOrder(2, 3, 4);
+ assertThat(leaderElectionResult.getLeaderAndIsr().standbyReplicas())
+ .containsExactlyInAnyOrder(3);
+
+ // third. test no enough live replicas.
+ assignments = Arrays.asList(2, 3, 4);
+ liveReplicas = Collections.singletonList(4);
+ originLeaderAndIsr =
+ new LeaderAndIsr(2, 0, Arrays.asList(2, 3, 4), Collections.emptyList(), 0, 0);
+ defaultLeaderElection = new DefaultLeaderElection();
+ leaderElectionResultOpt =
+ defaultLeaderElection.leaderElection(
+ assignments, liveReplicas, originLeaderAndIsr, true);
+ assertThat(leaderElectionResultOpt.isPresent()).isTrue();
+ leaderElectionResult = leaderElectionResultOpt.get();
+ assertThat(leaderElectionResult.getLiveReplicas()).containsExactlyInAnyOrder(4);
+ assertThat(leaderElectionResult.getLeaderAndIsr().leader()).isEqualTo(4);
+ assertThat(leaderElectionResult.getLeaderAndIsr().isr()).containsExactlyInAnyOrder(2, 3, 4);
+ assertThat(leaderElectionResult.getLeaderAndIsr().standbyReplicas()).isEmpty();
}
@Test
void testControlledShutdownReplicaLeaderElection() {
List assignments = Arrays.asList(2, 4);
List liveReplicas = Arrays.asList(2, 4);
- LeaderAndIsr originLeaderAndIsr = new LeaderAndIsr(2, 0, Arrays.asList(2, 4), 0, 0);
+ LeaderAndIsr originLeaderAndIsr =
+ new LeaderAndIsr(2, 0, Arrays.asList(2, 4), Collections.emptyList(), 0, 0);
Set shutdownTabletServers = Collections.singleton(2);
ControlledShutdownLeaderElection controlledShutdownLeaderElection =
new ControlledShutdownLeaderElection();
Optional leaderElectionResultOpt =
controlledShutdownLeaderElection.leaderElection(
- assignments, liveReplicas, originLeaderAndIsr, shutdownTabletServers);
+ assignments,
+ liveReplicas,
+ originLeaderAndIsr,
+ shutdownTabletServers,
+ false);
assertThat(leaderElectionResultOpt.isPresent()).isTrue();
ElectionResult leaderElectionResult = leaderElectionResultOpt.get();
assertThat(leaderElectionResult.getLiveReplicas()).containsExactlyInAnyOrder(4);
@@ -77,57 +137,317 @@ void testControlledShutdownReplicaLeaderElectionLastIsrShuttingDown() {
List assignments = Arrays.asList(2, 4);
List liveReplicas = Arrays.asList(2, 4);
LeaderAndIsr originLeaderAndIsr =
- new LeaderAndIsr(2, 0, Collections.singletonList(2), 0, 0);
+ new LeaderAndIsr(2, 0, Collections.singletonList(2), Collections.emptyList(), 0, 0);
Set shutdownTabletServers = Collections.singleton(2);
ControlledShutdownLeaderElection controlledShutdownLeaderElection =
new ControlledShutdownLeaderElection();
Optional leaderElectionResultOpt =
controlledShutdownLeaderElection.leaderElection(
- assignments, liveReplicas, originLeaderAndIsr, shutdownTabletServers);
+ assignments,
+ liveReplicas,
+ originLeaderAndIsr,
+ shutdownTabletServers,
+ false);
assertThat(leaderElectionResultOpt).isEmpty();
}
@Test
- void testControlledShutdownPartitionLeaderElectionAllIsrSimultaneouslyShutdown() {
+ void testControlledShutdownLeaderElectionAllIsrSimultaneouslyShutdown() {
List assignments = Arrays.asList(2, 4);
List liveReplicas = Arrays.asList(2, 4);
- LeaderAndIsr originLeaderAndIsr = new LeaderAndIsr(2, 0, Arrays.asList(2, 4), 0, 0);
+ LeaderAndIsr originLeaderAndIsr =
+ new LeaderAndIsr(2, 0, Arrays.asList(2, 4), Collections.emptyList(), 0, 0);
Set shutdownTabletServers = new HashSet<>(Arrays.asList(2, 4));
ControlledShutdownLeaderElection controlledShutdownLeaderElection =
new ControlledShutdownLeaderElection();
Optional leaderElectionResultOpt =
controlledShutdownLeaderElection.leaderElection(
- assignments, liveReplicas, originLeaderAndIsr, shutdownTabletServers);
+ assignments,
+ liveReplicas,
+ originLeaderAndIsr,
+ shutdownTabletServers,
+ false);
assertThat(leaderElectionResultOpt).isEmpty();
}
+ @Test
+ void testControlledShutdownReplicaLeaderElectionForPkTable() {
+ List assignments = Arrays.asList(2, 3, 4);
+ List liveReplicas = Arrays.asList(2, 3, 4);
+ LeaderAndIsr originLeaderAndIsr =
+ new LeaderAndIsr(2, 0, Arrays.asList(2, 3, 4), Collections.emptyList(), 0, 0);
+ Set shutdownTabletServers = Collections.singleton(2);
+ ControlledShutdownLeaderElection controlledShutdownLeaderElection =
+ new ControlledShutdownLeaderElection();
+ Optional leaderElectionResultOpt =
+ controlledShutdownLeaderElection.leaderElection(
+ assignments, liveReplicas, originLeaderAndIsr, shutdownTabletServers, true);
+ assertThat(leaderElectionResultOpt.isPresent()).isTrue();
+ ElectionResult leaderElectionResult = leaderElectionResultOpt.get();
+ assertThat(leaderElectionResult.getLiveReplicas()).containsExactlyInAnyOrder(3, 4);
+ assertThat(leaderElectionResult.getLeaderAndIsr().leader()).isEqualTo(3);
+ assertThat(leaderElectionResult.getLeaderAndIsr().isr()).containsExactlyInAnyOrder(3, 4);
+ assertThat(leaderElectionResult.getLeaderAndIsr().standbyReplicas())
+ .containsExactlyInAnyOrder(4);
+
+ originLeaderAndIsr =
+ new LeaderAndIsr(2, 0, Arrays.asList(2, 3, 4), Collections.singletonList(4), 0, 0);
+ controlledShutdownLeaderElection = new ControlledShutdownLeaderElection();
+ leaderElectionResultOpt =
+ controlledShutdownLeaderElection.leaderElection(
+ assignments, liveReplicas, originLeaderAndIsr, shutdownTabletServers, true);
+ assertThat(leaderElectionResultOpt.isPresent()).isTrue();
+ leaderElectionResult = leaderElectionResultOpt.get();
+ assertThat(leaderElectionResult.getLiveReplicas()).containsExactlyInAnyOrder(3, 4);
+ assertThat(leaderElectionResult.getLeaderAndIsr().leader()).isEqualTo(4);
+ assertThat(leaderElectionResult.getLeaderAndIsr().isr()).containsExactlyInAnyOrder(3, 4);
+ assertThat(leaderElectionResult.getLeaderAndIsr().standbyReplicas())
+ .containsExactlyInAnyOrder(3);
+ }
+
@Test
void testReassignBucketLeaderElection() {
List targetReplicas = Arrays.asList(1, 2, 3);
ReassignmentLeaderElection reassignmentLeaderElection =
new ReassignmentLeaderElection(targetReplicas);
List liveReplicas = Arrays.asList(1, 2, 3);
- LeaderAndIsr leaderAndIsr = new LeaderAndIsr(1, 0, Arrays.asList(1, 2, 3), 0, 0);
+ LeaderAndIsr leaderAndIsr =
+ new LeaderAndIsr(1, 0, Arrays.asList(1, 2, 3), Collections.emptyList(), 0, 0);
Optional leaderOpt =
- reassignmentLeaderElection.leaderElection(liveReplicas, leaderAndIsr);
+ reassignmentLeaderElection.leaderElection(liveReplicas, leaderAndIsr, false);
assertThat(leaderOpt).isPresent();
assertThat(leaderOpt.get().getLeaderAndIsr().leader()).isEqualTo(1);
targetReplicas = Arrays.asList(1, 2, 3);
reassignmentLeaderElection = new ReassignmentLeaderElection(targetReplicas);
liveReplicas = Arrays.asList(2, 3);
- leaderAndIsr = new LeaderAndIsr(1, 0, Arrays.asList(2, 3), 0, 0);
- leaderOpt = reassignmentLeaderElection.leaderElection(liveReplicas, leaderAndIsr);
+ leaderAndIsr = new LeaderAndIsr(1, 0, Arrays.asList(2, 3), Collections.emptyList(), 0, 0);
+ leaderOpt = reassignmentLeaderElection.leaderElection(liveReplicas, leaderAndIsr, false);
assertThat(leaderOpt).isPresent();
assertThat(leaderOpt.get().getLeaderAndIsr().leader()).isEqualTo(2);
targetReplicas = Arrays.asList(1, 2, 3);
reassignmentLeaderElection = new ReassignmentLeaderElection(targetReplicas);
liveReplicas = Arrays.asList(1, 2);
- leaderAndIsr = new LeaderAndIsr(2, 1, Collections.emptyList(), 0, 1);
- leaderOpt = reassignmentLeaderElection.leaderElection(liveReplicas, leaderAndIsr);
+ leaderAndIsr =
+ new LeaderAndIsr(2, 1, Collections.emptyList(), Collections.emptyList(), 0, 1);
+ leaderOpt = reassignmentLeaderElection.leaderElection(liveReplicas, leaderAndIsr, false);
assertThat(leaderOpt).isNotPresent();
}
+
+ @Test
+ void testDefaultLeaderElectionWithSingleReplica() {
+ // Test that single replica election doesn't throw IndexOutOfBoundsException
+ List assignments = Collections.singletonList(1);
+ List liveReplicas = Collections.singletonList(1);
+ LeaderAndIsr originLeaderAndIsr =
+ new LeaderAndIsr(1, 0, Collections.singletonList(1), Collections.emptyList(), 0, 0);
+
+ DefaultLeaderElection defaultLeaderElection = new DefaultLeaderElection();
+ Optional leaderElectionResultOpt =
+ defaultLeaderElection.leaderElection(
+ assignments, liveReplicas, originLeaderAndIsr, true);
+ assertThat(leaderElectionResultOpt.isPresent()).isTrue();
+ ElectionResult leaderElectionResult = leaderElectionResultOpt.get();
+ assertThat(leaderElectionResult.getLeaderAndIsr().leader()).isEqualTo(1);
+ // With only one replica, standby should be empty
+ assertThat(leaderElectionResult.getLeaderAndIsr().standbyReplicas()).isEmpty();
+ }
+
+ @Test
+ void testControlledShutdownWithSingleRemainingReplica() {
+ // Test controlled shutdown when only one replica remains available
+ List assignments = Arrays.asList(1, 2);
+ List liveReplicas = Arrays.asList(1, 2);
+ LeaderAndIsr originLeaderAndIsr =
+ new LeaderAndIsr(1, 0, Arrays.asList(1, 2), Collections.singletonList(2), 0, 0);
+ Set shutdownTabletServers = Collections.singleton(1);
+
+ ControlledShutdownLeaderElection controlledShutdownLeaderElection =
+ new ControlledShutdownLeaderElection();
+ Optional leaderElectionResultOpt =
+ controlledShutdownLeaderElection.leaderElection(
+ assignments, liveReplicas, originLeaderAndIsr, shutdownTabletServers, true);
+ assertThat(leaderElectionResultOpt.isPresent()).isTrue();
+ ElectionResult leaderElectionResult = leaderElectionResultOpt.get();
+ // Standby (2) should be promoted to leader
+ assertThat(leaderElectionResult.getLeaderAndIsr().leader()).isEqualTo(2);
+ // With only one remaining replica, standby should be empty
+ assertThat(leaderElectionResult.getLeaderAndIsr().standbyReplicas()).isEmpty();
+ }
+
+ @Test
+ void testReassignBucketLeaderElectionForPkTable() {
+ // Test reassignment election for PK table
+ List targetReplicas = Arrays.asList(1, 2, 3);
+ ReassignmentLeaderElection reassignmentLeaderElection =
+ new ReassignmentLeaderElection(targetReplicas);
+ List liveReplicas = Arrays.asList(1, 2, 3);
+ LeaderAndIsr leaderAndIsr =
+ new LeaderAndIsr(2, 0, Arrays.asList(1, 2, 3), Collections.singletonList(3), 0, 0);
+ Optional leaderOpt =
+ reassignmentLeaderElection.leaderElection(liveReplicas, leaderAndIsr, true);
+ assertThat(leaderOpt).isPresent();
+ // Standby (3) should be promoted to leader
+ assertThat(leaderOpt.get().getLeaderAndIsr().leader()).isEqualTo(3);
+ // A new standby should be assigned from remaining replicas
+ assertThat(leaderOpt.get().getLeaderAndIsr().standbyReplicas())
+ .containsExactlyInAnyOrder(1);
+ }
+
+ @Test
+ void testReassignBucketLeaderElectionForPkTableWithSingleReplica() {
+ // Test reassignment election for PK table with only one replica
+ List targetReplicas = Collections.singletonList(1);
+ ReassignmentLeaderElection reassignmentLeaderElection =
+ new ReassignmentLeaderElection(targetReplicas);
+ List liveReplicas = Collections.singletonList(1);
+ LeaderAndIsr leaderAndIsr =
+ new LeaderAndIsr(1, 0, Collections.singletonList(1), Collections.emptyList(), 0, 0);
+ Optional leaderOpt =
+ reassignmentLeaderElection.leaderElection(liveReplicas, leaderAndIsr, true);
+ assertThat(leaderOpt).isPresent();
+ assertThat(leaderOpt.get().getLeaderAndIsr().leader()).isEqualTo(1);
+ // With only one replica, standby should be empty
+ assertThat(leaderOpt.get().getLeaderAndIsr().standbyReplicas()).isEmpty();
+ }
+
+ @Test
+ void testDefaultLeaderElectionWithStandbyUnavailable() {
+ // Test when current standby is not in available replicas
+ List assignments = Arrays.asList(1, 2, 3);
+ List liveReplicas = Arrays.asList(1, 2); // standby (3) is not alive
+ LeaderAndIsr originLeaderAndIsr =
+ new LeaderAndIsr(1, 0, Arrays.asList(1, 2, 3), Collections.singletonList(3), 0, 0);
+
+ DefaultLeaderElection defaultLeaderElection = new DefaultLeaderElection();
+ Optional leaderElectionResultOpt =
+ defaultLeaderElection.leaderElection(
+ assignments, liveReplicas, originLeaderAndIsr, true);
+ assertThat(leaderElectionResultOpt.isPresent()).isTrue();
+ ElectionResult leaderElectionResult = leaderElectionResultOpt.get();
+ // Should select first available replica as leader since standby is unavailable
+ assertThat(leaderElectionResult.getLeaderAndIsr().leader()).isEqualTo(1);
+ // New standby should be selected from remaining available replicas
+ assertThat(leaderElectionResult.getLeaderAndIsr().standbyReplicas())
+ .containsExactlyInAnyOrder(2);
+ }
+
+ @Test
+ void testDefaultLeaderElectionWithNoAvailableReplicas() {
+ // Test when no replicas are available
+ List assignments = Arrays.asList(1, 2, 3);
+ List liveReplicas = Collections.emptyList();
+ LeaderAndIsr originLeaderAndIsr =
+ new LeaderAndIsr(1, 0, Arrays.asList(1, 2, 3), Collections.emptyList(), 0, 0);
+
+ DefaultLeaderElection defaultLeaderElection = new DefaultLeaderElection();
+ Optional leaderElectionResultOpt =
+ defaultLeaderElection.leaderElection(
+ assignments, liveReplicas, originLeaderAndIsr, true);
+ assertThat(leaderElectionResultOpt).isEmpty();
+ }
+
+ @Test
+ void testControlledShutdownForPkTableWithStandbyShuttingDown() {
+ // Test when the current standby is shutting down
+ List assignments = Arrays.asList(1, 2, 3);
+ List liveReplicas = Arrays.asList(1, 2, 3);
+ LeaderAndIsr originLeaderAndIsr =
+ new LeaderAndIsr(1, 0, Arrays.asList(1, 2, 3), Collections.singletonList(2), 0, 0);
+ Set shutdownTabletServers = Collections.singleton(2); // standby is shutting down
+
+ ControlledShutdownLeaderElection controlledShutdownLeaderElection =
+ new ControlledShutdownLeaderElection();
+ Optional leaderElectionResultOpt =
+ controlledShutdownLeaderElection.leaderElection(
+ assignments, liveReplicas, originLeaderAndIsr, shutdownTabletServers, true);
+ assertThat(leaderElectionResultOpt.isPresent()).isTrue();
+ ElectionResult leaderElectionResult = leaderElectionResultOpt.get();
+ // Leader (1) should remain leader since standby (2) is shutting down
+ assertThat(leaderElectionResult.getLeaderAndIsr().leader()).isEqualTo(1);
+ // New standby should be selected from remaining replicas
+ assertThat(leaderElectionResult.getLeaderAndIsr().standbyReplicas())
+ .containsExactlyInAnyOrder(3);
+ }
+
+ @Test
+ void testControlledShutdownForPkTableWithLeaderAndStandbyBothShuttingDown() {
+ // Test when both leader and standby are shutting down simultaneously
+ List assignments = Arrays.asList(1, 2, 3);
+ List liveReplicas = Arrays.asList(1, 2, 3);
+ LeaderAndIsr originLeaderAndIsr =
+ new LeaderAndIsr(1, 0, Arrays.asList(1, 2, 3), Collections.singletonList(2), 0, 0);
+ // Both leader (1) and standby (2) are shutting down
+ Set shutdownTabletServers = new HashSet<>(Arrays.asList(1, 2));
+
+ ControlledShutdownLeaderElection controlledShutdownLeaderElection =
+ new ControlledShutdownLeaderElection();
+ Optional leaderElectionResultOpt =
+ controlledShutdownLeaderElection.leaderElection(
+ assignments, liveReplicas, originLeaderAndIsr, shutdownTabletServers, true);
+ assertThat(leaderElectionResultOpt.isPresent()).isTrue();
+ ElectionResult leaderElectionResult = leaderElectionResultOpt.get();
+ // Replica 3 should become leader since both leader and standby are shutting down
+ assertThat(leaderElectionResult.getLeaderAndIsr().leader()).isEqualTo(3);
+ assertThat(leaderElectionResult.getLiveReplicas()).containsExactlyInAnyOrder(3);
+ assertThat(leaderElectionResult.getLeaderAndIsr().isr()).containsExactlyInAnyOrder(3);
+ // No remaining replicas for standby
+ assertThat(leaderElectionResult.getLeaderAndIsr().standbyReplicas()).isEmpty();
+ }
+
+ @Test
+ void testControlledShutdownAllIsrShutdownForPkTable() {
+ // Test when all ISR replicas are shutting down for PK table
+ List assignments = Arrays.asList(1, 2, 3);
+ List liveReplicas = Arrays.asList(1, 2, 3);
+ LeaderAndIsr originLeaderAndIsr =
+ new LeaderAndIsr(1, 0, Arrays.asList(1, 2, 3), Collections.singletonList(2), 0, 0);
+ Set shutdownTabletServers = new HashSet<>(Arrays.asList(1, 2, 3));
+
+ ControlledShutdownLeaderElection controlledShutdownLeaderElection =
+ new ControlledShutdownLeaderElection();
+ Optional leaderElectionResultOpt =
+ controlledShutdownLeaderElection.leaderElection(
+ assignments, liveReplicas, originLeaderAndIsr, shutdownTabletServers, true);
+ // No available replicas, should return empty
+ assertThat(leaderElectionResultOpt).isEmpty();
+ }
+
+ @Test
+ void testReassignBucketLeaderElectionForPkTableWithStandbyUnavailable() {
+ // Test reassignment election for PK table when current standby is not in new replicas
+ List targetReplicas = Arrays.asList(4, 5, 6);
+ ReassignmentLeaderElection reassignmentLeaderElection =
+ new ReassignmentLeaderElection(targetReplicas);
+ List liveReplicas = Arrays.asList(4, 5, 6);
+ // Current standby (3) is not in the new target replicas
+ LeaderAndIsr leaderAndIsr =
+ new LeaderAndIsr(1, 0, Arrays.asList(4, 5, 6), Collections.singletonList(3), 0, 0);
+ Optional leaderOpt =
+ reassignmentLeaderElection.leaderElection(liveReplicas, leaderAndIsr, true);
+ assertThat(leaderOpt).isPresent();
+ // First available replica should be leader since old standby (3) is not available
+ assertThat(leaderOpt.get().getLeaderAndIsr().leader()).isEqualTo(4);
+ // New standby should be selected from remaining available replicas
+ assertThat(leaderOpt.get().getLeaderAndIsr().standbyReplicas())
+ .containsExactlyInAnyOrder(5);
+ }
+
+ @Test
+ void testReassignBucketLeaderElectionForPkTableNoAvailableReplicas() {
+ // Test reassignment election for PK table when no replicas are available
+ List targetReplicas = Arrays.asList(1, 2, 3);
+ ReassignmentLeaderElection reassignmentLeaderElection =
+ new ReassignmentLeaderElection(targetReplicas);
+ List liveReplicas = Arrays.asList(4, 5); // None of the targets are live
+ LeaderAndIsr leaderAndIsr =
+ new LeaderAndIsr(1, 0, Arrays.asList(4, 5), Collections.singletonList(2), 0, 0);
+ Optional leaderOpt =
+ reassignmentLeaderElection.leaderElection(liveReplicas, leaderAndIsr, true);
+ // No available replicas should return empty
+ assertThat(leaderOpt).isEmpty();
+ }
}
diff --git a/fluss-server/src/test/java/org/apache/fluss/server/coordinator/statemachine/ReplicaStateMachineTest.java b/fluss-server/src/test/java/org/apache/fluss/server/coordinator/statemachine/ReplicaStateMachineTest.java
index 454ec5de4b..1bb1a14868 100644
--- a/fluss-server/src/test/java/org/apache/fluss/server/coordinator/statemachine/ReplicaStateMachineTest.java
+++ b/fluss-server/src/test/java/org/apache/fluss/server/coordinator/statemachine/ReplicaStateMachineTest.java
@@ -213,7 +213,8 @@ void testOfflineReplicasShouldBeRemovedFromIsr() throws Exception {
replicas.add(replica);
}
// put leader and isr
- LeaderAndIsr leaderAndIsr = new LeaderAndIsr(0, 0, Arrays.asList(0, 1, 2), 0, 0);
+ LeaderAndIsr leaderAndIsr =
+ new LeaderAndIsr(0, 0, Arrays.asList(0, 1, 2), Collections.emptyList(), 0, 0);
zookeeperClient.registerLeaderAndIsr(tableBucket, leaderAndIsr);
coordinatorContext.updateBucketReplicaAssignment(tableBucket, Arrays.asList(0, 1, 2));
coordinatorContext.putBucketLeaderAndIsr(tableBucket, leaderAndIsr);
@@ -223,7 +224,14 @@ void testOfflineReplicasShouldBeRemovedFromIsr() throws Exception {
replicaStateMachine.handleStateChanges(replicas, OfflineReplica);
leaderAndIsr = coordinatorContext.getBucketLeaderAndIsr(tableBucket).get();
assertThat(leaderAndIsr)
- .isEqualTo(new LeaderAndIsr(LeaderAndIsr.NO_LEADER, 3, Arrays.asList(2), 0, 3));
+ .isEqualTo(
+ new LeaderAndIsr(
+ LeaderAndIsr.NO_LEADER,
+ 3,
+ Collections.singletonList(2),
+ Collections.emptyList(),
+ 0,
+ 3));
}
@Test
@@ -249,7 +257,8 @@ void testOfflineReplicaShouldBeRemovedFromIsr() throws Exception {
coordinatorContext.putReplicaState(replica, OnlineReplica);
}
// put leader and isr
- LeaderAndIsr leaderAndIsr = new LeaderAndIsr(0, 0, Arrays.asList(0, 1, 2), 0, 0);
+ LeaderAndIsr leaderAndIsr =
+ new LeaderAndIsr(0, 0, Arrays.asList(0, 1, 2), Collections.emptyList(), 0, 0);
zookeeperClient.registerLeaderAndIsr(tableBucket, leaderAndIsr);
coordinatorContext.updateBucketReplicaAssignment(tableBucket, Arrays.asList(0, 1, 2));
coordinatorContext.putBucketLeaderAndIsr(tableBucket, leaderAndIsr);
@@ -258,14 +267,18 @@ void testOfflineReplicaShouldBeRemovedFromIsr() throws Exception {
replicaStateMachine.handleStateChanges(
Collections.singleton(new TableBucketReplica(tableBucket, 1)), OfflineReplica);
leaderAndIsr = coordinatorContext.getBucketLeaderAndIsr(tableBucket).get();
- assertThat(leaderAndIsr).isEqualTo(new LeaderAndIsr(0, 0, Arrays.asList(0, 2), 0, 1));
+ assertThat(leaderAndIsr)
+ .isEqualTo(
+ new LeaderAndIsr(0, 0, Arrays.asList(0, 2), Collections.emptyList(), 0, 1));
// set replica 2 to offline
replicaStateMachine.handleStateChanges(
Collections.singleton(new TableBucketReplica(tableBucket, 2)), OfflineReplica);
leaderAndIsr = coordinatorContext.getBucketLeaderAndIsr(tableBucket).get();
assertThat(leaderAndIsr)
- .isEqualTo(new LeaderAndIsr(0, 0, Collections.singletonList(0), 0, 2));
+ .isEqualTo(
+ new LeaderAndIsr(
+ 0, 0, Collections.singletonList(0), Collections.emptyList(), 0, 2));
// set replica 0 to offline, isr shouldn't be empty, leader should be NO_LEADER
replicaStateMachine.handleStateChanges(
@@ -274,7 +287,12 @@ void testOfflineReplicaShouldBeRemovedFromIsr() throws Exception {
assertThat(leaderAndIsr)
.isEqualTo(
new LeaderAndIsr(
- LeaderAndIsr.NO_LEADER, 1, Collections.singletonList(0), 0, 3));
+ LeaderAndIsr.NO_LEADER,
+ 1,
+ Collections.singletonList(0),
+ Collections.emptyList(),
+ 0,
+ 3));
}
private void toReplicaDeletionStartedState(
diff --git a/fluss-server/src/test/java/org/apache/fluss/server/coordinator/statemachine/TableBucketStateMachineTest.java b/fluss-server/src/test/java/org/apache/fluss/server/coordinator/statemachine/TableBucketStateMachineTest.java
index b905ff9694..816256e6b2 100644
--- a/fluss-server/src/test/java/org/apache/fluss/server/coordinator/statemachine/TableBucketStateMachineTest.java
+++ b/fluss-server/src/test/java/org/apache/fluss/server/coordinator/statemachine/TableBucketStateMachineTest.java
@@ -145,6 +145,22 @@ void testStartup() throws Exception {
TableBucket t2b0 = new TableBucket(t2Id, 0);
coordinatorContext.putTablePath(t1Id, TablePath.of("db1", "t1"));
coordinatorContext.putTablePath(t2Id, TablePath.of("db1", "t2"));
+ coordinatorContext.putTableInfo(
+ TableInfo.of(
+ TablePath.of("db1", "t1"),
+ t1Id,
+ 0,
+ DATA1_TABLE_DESCRIPTOR,
+ System.currentTimeMillis(),
+ System.currentTimeMillis()));
+ coordinatorContext.putTableInfo(
+ TableInfo.of(
+ TablePath.of("db1", "t2"),
+ t2Id,
+ 0,
+ DATA1_TABLE_DESCRIPTOR,
+ System.currentTimeMillis(),
+ System.currentTimeMillis()));
coordinatorContext.setLiveTabletServers(createServers(Arrays.asList(0, 1, 3)));
makeSendLeaderAndStopRequestAlwaysSuccess(
@@ -156,9 +172,11 @@ void testStartup() throws Exception {
// create LeaderAndIsr for t10/t11 info in zk,
zookeeperClient.registerLeaderAndIsr(
- new TableBucket(t1Id, 0), new LeaderAndIsr(0, 0, Arrays.asList(0, 1), 0, 0));
+ new TableBucket(t1Id, 0),
+ new LeaderAndIsr(0, 0, Arrays.asList(0, 1), Collections.emptyList(), 0, 0));
zookeeperClient.registerLeaderAndIsr(
- new TableBucket(t1Id, 1), new LeaderAndIsr(2, 0, Arrays.asList(2, 3), 0, 0));
+ new TableBucket(t1Id, 1),
+ new LeaderAndIsr(2, 0, Arrays.asList(2, 3), Collections.emptyList(), 0, 0));
// update the LeaderAndIsr to context
coordinatorContext.putBucketLeaderAndIsr(
t1b0, zookeeperClient.getLeaderAndIsr(new TableBucket(t1Id, 0)).get());
@@ -302,6 +320,15 @@ void testStateChangeToOnline() throws Exception {
tableId = 5;
final TableBucket tableBucket1 = new TableBucket(tableId, 0);
coordinatorContext.putTablePath(tableId, fakeTablePath);
+ coordinatorContext.putTableInfo(
+ TableInfo.of(
+ fakeTablePath,
+ tableId,
+ 0,
+ DATA1_TABLE_DESCRIPTOR,
+ System.currentTimeMillis(),
+ System.currentTimeMillis()));
+ coordinatorContext.putTablePath(tableId, fakeTablePath);
coordinatorContext.updateBucketReplicaAssignment(tableBucket1, Arrays.asList(0, 1, 2));
coordinatorContext.putBucketState(tableBucket1, NewBucket);
tableBucketStateMachine.handleStateChange(
@@ -408,12 +435,71 @@ void testInitReplicaLeaderElection() {
List liveReplicas = Collections.singletonList(4);
Optional leaderElectionResultOpt =
- initReplicaLeaderElection(assignments, liveReplicas, 0);
+ initReplicaLeaderElection(assignments, liveReplicas, 0, false);
assertThat(leaderElectionResultOpt.isPresent()).isTrue();
ElectionResult leaderElectionResult = leaderElectionResultOpt.get();
assertThat(leaderElectionResult.getLiveReplicas()).containsExactlyInAnyOrder(4);
assertThat(leaderElectionResult.getLeaderAndIsr().leader()).isEqualTo(4);
assertThat(leaderElectionResult.getLeaderAndIsr().isr()).containsExactlyInAnyOrder(4);
+ assertThat(leaderElectionResult.getLeaderAndIsr().standbyReplicas()).isEmpty();
+ }
+
+ @Test
+ void testInitReplicaLeaderElectionForPkTable() {
+ // PK table with multiple available replicas: first as leader, second as standby
+ List assignments = Arrays.asList(1, 2, 3);
+ List liveReplicas = Arrays.asList(1, 2, 3);
+
+ Optional resultOpt =
+ initReplicaLeaderElection(assignments, liveReplicas, 0, true);
+ assertThat(resultOpt).isPresent();
+ ElectionResult result = resultOpt.get();
+ assertThat(result.getLeaderAndIsr().leader()).isEqualTo(1);
+ assertThat(result.getLeaderAndIsr().isr()).containsExactlyInAnyOrder(1, 2, 3);
+ assertThat(result.getLeaderAndIsr().standbyReplicas()).containsExactly(2);
+ }
+
+ @Test
+ void testInitReplicaLeaderElectionForPkTableWithSingleAvailableReplica() {
+ // PK table with only one available replica: leader only, no standby
+ List assignments = Arrays.asList(1, 2, 3);
+ List liveReplicas = Collections.singletonList(2);
+
+ Optional resultOpt =
+ initReplicaLeaderElection(assignments, liveReplicas, 0, true);
+ assertThat(resultOpt).isPresent();
+ ElectionResult result = resultOpt.get();
+ assertThat(result.getLeaderAndIsr().leader()).isEqualTo(2);
+ assertThat(result.getLeaderAndIsr().isr()).containsExactlyInAnyOrder(2);
+ assertThat(result.getLeaderAndIsr().standbyReplicas()).isEmpty();
+ }
+
+ @Test
+ void testInitReplicaLeaderElectionForPkTableWithNoAliveReplica() {
+ // PK table with no alive replicas: should return empty
+ List assignments = Arrays.asList(1, 2, 3);
+ List liveReplicas = Collections.emptyList();
+
+ Optional resultOpt =
+ initReplicaLeaderElection(assignments, liveReplicas, 0, true);
+ assertThat(resultOpt).isEmpty();
+ }
+
+ @Test
+ void testInitReplicaLeaderElectionForPkTableWithPartiallyAliveReplicas() {
+ // PK table where first assigned replica is not alive,
+ // second and third are alive
+ List assignments = Arrays.asList(1, 2, 3);
+ List liveReplicas = Arrays.asList(2, 3);
+
+ Optional resultOpt =
+ initReplicaLeaderElection(assignments, liveReplicas, 0, true);
+ assertThat(resultOpt).isPresent();
+ ElectionResult result = resultOpt.get();
+ // First available (2) should be leader, second available (3) should be standby
+ assertThat(result.getLeaderAndIsr().leader()).isEqualTo(2);
+ assertThat(result.getLeaderAndIsr().isr()).containsExactlyInAnyOrder(2, 3);
+ assertThat(result.getLeaderAndIsr().standbyReplicas()).containsExactly(3);
}
private TableBucketStateMachine createTableBucketStateMachine() {
diff --git a/fluss-server/src/test/java/org/apache/fluss/server/kv/KvManagerTest.java b/fluss-server/src/test/java/org/apache/fluss/server/kv/KvManagerTest.java
index cecff0f8ff..2cb4d549f1 100644
--- a/fluss-server/src/test/java/org/apache/fluss/server/kv/KvManagerTest.java
+++ b/fluss-server/src/test/java/org/apache/fluss/server/kv/KvManagerTest.java
@@ -289,7 +289,7 @@ void testSameTableNameInDifferentDb(String partitionName) throws Exception {
void testDropKv(String partitionName) throws Exception {
initTableBuckets(partitionName);
KvTablet kv1 = getOrCreateKv(tablePath1, partitionName, tableBucket1);
- kvManager.dropKv(kv1.getTableBucket());
+ kvManager.closeOrDropKv(kv1.getTableBucket(), true);
assertThat(kv1.getKvTabletDir()).doesNotExist();
assertThat(kvManager.getKv(tableBucket1)).isNotPresent();
diff --git a/fluss-server/src/test/java/org/apache/fluss/server/kv/snapshot/KvSnapshotManagerTest.java b/fluss-server/src/test/java/org/apache/fluss/server/kv/snapshot/KvSnapshotManagerTest.java
new file mode 100644
index 0000000000..a8d0412cd3
--- /dev/null
+++ b/fluss-server/src/test/java/org/apache/fluss/server/kv/snapshot/KvSnapshotManagerTest.java
@@ -0,0 +1,962 @@
+/*
+ * 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.fluss.server.kv.snapshot;
+
+import org.apache.fluss.config.ConfigOptions;
+import org.apache.fluss.config.Configuration;
+import org.apache.fluss.exception.FlussRuntimeException;
+import org.apache.fluss.fs.FsPath;
+import org.apache.fluss.metadata.TableBucket;
+import org.apache.fluss.server.kv.KvSnapshotResource;
+import org.apache.fluss.server.kv.rocksdb.RocksDBKvBuilder;
+import org.apache.fluss.server.zk.NOPErrorHandler;
+import org.apache.fluss.server.zk.ZooKeeperClient;
+import org.apache.fluss.server.zk.ZooKeeperExtension;
+import org.apache.fluss.testutils.common.AllCallbackWrapper;
+import org.apache.fluss.testutils.common.ManuallyTriggeredScheduledExecutorService;
+import org.apache.fluss.utils.CloseableRegistry;
+import org.apache.fluss.utils.clock.ManualClock;
+import org.apache.fluss.utils.function.FunctionWithException;
+
+import org.junit.jupiter.api.AfterEach;
+import org.junit.jupiter.api.Assertions;
+import org.junit.jupiter.api.BeforeAll;
+import org.junit.jupiter.api.BeforeEach;
+import org.junit.jupiter.api.Test;
+import org.junit.jupiter.api.extension.RegisterExtension;
+import org.junit.jupiter.api.io.TempDir;
+
+import java.io.File;
+import java.io.IOException;
+import java.nio.file.Files;
+import java.nio.file.Path;
+import java.time.Duration;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.List;
+import java.util.Optional;
+import java.util.concurrent.ExecutionException;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+import java.util.concurrent.FutureTask;
+import java.util.concurrent.RunnableFuture;
+import java.util.concurrent.ScheduledExecutorService;
+import java.util.stream.Collectors;
+
+import static java.util.concurrent.TimeUnit.MILLISECONDS;
+import static org.apache.fluss.shaded.guava32.com.google.common.collect.Iterators.getOnlyElement;
+import static org.assertj.core.api.Assertions.assertThat;
+import static org.assertj.core.api.Assertions.assertThatThrownBy;
+
+/** Test for {@link KvSnapshotManager} . */
+class KvSnapshotManagerTest {
+ @RegisterExtension
+ public static final AllCallbackWrapper ZOO_KEEPER_EXTENSION_WRAPPER =
+ new AllCallbackWrapper<>(new ZooKeeperExtension());
+
+ private static final long periodicMaterializeDelay = 10_000L;
+ private static ZooKeeperClient zkClient;
+ private final TableBucket tableBucket = new TableBucket(1, 1);
+ private ManuallyTriggeredScheduledExecutorService scheduledExecutorService;
+ private ManuallyTriggeredScheduledExecutorService asyncSnapshotExecutorService;
+ private KvSnapshotResource kvSnapshotResource;
+ private KvSnapshotManager kvSnapshotManager;
+ private DefaultSnapshotContext snapshotContext;
+ private Configuration conf;
+ private ManualClock manualClock;
+ private @TempDir File tmpKvDir;
+
+ @BeforeAll
+ static void baseBeforeAll() {
+ zkClient =
+ ZOO_KEEPER_EXTENSION_WRAPPER
+ .getCustomExtension()
+ .getZooKeeperClient(NOPErrorHandler.INSTANCE);
+ }
+
+ @BeforeEach
+ void before() {
+ conf = new Configuration();
+ conf.set(ConfigOptions.KV_SNAPSHOT_INTERVAL, Duration.ofMillis(periodicMaterializeDelay));
+ scheduledExecutorService = new ManuallyTriggeredScheduledExecutorService();
+ asyncSnapshotExecutorService = new ManuallyTriggeredScheduledExecutorService();
+ ExecutorService dataTransferThreadPool = Executors.newFixedThreadPool(1);
+ kvSnapshotResource =
+ new KvSnapshotResource(
+ scheduledExecutorService,
+ new KvSnapshotDataUploader(dataTransferThreadPool),
+ new KvSnapshotDataDownloader(dataTransferThreadPool),
+ asyncSnapshotExecutorService);
+ snapshotContext =
+ DefaultSnapshotContext.create(
+ zkClient,
+ new TestingCompletedKvSnapshotCommitter(),
+ kvSnapshotResource,
+ conf);
+ manualClock = new ManualClock(System.currentTimeMillis());
+ }
+
+ @AfterEach
+ void close() {
+ if (kvSnapshotManager != null) {
+ kvSnapshotManager.close();
+ }
+ }
+
+ @Test
+ void testInitialDelay() {
+ kvSnapshotManager = createSnapshotManager(true);
+ startPeriodicUploadSnapshot(NopUploadSnapshotTarget.INSTANCE);
+ checkOnlyOneScheduledTasks();
+ }
+
+ @Test
+ void testInitWithNonPositiveSnapshotInterval() {
+ conf.set(ConfigOptions.KV_SNAPSHOT_INTERVAL, Duration.ofMillis(0));
+ snapshotContext =
+ DefaultSnapshotContext.create(
+ zkClient,
+ new TestingCompletedKvSnapshotCommitter(),
+ kvSnapshotResource,
+ conf);
+ kvSnapshotManager = createSnapshotManager(snapshotContext);
+ startPeriodicUploadSnapshot(NopUploadSnapshotTarget.INSTANCE);
+ // periodic snapshot is disabled when periodicMaterializeDelay is not positive
+ Assertions.assertEquals(0, scheduledExecutorService.getAllScheduledTasks().size());
+ }
+
+ @Test
+ void testPeriodicSnapshot() {
+ kvSnapshotManager = createSnapshotManager(true);
+ startPeriodicUploadSnapshot(NopUploadSnapshotTarget.INSTANCE);
+ // check only one schedule task
+ checkOnlyOneScheduledTasks();
+ scheduledExecutorService.triggerNonPeriodicScheduledTasks();
+ // after trigger, should still remain one task
+ checkOnlyOneScheduledTasks();
+ }
+
+ @Test
+ void testSnapshot() {
+ // use local filesystem to make the FileSystem plugin happy
+ String snapshotDir = "file:/test/snapshot1";
+ TestUploadSnapshotTarget target = new TestUploadSnapshotTarget(new FsPath(snapshotDir));
+ kvSnapshotManager = createSnapshotManager(true);
+ startPeriodicUploadSnapshot(target);
+ // trigger schedule
+ scheduledExecutorService.triggerNonPeriodicScheduledTasks();
+ // trigger async snapshot
+ asyncSnapshotExecutorService.trigger();
+
+ // now, check the result
+ assertThat(target.getCollectedRemoteDirs())
+ .isEqualTo(Collections.singletonList(snapshotDir));
+ }
+
+ @Test
+ void testSnapshotWithException() {
+ // use local filesystem to make the FileSystem plugin happy
+ String remoteDir = "file:/test/snapshot1";
+ String exceptionMessage = "Exception while initializing Materialization";
+ TestUploadSnapshotTarget target =
+ new TestUploadSnapshotTarget(new FsPath(remoteDir), exceptionMessage);
+ kvSnapshotManager = createSnapshotManager(true);
+ startPeriodicUploadSnapshot(target);
+ // trigger schedule
+ scheduledExecutorService.triggerNonPeriodicScheduledTasks();
+
+ // trigger async snapshot
+ asyncSnapshotExecutorService.trigger();
+
+ assertThat(target.getCause())
+ .isInstanceOf(ExecutionException.class)
+ .cause()
+ .isInstanceOf(FlussRuntimeException.class)
+ .hasMessage(exceptionMessage);
+ }
+
+ @Test
+ void testDownloadSnapshotByIdSuccessfully() throws Exception {
+ // Create a snapshot manager with successful downloader
+ SuccessfulDownloadSnapshotContext successContext =
+ new SuccessfulDownloadSnapshotContext(snapshotContext);
+ kvSnapshotManager =
+ new KvSnapshotManager(tableBucket, tmpKvDir, successContext, manualClock);
+ // Must call becomeStandby() first to create db directory
+ kvSnapshotManager.becomeStandby();
+
+ // Create a mock completed snapshot with specific snapshotId
+ long snapshotId = 123L;
+ CompletedSnapshot mockSnapshot = createMockCompletedSnapshot(snapshotId);
+ successContext.setMockSnapshot(mockSnapshot);
+
+ // Download by specific snapshotId should succeed
+ kvSnapshotManager.downloadSnapshot(snapshotId);
+
+ // Verify no temp directories left behind
+ File[] tempDirs = tmpKvDir.listFiles((dir, name) -> name.startsWith(".tmp_snapshot_"));
+ assertThat(tempDirs).isEmpty();
+
+ // Verify files are in the final db directory
+ Path dbPath = tmpKvDir.toPath().resolve(RocksDBKvBuilder.DB_INSTANCE_DIR_STRING);
+ assertThat(Files.exists(dbPath)).isTrue();
+
+ // Verify downloaded SST files cache is updated
+ assertThat(kvSnapshotManager.getDownloadedSstFiles()).isNotNull();
+ assertThat(kvSnapshotManager.getDownloadedSstFiles()).hasSize(1);
+
+ // Verify downloaded misc files cache is updated
+ assertThat(kvSnapshotManager.getDownloadedMiscFiles()).isNotNull();
+ assertThat(kvSnapshotManager.getDownloadedMiscFiles()).hasSize(1);
+
+ // Verify standby snapshot size is updated
+ assertThat(kvSnapshotManager.getStandbySnapshotSize())
+ .isEqualTo(mockSnapshot.getSnapshotSize());
+ }
+
+ @Test
+ void testDownloadSnapshotByIdFailure() {
+ // Create a snapshot manager with failing downloader
+ FailingSnapshotContext failingContext =
+ new FailingSnapshotContext(
+ snapshotContext, new IOException("Simulated download failure"));
+ kvSnapshotManager =
+ new KvSnapshotManager(tableBucket, tmpKvDir, failingContext, manualClock);
+ // Must call becomeStandby() first to create db directory
+ kvSnapshotManager.becomeStandby();
+
+ long snapshotId = 456L;
+ CompletedSnapshot mockSnapshot = createMockCompletedSnapshot(snapshotId);
+ failingContext.setMockSnapshot(mockSnapshot);
+
+ // Attempt to download by snapshotId - should fail
+ assertThatThrownBy(() -> kvSnapshotManager.downloadSnapshot(snapshotId))
+ .isInstanceOf(IOException.class)
+ .hasMessageContaining("Fail to download kv snapshot");
+
+ // Verify temp directories are cleaned up
+ File[] tempDirs = tmpKvDir.listFiles((dir, name) -> name.startsWith(".tmp_snapshot_"));
+ assertThat(tempDirs).isEmpty();
+ }
+
+ @Test
+ void testDownloadLatestSnapshotWhenNoSnapshotExists() throws Exception {
+ // Create a snapshot context that returns null for latest snapshot
+ NoSnapshotContext noSnapshotContext = new NoSnapshotContext(snapshotContext);
+ kvSnapshotManager =
+ new KvSnapshotManager(tableBucket, tmpKvDir, noSnapshotContext, manualClock);
+ // Must call becomeStandby() first to create db directory
+ kvSnapshotManager.becomeStandby();
+
+ // Download latest snapshot should return empty Optional when no snapshot exists
+ Optional result = kvSnapshotManager.downloadLatestSnapshot();
+ assertThat(result).isEmpty();
+
+ // Verify downloaded SST files cache is not initialized (still null)
+ assertThat(kvSnapshotManager.getDownloadedSstFiles()).isNull();
+ }
+
+ @Test
+ void testDownloadLatestSnapshotSuccessfullyUpdatesCache() throws Exception {
+ // Create a snapshot manager with successful downloader
+ SuccessfulDownloadSnapshotContext successContext =
+ new SuccessfulDownloadSnapshotContext(snapshotContext);
+ kvSnapshotManager =
+ new KvSnapshotManager(tableBucket, tmpKvDir, successContext, manualClock);
+ // Must call becomeStandby() first to create db directory
+ kvSnapshotManager.becomeStandby();
+
+ CompletedSnapshot mockSnapshot = createMockCompletedSnapshot();
+ successContext.setMockSnapshot(mockSnapshot);
+
+ // Download latest snapshot
+ Optional result = kvSnapshotManager.downloadLatestSnapshot();
+ assertThat(result).isPresent();
+ assertThat(result.get()).isEqualTo(mockSnapshot);
+
+ // Verify downloaded files caches are updated
+ assertThat(kvSnapshotManager.getDownloadedSstFiles()).isNotNull();
+ assertThat(kvSnapshotManager.getDownloadedSstFiles()).hasSize(1);
+ assertThat(kvSnapshotManager.getDownloadedMiscFiles()).isNotNull();
+ assertThat(kvSnapshotManager.getDownloadedMiscFiles()).hasSize(1);
+ }
+
+ @Test
+ void testIncrementalDownloadSkipsExistingFiles() throws Exception {
+ // Create a snapshot manager with successful downloader
+ SuccessfulDownloadSnapshotContext successContext =
+ new SuccessfulDownloadSnapshotContext(snapshotContext);
+ kvSnapshotManager =
+ new KvSnapshotManager(tableBucket, tmpKvDir, successContext, manualClock);
+ // Must call becomeStandby() first to create db directory
+ kvSnapshotManager.becomeStandby();
+
+ // First download
+ CompletedSnapshot firstSnapshot = createMockCompletedSnapshot(1L);
+ successContext.setMockSnapshot(firstSnapshot);
+ kvSnapshotManager.downloadSnapshot(1L);
+
+ // Verify first download
+ assertThat(kvSnapshotManager.getDownloadedSstFiles()).hasSize(1);
+
+ // Second download with same files - should be incremental (no new files to download)
+ CompletedSnapshot secondSnapshot = createMockCompletedSnapshot(2L);
+ successContext.setMockSnapshot(secondSnapshot);
+ kvSnapshotManager.downloadSnapshot(2L);
+
+ // Cache should still have one SST file
+ assertThat(kvSnapshotManager.getDownloadedSstFiles()).hasSize(1);
+ }
+
+ @Test
+ void testIncrementalDownloadWithNewSstFiles() throws Exception {
+ // Create a snapshot manager with successful downloader
+ SuccessfulDownloadSnapshotContext successContext =
+ new SuccessfulDownloadSnapshotContext(snapshotContext);
+ kvSnapshotManager =
+ new KvSnapshotManager(tableBucket, tmpKvDir, successContext, manualClock);
+ // Must call becomeStandby() first to create db directory
+ kvSnapshotManager.becomeStandby();
+ Path dbPath = tmpKvDir.toPath().resolve(RocksDBKvBuilder.DB_INSTANCE_DIR_STRING);
+
+ // First download: snapshot contains only test.sst
+ List firstSstFiles = Collections.singletonList("test.sst");
+ CompletedSnapshot firstSnapshot =
+ createMockCompletedSnapshotWithSstFiles(1L, firstSstFiles);
+ successContext.setMockSnapshot(firstSnapshot);
+ kvSnapshotManager.downloadLatestSnapshot();
+
+ // Verify first download: 1 SST file
+ assertThat(kvSnapshotManager.getDownloadedSstFiles()).hasSize(1);
+ assertThat(
+ kvSnapshotManager.getDownloadedSstFiles().stream()
+ .map(Path::toString)
+ .collect(Collectors.toSet()))
+ .containsExactly(dbPath.resolve("test.sst").toString());
+
+ // Second download: snapshot contains test.sst and new.sst
+ List secondSstFiles = Arrays.asList("test.sst", "new.sst");
+ CompletedSnapshot secondSnapshot =
+ createMockCompletedSnapshotWithSstFiles(2L, secondSstFiles);
+ successContext.setMockSnapshot(secondSnapshot);
+ kvSnapshotManager.downloadLatestSnapshot();
+
+ // Verify second download: 2 SST files (incremental download got new.sst)
+ assertThat(kvSnapshotManager.getDownloadedSstFiles()).hasSize(2);
+ assertThat(
+ kvSnapshotManager.getDownloadedSstFiles().stream()
+ .map(Path::toString)
+ .collect(Collectors.toSet()))
+ .containsExactlyInAnyOrder(
+ dbPath.resolve("test.sst").toString(),
+ dbPath.resolve("new.sst").toString());
+
+ // Verify files exist in db directory
+ assertThat(Files.exists(dbPath.resolve("test.sst"))).isTrue();
+ assertThat(Files.exists(dbPath.resolve("new.sst"))).isTrue();
+ }
+
+ @Test
+ void testIncrementalDownloadDeletesObsoleteSstFiles() throws Exception {
+ // Create a snapshot manager with successful downloader
+ SuccessfulDownloadSnapshotContext successContext =
+ new SuccessfulDownloadSnapshotContext(snapshotContext);
+ kvSnapshotManager =
+ new KvSnapshotManager(tableBucket, tmpKvDir, successContext, manualClock);
+ // Must call becomeStandby() first to create db directory
+ kvSnapshotManager.becomeStandby();
+ Path dbPath = tmpKvDir.toPath().resolve(RocksDBKvBuilder.DB_INSTANCE_DIR_STRING);
+
+ // First download: snapshot contains old.sst and common.sst
+ List