This is an automated email from the ASF dual-hosted git repository.

jimin pushed a commit to branch 2.x
in repository https://gitbox.apache.org/repos/asf/incubator-seata.git


The following commit(s) were added to refs/heads/2.x by this push:
     new 5c257765c1 test: add unit tests for server cluster module (#7767)
5c257765c1 is described below

commit 5c257765c165c532c41b71e5547a677084b31823
Author: Eric Wang <[email protected]>
AuthorDate: Tue Nov 11 09:01:05 2025 +0300

    test: add unit tests for server cluster module (#7767)
---
 .../cluster/listener/ClusterChangeEventTest.java   | 101 +++
 .../server/{ => cluster}/raft/RaftServerTest.java  |  41 +-
 .../server/cluster/raft/RaftStateMachineTest.java  | 895 +++++++++++++++++++++
 .../{ => cluster}/raft/RaftSyncMessageTest.java    |   2 +-
 .../server/{ => cluster}/raft/TestSecurity.java    |   2 +-
 .../raft/context/SeataClusterContextTest.java      | 167 ++++
 .../raft/execute/BranchSessionExecuteTest.java     |   2 +-
 .../raft/execute/GlobalSessionExecuteTest.java     |   2 +-
 .../raft/execute/LockExecuteTest.java              |   2 +-
 .../PutNodeInfoRequestProcessorTest.java}          |  24 +-
 .../request/PutNodeMetadataRequestTest.java        |  60 ++
 .../response/PutNodeMetadataResponseTest.java      |  65 ++
 .../raft/serializer/JacksonSerializerTest.java     |  80 ++
 .../cluster/raft/sync/msg/RaftBaseMsgTest.java     |  74 ++
 .../sync/msg/RaftBranchSessionSyncMsgTest.java     | 105 +++
 .../raft/sync/msg/RaftClusterMetadataMsgTest.java  |  89 ++
 .../sync/msg/RaftGlobalSessionSyncMsgTest.java     |  91 +++
 .../raft/sync/msg/RaftVGroupSyncMsgTest.java       |  69 ++
 .../sync/msg/dto/BranchTransactionDTOTest.java     |  78 ++
 .../sync/msg/dto/GlobalTransactionDTOTest.java     |  63 ++
 .../raft/sync/msg/dto/RaftClusterMetadataTest.java | 104 +++
 .../server/cluster/raft/util/RaftTaskUtilTest.java | 208 +++++
 22 files changed, 2309 insertions(+), 15 deletions(-)

diff --git 
a/server/src/test/java/org/apache/seata/server/cluster/listener/ClusterChangeEventTest.java
 
b/server/src/test/java/org/apache/seata/server/cluster/listener/ClusterChangeEventTest.java
new file mode 100644
index 0000000000..a99eb954ca
--- /dev/null
+++ 
b/server/src/test/java/org/apache/seata/server/cluster/listener/ClusterChangeEventTest.java
@@ -0,0 +1,101 @@
+/*
+ * 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.seata.server.cluster.listener;
+
+import org.junit.jupiter.api.Test;
+
+import java.time.Clock;
+
+import static org.junit.jupiter.api.Assertions.*;
+
+public class ClusterChangeEventTest {
+
+    @Test
+    public void testConstructorWithFullParameters() {
+        Object source = new Object();
+        String group = "test-group";
+        long term = 123L;
+        boolean leader = true;
+
+        ClusterChangeEvent event = new ClusterChangeEvent(source, group, term, 
leader);
+
+        assertEquals(source, event.getSource());
+        assertEquals(group, event.getGroup());
+        assertEquals(term, event.getTerm());
+        assertTrue(event.isLeader());
+    }
+
+    @Test
+    public void testConstructorWithSourceAndGroup() {
+        Object source = new Object();
+        String group = "test-group-2";
+
+        ClusterChangeEvent event = new ClusterChangeEvent(source, group);
+
+        assertEquals(source, event.getSource());
+        assertEquals(group, event.getGroup());
+        assertEquals(0L, event.getTerm());
+        assertFalse(event.isLeader());
+    }
+
+    @Test
+    public void testConstructorWithClock() {
+        Object source = new Object();
+        Clock clock = Clock.systemUTC();
+
+        ClusterChangeEvent event = new ClusterChangeEvent(source, clock);
+
+        assertEquals(source, event.getSource());
+        assertNull(event.getGroup());
+        assertEquals(0L, event.getTerm());
+        assertFalse(event.isLeader());
+    }
+
+    @Test
+    public void testSetAndGetGroup() {
+        ClusterChangeEvent event = new ClusterChangeEvent(new Object(), 
"initial-group");
+        event.setGroup("updated-group");
+        assertEquals("updated-group", event.getGroup());
+    }
+
+    @Test
+    public void testSetAndGetTerm() {
+        ClusterChangeEvent event = new ClusterChangeEvent(new Object(), 
Clock.systemUTC());
+        event.setTerm(999L);
+        assertEquals(999L, event.getTerm());
+    }
+
+    @Test
+    public void testSetAndGetLeader() {
+        ClusterChangeEvent event = new ClusterChangeEvent(new Object(), 
"group");
+        event.setLeader(true);
+        assertTrue(event.isLeader());
+
+        event.setLeader(false);
+        assertFalse(event.isLeader());
+    }
+
+    @Test
+    public void testLeaderStatusChange() {
+        Object source = new Object();
+        ClusterChangeEvent event = new ClusterChangeEvent(source, "group", 1L, 
false);
+        assertFalse(event.isLeader());
+
+        event.setLeader(true);
+        assertTrue(event.isLeader());
+    }
+}
diff --git 
a/server/src/test/java/org/apache/seata/server/raft/RaftServerTest.java 
b/server/src/test/java/org/apache/seata/server/cluster/raft/RaftServerTest.java
similarity index 79%
rename from 
server/src/test/java/org/apache/seata/server/raft/RaftServerTest.java
rename to 
server/src/test/java/org/apache/seata/server/cluster/raft/RaftServerTest.java
index eddc15740f..71e6bed711 100644
--- a/server/src/test/java/org/apache/seata/server/raft/RaftServerTest.java
+++ 
b/server/src/test/java/org/apache/seata/server/cluster/raft/RaftServerTest.java
@@ -14,14 +14,13 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.apache.seata.server.raft;
+package org.apache.seata.server.cluster.raft;
 
 import org.apache.seata.common.ConfigurationKeys;
 import org.apache.seata.common.XID;
 import org.apache.seata.config.ConfigurationCache;
 import org.apache.seata.config.ConfigurationFactory;
 import org.apache.seata.server.BaseSpringBootTest;
-import org.apache.seata.server.cluster.raft.RaftServerManager;
 import org.apache.seata.server.lock.LockerManagerFactory;
 import org.apache.seata.server.session.SessionHolder;
 import org.apache.seata.server.store.StoreConfig;
@@ -96,4 +95,42 @@ public class RaftServerTest extends BaseSpringBootTest {
         StoreConfig.setStartupParameter("raft", "raft", "raft");
         Assertions.assertThrows(IllegalArgumentException.class, 
RaftServerManager::init);
     }
+
+    @Test
+    public void testIsRaftModeWhenNotInitialized() {
+        Assertions.assertFalse(RaftServerManager.isRaftMode());
+    }
+
+    @Test
+    public void testGetRaftServerWhenNotInitialized() {
+        Assertions.assertNull(RaftServerManager.getRaftServer("default"));
+    }
+
+    @Test
+    public void testGetRaftServersWhenNotInitialized() {
+        Assertions.assertNotNull(RaftServerManager.getRaftServers());
+        Assertions.assertTrue(RaftServerManager.getRaftServers().isEmpty());
+    }
+
+    @Test
+    public void testGroupsWhenNotInitialized() {
+        Assertions.assertNotNull(RaftServerManager.groups());
+        Assertions.assertTrue(RaftServerManager.groups().isEmpty());
+    }
+
+    @Test
+    public void testIsLeaderWhenNotInRaftMode() {
+        StoreConfig.setStartupParameter("file", "file", "file");
+        Assertions.assertTrue(RaftServerManager.isLeader("default"));
+    }
+
+    @Test
+    public void testCliServiceInstance() {
+        Assertions.assertNotNull(RaftServerManager.getCliServiceInstance());
+    }
+
+    @Test
+    public void testCliClientServiceInstance() {
+        
Assertions.assertNotNull(RaftServerManager.getCliClientServiceInstance());
+    }
 }
diff --git 
a/server/src/test/java/org/apache/seata/server/cluster/raft/RaftStateMachineTest.java
 
b/server/src/test/java/org/apache/seata/server/cluster/raft/RaftStateMachineTest.java
new file mode 100644
index 0000000000..d19c9ea807
--- /dev/null
+++ 
b/server/src/test/java/org/apache/seata/server/cluster/raft/RaftStateMachineTest.java
@@ -0,0 +1,895 @@
+/*
+ * 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.seata.server.cluster.raft;
+
+import com.alipay.sofa.jraft.Closure;
+import com.alipay.sofa.jraft.Iterator;
+import com.alipay.sofa.jraft.Status;
+import com.alipay.sofa.jraft.conf.Configuration;
+import com.alipay.sofa.jraft.entity.LeaderChangeContext;
+import com.alipay.sofa.jraft.entity.PeerId;
+import com.alipay.sofa.jraft.storage.snapshot.SnapshotReader;
+import com.alipay.sofa.jraft.storage.snapshot.SnapshotWriter;
+import org.apache.seata.common.metadata.ClusterRole;
+import org.apache.seata.common.metadata.Node;
+import org.apache.seata.server.BaseSpringBootTest;
+import org.apache.seata.server.cluster.raft.execute.RaftMsgExecute;
+import org.apache.seata.server.cluster.raft.snapshot.StoreSnapshotFile;
+import 
org.apache.seata.server.cluster.raft.snapshot.metadata.LeaderMetadataSnapshotFile;
+import org.apache.seata.server.cluster.raft.sync.RaftSyncMessageSerializer;
+import org.apache.seata.server.cluster.raft.sync.msg.RaftBaseMsg;
+import org.apache.seata.server.cluster.raft.sync.msg.RaftClusterMetadataMsg;
+import org.apache.seata.server.cluster.raft.sync.msg.RaftSyncMessage;
+import org.apache.seata.server.cluster.raft.sync.msg.RaftSyncMsgType;
+import org.apache.seata.server.cluster.raft.sync.msg.dto.RaftClusterMetadata;
+import org.apache.seata.server.store.StoreConfig;
+import org.junit.jupiter.api.AfterEach;
+import org.junit.jupiter.api.BeforeEach;
+import org.junit.jupiter.api.Test;
+
+import java.lang.reflect.Field;
+import java.nio.ByteBuffer;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.List;
+import java.util.Map;
+
+import static org.junit.jupiter.api.Assertions.assertDoesNotThrow;
+import static org.junit.jupiter.api.Assertions.assertEquals;
+import static org.junit.jupiter.api.Assertions.assertFalse;
+import static org.junit.jupiter.api.Assertions.assertNotNull;
+import static org.junit.jupiter.api.Assertions.assertThrows;
+import static org.junit.jupiter.api.Assertions.assertTrue;
+import static org.junit.jupiter.api.Assertions.fail;
+import static org.mockito.ArgumentMatchers.any;
+import static org.mockito.Mockito.anyString;
+import static org.mockito.Mockito.argThat;
+import static org.mockito.Mockito.doThrow;
+import static org.mockito.Mockito.mock;
+import static org.mockito.Mockito.never;
+import static org.mockito.Mockito.times;
+import static org.mockito.Mockito.verify;
+import static org.mockito.Mockito.when;
+
+public class RaftStateMachineTest extends BaseSpringBootTest {
+
+    private RaftStateMachine raftStateMachine;
+    private static final String TEST_GROUP = "test-group";
+
+    @BeforeEach
+    public void setUp() {
+        StoreConfig.setStartupParameter("file", "file", "file");
+        raftStateMachine = new RaftStateMachine(TEST_GROUP);
+    }
+
+    @AfterEach
+    public void tearDown() {
+        StoreConfig.setStartupParameter("file", "file", "file");
+    }
+
+    @Test
+    public void testConstructorInitializesBasicFields() {
+        assertNotNull(raftStateMachine);
+        assertFalse(raftStateMachine.isLeader());
+        assertEquals(-1, raftStateMachine.getCurrentTerm().get());
+    }
+
+    @Test
+    public void testOnLeaderStartUpdatesLeaderTerm() {
+        long term = 5L;
+        assertFalse(raftStateMachine.isLeader());
+
+        raftStateMachine.onLeaderStart(term);
+
+        assertTrue(raftStateMachine.isLeader());
+        assertEquals(term, raftStateMachine.getCurrentTerm().get());
+    }
+
+    @Test
+    public void testOnLeaderStopResetsLeaderTerm() {
+        // First become leader
+        raftStateMachine.onLeaderStart(5L);
+        assertTrue(raftStateMachine.isLeader());
+
+        // Then stop being leader
+        raftStateMachine.onLeaderStop(Status.OK());
+
+        assertFalse(raftStateMachine.isLeader());
+    }
+
+    @Test
+    public void testOnLeaderStartMultipleTimes() {
+        raftStateMachine.onLeaderStart(1L);
+        assertTrue(raftStateMachine.isLeader());
+
+        raftStateMachine.onLeaderStart(2L);
+        assertTrue(raftStateMachine.isLeader());
+        assertEquals(2L, raftStateMachine.getCurrentTerm().get());
+    }
+
+    @Test
+    public void testOnStartFollowingUpdatesCurrentTerm() {
+        LeaderChangeContext ctx = new LeaderChangeContext(null, 1L, 
Status.OK());
+
+        raftStateMachine.onStartFollowing(ctx);
+
+        assertEquals(1L, raftStateMachine.getCurrentTerm().get());
+    }
+
+    @Test
+    public void testOnStopFollowingDoesNotThrow() {
+        LeaderChangeContext ctx = new LeaderChangeContext(null, 1L, 
Status.OK());
+
+        assertDoesNotThrow(() -> raftStateMachine.onStopFollowing(ctx));
+    }
+
+    @Test
+    public void testIsLeaderReturnsFalseInitially() {
+        assertFalse(raftStateMachine.isLeader());
+    }
+
+    @Test
+    public void testIsLeaderReturnsTrueAfterOnLeaderStart() {
+        raftStateMachine.onLeaderStart(1L);
+        assertTrue(raftStateMachine.isLeader());
+    }
+
+    @Test
+    public void testIsLeaderReturnsFalseAfterOnLeaderStop() {
+        raftStateMachine.onLeaderStart(1L);
+        raftStateMachine.onLeaderStop(Status.OK());
+        assertFalse(raftStateMachine.isLeader());
+    }
+
+    @Test
+    public void testGetCurrentTermInitialValue() {
+        assertEquals(-1, raftStateMachine.getCurrentTerm().get());
+    }
+
+    @Test
+    public void testGetCurrentTermAfterLeaderStart() {
+        raftStateMachine.onLeaderStart(10L);
+        assertEquals(10L, raftStateMachine.getCurrentTerm().get());
+    }
+
+    @Test
+    public void testRegistryStoreSnapshotFile() throws Exception {
+        LeaderMetadataSnapshotFile snapshotFile = new 
LeaderMetadataSnapshotFile(TEST_GROUP);
+        raftStateMachine.registryStoreSnapshotFile(snapshotFile);
+
+        Field snapshotFilesField = 
RaftStateMachine.class.getDeclaredField("snapshotFiles");
+        snapshotFilesField.setAccessible(true);
+        @SuppressWarnings("unchecked")
+        List<StoreSnapshotFile> snapshotFiles = (List<StoreSnapshotFile>) 
snapshotFilesField.get(raftStateMachine);
+
+        assertTrue(snapshotFiles.size() >= 2); // At least 
LeaderMetadataSnapshotFile from constructor + new one
+    }
+
+    @Test
+    public void testGetAndSetRaftLeaderMetadata() {
+        RaftClusterMetadata metadata = new RaftClusterMetadata(100L);
+        raftStateMachine.setRaftLeaderMetadata(metadata);
+
+        RaftClusterMetadata retrieved = 
raftStateMachine.getRaftLeaderMetadata();
+        assertEquals(100L, retrieved.getTerm());
+    }
+
+    @Test
+    public void testMultipleLeaderStarts() {
+        for (int i = 1; i <= 5; i++) {
+            raftStateMachine.onLeaderStart(i);
+            assertTrue(raftStateMachine.isLeader());
+            assertEquals(i, raftStateMachine.getCurrentTerm().get());
+        }
+    }
+
+    @Test
+    public void testLeaderStartStopCycle() {
+        raftStateMachine.onLeaderStart(1L);
+        assertTrue(raftStateMachine.isLeader());
+
+        raftStateMachine.onLeaderStop(Status.OK());
+        assertFalse(raftStateMachine.isLeader());
+
+        raftStateMachine.onLeaderStart(2L);
+        assertTrue(raftStateMachine.isLeader());
+        assertEquals(2L, raftStateMachine.getCurrentTerm().get());
+    }
+
+    @Test
+    public void testFollowerStartWithDifferentTerms() {
+        LeaderChangeContext ctx1 = new LeaderChangeContext(null, 5L, 
Status.OK());
+        raftStateMachine.onStartFollowing(ctx1);
+        assertEquals(5L, raftStateMachine.getCurrentTerm().get());
+
+        LeaderChangeContext ctx2 = new LeaderChangeContext(null, 10L, 
Status.OK());
+        raftStateMachine.onStartFollowing(ctx2);
+        assertEquals(10L, raftStateMachine.getCurrentTerm().get());
+    }
+
+    @Test
+    public void testLeaderTermProgression() {
+        assertEquals(-1, raftStateMachine.getCurrentTerm().get());
+
+        raftStateMachine.onLeaderStart(1L);
+        assertEquals(1L, raftStateMachine.getCurrentTerm().get());
+
+        raftStateMachine.onLeaderStart(5L);
+        assertEquals(5L, raftStateMachine.getCurrentTerm().get());
+
+        raftStateMachine.onLeaderStop(Status.OK());
+        assertEquals(5L, raftStateMachine.getCurrentTerm().get()); // 
currentTerm should remain
+    }
+
+    // ========== Tests for codecov uncovered methods ==========
+
+    @Test
+    public void testOnSnapshotSaveInFileMode() {
+        // In FILE mode, should call done.run(Status.OK()) immediately without 
saving
+        Closure done = mock(Closure.class);
+        SnapshotWriter writer = mock(SnapshotWriter.class);
+
+        raftStateMachine.onSnapshotSave(writer, done);
+
+        verify(done).run(argThat(Status::isOk));
+        verify(writer, never()).addFile(anyString());
+    }
+
+    @Test
+    public void testOnSnapshotSaveInRaftMode() throws Exception {
+        // Use reflection to change mode to RAFT without triggering singleton 
initialization
+        Field modeField = RaftStateMachine.class.getDeclaredField("mode");
+        modeField.setAccessible(true);
+        modeField.set(raftStateMachine, "raft");
+
+        // Clear default snapshot files and add only our mock
+        Field snapshotFilesField = 
RaftStateMachine.class.getDeclaredField("snapshotFiles");
+        snapshotFilesField.setAccessible(true);
+        @SuppressWarnings("unchecked")
+        List<StoreSnapshotFile> snapshotFiles = (List<StoreSnapshotFile>) 
snapshotFilesField.get(raftStateMachine);
+        snapshotFiles.clear();
+
+        Closure done = mock(Closure.class);
+        SnapshotWriter writer = mock(SnapshotWriter.class);
+        when(writer.getPath()).thenReturn("/tmp/snapshot");
+
+        // Register a mock snapshot file
+        StoreSnapshotFile mockSnapshotFile = mock(StoreSnapshotFile.class);
+        when(mockSnapshotFile.save(writer)).thenReturn(Status.OK());
+        raftStateMachine.registryStoreSnapshotFile(mockSnapshotFile);
+
+        raftStateMachine.onSnapshotSave(writer, done);
+
+        // Should call save on the snapshot file
+        verify(mockSnapshotFile).save(writer);
+        verify(done).run(argThat(Status::isOk));
+    }
+
+    @Test
+    public void testOnSnapshotSaveFailsWhenSnapshotFileReturnsError() throws 
Exception {
+        // Use reflection to change mode to RAFT
+        Field modeField = RaftStateMachine.class.getDeclaredField("mode");
+        modeField.setAccessible(true);
+        modeField.set(raftStateMachine, "raft");
+
+        // Clear default snapshot files
+        Field snapshotFilesField = 
RaftStateMachine.class.getDeclaredField("snapshotFiles");
+        snapshotFilesField.setAccessible(true);
+        @SuppressWarnings("unchecked")
+        List<StoreSnapshotFile> snapshotFiles = (List<StoreSnapshotFile>) 
snapshotFilesField.get(raftStateMachine);
+        snapshotFiles.clear();
+
+        Closure done = mock(Closure.class);
+        SnapshotWriter writer = mock(SnapshotWriter.class);
+        when(writer.getPath()).thenReturn("/tmp/snapshot");
+
+        // Register a mock snapshot file that fails
+        StoreSnapshotFile mockSnapshotFile = mock(StoreSnapshotFile.class);
+        Status errorStatus = new Status(-1, "Save failed");
+        when(mockSnapshotFile.save(writer)).thenReturn(errorStatus);
+        raftStateMachine.registryStoreSnapshotFile(mockSnapshotFile);
+
+        raftStateMachine.onSnapshotSave(writer, done);
+
+        // Should call done with error status
+        verify(done).run(argThat(status -> !status.isOk()));
+    }
+
+    @Test
+    public void testOnSnapshotLoadInFileMode() {
+        // In FILE mode, should return true immediately
+        SnapshotReader reader = mock(SnapshotReader.class);
+
+        boolean result = raftStateMachine.onSnapshotLoad(reader);
+
+        assertTrue(result);
+        verify(reader, never()).getPath();
+    }
+
+    @Test
+    public void testOnSnapshotLoadWhenIsLeader() throws Exception {
+        // Leader should not load snapshot
+        // Use reflection to change mode to RAFT
+        Field modeField = RaftStateMachine.class.getDeclaredField("mode");
+        modeField.setAccessible(true);
+        modeField.set(raftStateMachine, "raft");
+
+        raftStateMachine.onLeaderStart(1L); // Become leader
+
+        SnapshotReader reader = mock(SnapshotReader.class);
+
+        boolean result = raftStateMachine.onSnapshotLoad(reader);
+
+        assertFalse(result);
+    }
+
+    @Test
+    public void testOnSnapshotLoadInRaftModeAsFollower() throws Exception {
+        // Use reflection to change mode to RAFT
+        Field modeField = RaftStateMachine.class.getDeclaredField("mode");
+        modeField.setAccessible(true);
+        modeField.set(raftStateMachine, "raft");
+        // Not a leader (leaderTerm should be -1)
+
+        // Clear default snapshot files
+        Field snapshotFilesField = 
RaftStateMachine.class.getDeclaredField("snapshotFiles");
+        snapshotFilesField.setAccessible(true);
+        @SuppressWarnings("unchecked")
+        List<StoreSnapshotFile> snapshotFiles = (List<StoreSnapshotFile>) 
snapshotFilesField.get(raftStateMachine);
+        snapshotFiles.clear();
+
+        SnapshotReader reader = mock(SnapshotReader.class);
+        when(reader.getPath()).thenReturn("/tmp/snapshot");
+
+        // Register a mock snapshot file
+        StoreSnapshotFile mockSnapshotFile = mock(StoreSnapshotFile.class);
+        when(mockSnapshotFile.load(reader)).thenReturn(true);
+        raftStateMachine.registryStoreSnapshotFile(mockSnapshotFile);
+
+        boolean result = raftStateMachine.onSnapshotLoad(reader);
+
+        // Should call load on the snapshot file
+        verify(mockSnapshotFile).load(reader);
+        assertTrue(result);
+    }
+
+    @Test
+    public void testOnSnapshotLoadFailsWhenSnapshotFileReturnsFalse() throws 
Exception {
+        // Use reflection to change mode to RAFT
+        Field modeField = RaftStateMachine.class.getDeclaredField("mode");
+        modeField.setAccessible(true);
+        modeField.set(raftStateMachine, "raft");
+
+        // Clear default snapshot files
+        Field snapshotFilesField = 
RaftStateMachine.class.getDeclaredField("snapshotFiles");
+        snapshotFilesField.setAccessible(true);
+        @SuppressWarnings("unchecked")
+        List<StoreSnapshotFile> snapshotFiles = (List<StoreSnapshotFile>) 
snapshotFilesField.get(raftStateMachine);
+        snapshotFiles.clear();
+
+        SnapshotReader reader = mock(SnapshotReader.class);
+        when(reader.getPath()).thenReturn("/tmp/snapshot");
+
+        // Register a mock snapshot file that fails to load
+        StoreSnapshotFile mockSnapshotFile = mock(StoreSnapshotFile.class);
+        when(mockSnapshotFile.load(reader)).thenReturn(false);
+        raftStateMachine.registryStoreSnapshotFile(mockSnapshotFile);
+
+        boolean result = raftStateMachine.onSnapshotLoad(reader);
+
+        assertFalse(result);
+    }
+
+    @Test
+    public void testOnLeaderStartSetsTermsCorrectly() {
+        long term = 10L;
+        assertFalse(raftStateMachine.isLeader());
+
+        raftStateMachine.onLeaderStart(term);
+
+        assertTrue(raftStateMachine.isLeader());
+        assertEquals(term, raftStateMachine.getCurrentTerm().get());
+    }
+
+    @Test
+    public void testOnConfigurationCommitted() {
+        // Create a configuration
+        Configuration conf = new Configuration();
+        conf.addPeer(new PeerId("127.0.0.1", 8091));
+
+        // Should not throw exception
+        assertDoesNotThrow(() -> 
raftStateMachine.onConfigurationCommitted(conf));
+    }
+
+    @Test
+    public void testChangePeersWhenLeader() throws Exception {
+        // Become leader first
+        raftStateMachine.onLeaderStart(1L);
+        assertTrue(raftStateMachine.isLeader());
+
+        // Set up initial metadata with followers
+        RaftClusterMetadata metadata = new RaftClusterMetadata(1L);
+        Node follower1 = new Node();
+        follower1.setRole(ClusterRole.FOLLOWER);
+        follower1.setGroup(TEST_GROUP);
+        Node.Endpoint endpoint1 = new Node.Endpoint();
+        endpoint1.setHost("127.0.0.1");
+        endpoint1.setPort(8091);
+        follower1.setInternal(endpoint1);
+
+        Node follower2 = new Node();
+        follower2.setRole(ClusterRole.FOLLOWER);
+        follower2.setGroup(TEST_GROUP);
+        Node.Endpoint endpoint2 = new Node.Endpoint();
+        endpoint2.setHost("127.0.0.1");
+        endpoint2.setPort(8092);
+        follower2.setInternal(endpoint2);
+
+        metadata.setFollowers(Arrays.asList(follower1, follower2));
+        raftStateMachine.setRaftLeaderMetadata(metadata);
+
+        // Create a new configuration with only one peer
+        Configuration conf = new Configuration();
+        conf.addPeer(new PeerId("127.0.0.1", 8091));
+
+        // Call onConfigurationCommitted which should trigger changePeers
+        raftStateMachine.onConfigurationCommitted(conf);
+
+        // Give some time for async operations
+        Thread.sleep(100);
+
+        // Verify that the metadata has been updated
+        RaftClusterMetadata updatedMetadata = 
raftStateMachine.getRaftLeaderMetadata();
+        assertNotNull(updatedMetadata);
+    }
+
+    @Test
+    public void testChangePeersWithLearners() throws Exception {
+        // Become leader first
+        raftStateMachine.onLeaderStart(1L);
+        assertTrue(raftStateMachine.isLeader());
+
+        // Set up initial metadata with learners
+        RaftClusterMetadata metadata = new RaftClusterMetadata(1L);
+        Node learner1 = new Node();
+        learner1.setRole(ClusterRole.LEARNER);
+        learner1.setGroup(TEST_GROUP);
+        Node.Endpoint endpoint1 = new Node.Endpoint();
+        endpoint1.setHost("127.0.0.1");
+        endpoint1.setPort(8093);
+        learner1.setInternal(endpoint1);
+
+        metadata.setLearner(Collections.singletonList(learner1));
+        raftStateMachine.setRaftLeaderMetadata(metadata);
+
+        // Create a new configuration with learners
+        Configuration conf = new Configuration();
+        conf.addPeer(new PeerId("127.0.0.1", 8091));
+        conf.addLearner(new PeerId("127.0.0.1", 8093));
+
+        // Call onConfigurationCommitted which should trigger changePeers
+        raftStateMachine.onConfigurationCommitted(conf);
+
+        // Give some time for async operations
+        Thread.sleep(100);
+
+        // Verify that the metadata has been updated
+        RaftClusterMetadata updatedMetadata = 
raftStateMachine.getRaftLeaderMetadata();
+        assertNotNull(updatedMetadata);
+    }
+
+    @Test
+    public void testChangePeersWhenNotLeader() {
+        // Not a leader
+        assertFalse(raftStateMachine.isLeader());
+
+        // Create a configuration
+        Configuration conf = new Configuration();
+        conf.addPeer(new PeerId("127.0.0.1", 8091));
+
+        // Call onConfigurationCommitted - changePeers should not be called
+        assertDoesNotThrow(() -> 
raftStateMachine.onConfigurationCommitted(conf));
+    }
+
+    @Test
+    public void testChangeNodeMetadataForFollower() {
+        // Test adding a follower node
+        Node node = new Node();
+        node.setRole(ClusterRole.FOLLOWER);
+        node.setGroup(TEST_GROUP);
+
+        // Should not throw exception
+        assertDoesNotThrow(() -> raftStateMachine.changeNodeMetadata(node));
+
+        // Verify the node was added to followers
+        RaftClusterMetadata metadata = 
raftStateMachine.getRaftLeaderMetadata();
+        assertNotNull(metadata);
+    }
+
+    @Test
+    public void testChangeNodeMetadataForLearner() {
+        // Test adding a learner node
+        Node node = new Node();
+        node.setRole(ClusterRole.LEARNER);
+        node.setGroup(TEST_GROUP);
+
+        // Should not throw exception
+        assertDoesNotThrow(() -> raftStateMachine.changeNodeMetadata(node));
+
+        // Verify the node was added to learners
+        RaftClusterMetadata metadata = 
raftStateMachine.getRaftLeaderMetadata();
+        assertNotNull(metadata);
+    }
+
+    @Test
+    public void testSyncCurrentNodeInfoStringParameter() throws Exception {
+        // Use reflection to access the private method
+        java.lang.reflect.Method method = 
RaftStateMachine.class.getDeclaredMethod("syncCurrentNodeInfo", String.class);
+        method.setAccessible(true);
+
+        // Check that initSync is false initially
+        Field initSyncField = 
RaftStateMachine.class.getDeclaredField("initSync");
+        initSyncField.setAccessible(true);
+        java.util.concurrent.atomic.AtomicBoolean initSync =
+                (java.util.concurrent.atomic.AtomicBoolean) 
initSyncField.get(raftStateMachine);
+        assertFalse(initSync.get());
+
+        // Invoke the method - it will try to refresh leader and may fail due 
to test environment
+        // but we're just testing that the code path is executed without 
throwing unexpected exceptions
+        assertDoesNotThrow(() -> {
+            try {
+                method.invoke(raftStateMachine, TEST_GROUP);
+            } catch (java.lang.reflect.InvocationTargetException e) {
+                // Expected if dependencies are not set up - just ensure it's 
not a null pointer
+                Throwable cause = e.getCause();
+                // We expect some kind of initialization or configuration 
error in test environment
+                assertTrue(cause == null
+                        || cause instanceof NullPointerException
+                        || cause instanceof IllegalStateException
+                        || cause instanceof RuntimeException);
+            }
+        });
+    }
+
+    @Test
+    public void testSyncCurrentNodeInfoWithInitSyncAlreadyTrue() throws 
Exception {
+        // Use reflection to set initSync to true
+        Field initSyncField = 
RaftStateMachine.class.getDeclaredField("initSync");
+        initSyncField.setAccessible(true);
+        java.util.concurrent.atomic.AtomicBoolean initSync =
+                (java.util.concurrent.atomic.AtomicBoolean) 
initSyncField.get(raftStateMachine);
+        initSync.set(true);
+
+        // Use reflection to access the private method
+        java.lang.reflect.Method method = 
RaftStateMachine.class.getDeclaredMethod("syncCurrentNodeInfo", String.class);
+        method.setAccessible(true);
+
+        // Invoke the method - should return early without doing anything 
since initSync is already true
+        assertDoesNotThrow(() -> {
+            try {
+                method.invoke(raftStateMachine, TEST_GROUP);
+            } catch (java.lang.reflect.InvocationTargetException e) {
+                fail("Should not throw exception when initSync is already 
true");
+            }
+        });
+
+        // initSync should still be true
+        assertTrue(initSync.get());
+    }
+
+    @Test
+    public void testSyncCurrentNodeInfoPeerIdParameter() throws Exception {
+        // Use reflection to access the private method
+        java.lang.reflect.Method method = 
RaftStateMachine.class.getDeclaredMethod("syncCurrentNodeInfo", PeerId.class);
+        method.setAccessible(true);
+
+        // Create a test PeerId
+        PeerId leaderPeerId = new PeerId("127.0.0.1", 8091);
+
+        // Set up metadata with a leader that has a version
+        RaftClusterMetadata metadata = new RaftClusterMetadata(1L);
+        Node leader = new Node();
+        leader.setVersion("2.1.0");
+        metadata.setLeader(leader);
+        raftStateMachine.setRaftLeaderMetadata(metadata);
+
+        // Invoke the method - it will fail due to missing dependencies but 
we're testing the code path
+        assertDoesNotThrow(() -> {
+            try {
+                method.invoke(raftStateMachine, leaderPeerId);
+            } catch (java.lang.reflect.InvocationTargetException e) {
+                // Expected if RaftServerManager is not initialized in test 
environment
+                Throwable cause = e.getCause();
+                assertTrue(cause == null
+                        || cause instanceof NullPointerException
+                        || cause instanceof IllegalStateException
+                        || cause instanceof RuntimeException);
+            }
+        });
+    }
+
+    @Test
+    public void testSyncCurrentNodeInfoWithNoLeaderVersion() throws Exception {
+        // Use reflection to access the private method
+        java.lang.reflect.Method method = 
RaftStateMachine.class.getDeclaredMethod("syncCurrentNodeInfo", PeerId.class);
+        method.setAccessible(true);
+
+        // Create a test PeerId
+        PeerId leaderPeerId = new PeerId("127.0.0.1", 8091);
+
+        // Set up metadata with a leader that has NO version (should return 
early)
+        RaftClusterMetadata metadata = new RaftClusterMetadata(1L);
+        Node leader = new Node();
+        // No version set
+        metadata.setLeader(leader);
+        raftStateMachine.setRaftLeaderMetadata(metadata);
+
+        // Invoke the method - should return early without error
+        assertDoesNotThrow(() -> {
+            try {
+                method.invoke(raftStateMachine, leaderPeerId);
+            } catch (java.lang.reflect.InvocationTargetException e) {
+                fail("Should not throw when leader version is blank: " + 
e.getCause());
+            }
+        });
+    }
+
+    @Test
+    public void testSyncCurrentNodeInfoWithNullLeader() throws Exception {
+        // Use reflection to access the private method
+        java.lang.reflect.Method method = 
RaftStateMachine.class.getDeclaredMethod("syncCurrentNodeInfo", PeerId.class);
+        method.setAccessible(true);
+
+        // Create a test PeerId
+        PeerId leaderPeerId = new PeerId("127.0.0.1", 8091);
+
+        // Set up metadata with null leader
+        RaftClusterMetadata metadata = new RaftClusterMetadata(1L);
+        metadata.setLeader(null);
+        raftStateMachine.setRaftLeaderMetadata(metadata);
+
+        // Invoke the method - should return early without error
+        assertDoesNotThrow(() -> {
+            try {
+                method.invoke(raftStateMachine, leaderPeerId);
+            } catch (java.lang.reflect.InvocationTargetException e) {
+                fail("Should not throw when leader is null: " + e.getCause());
+            }
+        });
+    }
+
+    // ========== Tests for uncovered code paths from codecov ==========
+
+    @Test
+    public void testOnApplyWithFollowerPath() throws Exception {
+        // Test the follower execution path where iterator.done() returns null
+        Iterator iterator = mock(Iterator.class);
+
+        // Create a RaftClusterMetadataMsg wrapped in RaftSyncMessage
+        RaftClusterMetadata metadata = new RaftClusterMetadata(1L);
+        RaftClusterMetadataMsg msg = new RaftClusterMetadataMsg(metadata);
+        RaftSyncMessage syncMessage = new RaftSyncMessage();
+        syncMessage.setBody(msg);
+
+        // Serialize the message
+        byte[] msgBytes = RaftSyncMessageSerializer.encode(syncMessage);
+        ByteBuffer byteBuffer = ByteBuffer.wrap(msgBytes);
+
+        // Set up the iterator to simulate follower behavior
+        when(iterator.hasNext()).thenReturn(true, false); // One entry then 
stop
+        when(iterator.done()).thenReturn(null); // null means follower path
+        when(iterator.getData()).thenReturn(byteBuffer);
+
+        // Execute onApply
+        assertDoesNotThrow(() -> raftStateMachine.onApply(iterator));
+
+        // Verify iterator methods were called
+        verify(iterator, times(2)).hasNext();
+        verify(iterator).done();
+        verify(iterator).getData();
+        verify(iterator).next();
+    }
+
+    @Test
+    public void testOnApplyWithEmptyByteBuffer() {
+        // Test heartbeat event with empty ByteBuffer
+        Iterator iterator = mock(Iterator.class);
+        ByteBuffer emptyBuffer = ByteBuffer.allocate(0);
+
+        when(iterator.hasNext()).thenReturn(true, false);
+        when(iterator.done()).thenReturn(null);
+        when(iterator.getData()).thenReturn(emptyBuffer);
+
+        // Should not throw exception for empty buffer (heartbeat)
+        assertDoesNotThrow(() -> raftStateMachine.onApply(iterator));
+
+        verify(iterator).next();
+    }
+
+    @Test
+    public void testOnApplyWithNullByteBuffer() {
+        // Test with null ByteBuffer (heartbeat event)
+        Iterator iterator = mock(Iterator.class);
+
+        when(iterator.hasNext()).thenReturn(true, false);
+        when(iterator.done()).thenReturn(null);
+        when(iterator.getData()).thenReturn(null);
+
+        // Should not throw exception for null buffer
+        assertDoesNotThrow(() -> raftStateMachine.onApply(iterator));
+
+        verify(iterator).next();
+    }
+
+    @Test
+    public void testOnApplyWithLeaderPath() {
+        // Test the leader execution path where iterator.done() returns a 
Closure
+        Iterator iterator = mock(Iterator.class);
+        Closure done = mock(Closure.class);
+
+        when(iterator.hasNext()).thenReturn(true, false);
+        when(iterator.done()).thenReturn(done);
+
+        assertDoesNotThrow(() -> raftStateMachine.onApply(iterator));
+
+        // Verify done.run was called with OK status
+        verify(done).run(argThat(Status::isOk));
+        verify(iterator).next();
+    }
+
+    @Test
+    public void testOnExecuteRaftWithUnknownMessageType() throws Exception {
+        // Test error path when message type is not in EXECUTES map
+        // Create a custom message type that doesn't exist in EXECUTES
+        RaftBaseMsg msg = new RaftBaseMsg() {
+            @Override
+            public RaftSyncMsgType getMsgType() {
+                return null; // Unknown type
+            }
+        };
+
+        // Access the private method
+        java.lang.reflect.Method method = 
RaftStateMachine.class.getDeclaredMethod("onExecuteRaft", RaftBaseMsg.class);
+        method.setAccessible(true);
+
+        // Should throw RuntimeException for unknown message type
+        assertThrows(java.lang.reflect.InvocationTargetException.class, () -> {
+            method.invoke(raftStateMachine, msg);
+        });
+    }
+
+    @Test
+    public void testOnExecuteRaftWithExecutionException() throws Throwable {
+        // Test error path when execute.execute() throws an exception
+        RaftClusterMetadata metadata = new RaftClusterMetadata(1L);
+        RaftClusterMetadataMsg msg = new RaftClusterMetadataMsg(metadata);
+
+        // Access the EXECUTES map and add a mock that throws exception
+        Field executesField = 
RaftStateMachine.class.getDeclaredField("EXECUTES");
+        executesField.setAccessible(true);
+        @SuppressWarnings("unchecked")
+        Map<RaftSyncMsgType, RaftMsgExecute<?>> executes =
+                (Map<RaftSyncMsgType, RaftMsgExecute<?>>) 
executesField.get(null);
+
+        // Store original execute
+        RaftMsgExecute<?> originalExecute = 
executes.get(RaftSyncMsgType.REFRESH_CLUSTER_METADATA);
+
+        try {
+            // Replace with mock that throws exception
+            RaftMsgExecute<?> mockExecute = mock(RaftMsgExecute.class);
+            doThrow(new RuntimeException("Test 
exception")).when(mockExecute).execute(any());
+            executes.put(RaftSyncMsgType.REFRESH_CLUSTER_METADATA, 
mockExecute);
+
+            // Access the private method
+            java.lang.reflect.Method method =
+                    RaftStateMachine.class.getDeclaredMethod("onExecuteRaft", 
RaftBaseMsg.class);
+            method.setAccessible(true);
+
+            // Should wrap exception in RuntimeException
+            assertThrows(java.lang.reflect.InvocationTargetException.class, () 
-> {
+                method.invoke(raftStateMachine, msg);
+            });
+        } finally {
+            // Restore original execute
+            if (originalExecute != null) {
+                executes.put(RaftSyncMsgType.REFRESH_CLUSTER_METADATA, 
originalExecute);
+            }
+        }
+    }
+
+    @Test
+    public void testChangeNodeMetadataUpdatesExistingNode() {
+        // Test the path where an existing node is found and updated
+        RaftClusterMetadata metadata = new RaftClusterMetadata(1L);
+
+        // Create an existing follower with internal endpoint
+        Node existingFollower = new Node();
+        existingFollower.setRole(ClusterRole.FOLLOWER);
+        existingFollower.setGroup(TEST_GROUP);
+        Node.Endpoint endpoint = new Node.Endpoint();
+        endpoint.setHost("127.0.0.1");
+        endpoint.setPort(8091);
+        existingFollower.setInternal(endpoint);
+        existingFollower.setVersion("1.0.0");
+
+        metadata.setFollowers(Arrays.asList(existingFollower));
+        raftStateMachine.setRaftLeaderMetadata(metadata);
+
+        // Create a node update with same host/port but different metadata
+        Node updatedNode = new Node();
+        updatedNode.setRole(ClusterRole.FOLLOWER);
+        updatedNode.setGroup(TEST_GROUP);
+        Node.Endpoint updatedEndpoint = new Node.Endpoint();
+        updatedEndpoint.setHost("127.0.0.1");
+        updatedEndpoint.setPort(8091);
+        updatedNode.setInternal(updatedEndpoint);
+        updatedNode.setVersion("2.0.0");
+        updatedNode.setTransaction(new Node.Endpoint());
+        updatedNode.setControl(new Node.Endpoint());
+        updatedNode.setMetadata(Collections.singletonMap("key", "value"));
+
+        // Apply the update
+        raftStateMachine.changeNodeMetadata(updatedNode);
+
+        // Verify the existing node was updated
+        RaftClusterMetadata updatedMetadata = 
raftStateMachine.getRaftLeaderMetadata();
+        Node resultNode = updatedMetadata.getFollowers().get(0);
+        assertEquals("2.0.0", resultNode.getVersion());
+        assertNotNull(resultNode.getTransaction());
+        assertNotNull(resultNode.getControl());
+        assertNotNull(resultNode.getMetadata());
+        assertEquals(1, updatedMetadata.getFollowers().size()); // Should 
still be 1, not 2
+    }
+
+    @Test
+    public void testChangeNodeMetadataUpdatesExistingLearner() {
+        // Test updating an existing learner node
+        RaftClusterMetadata metadata = new RaftClusterMetadata(1L);
+
+        // Create an existing learner with internal endpoint
+        Node existingLearner = new Node();
+        existingLearner.setRole(ClusterRole.LEARNER);
+        existingLearner.setGroup(TEST_GROUP);
+        Node.Endpoint endpoint = new Node.Endpoint();
+        endpoint.setHost("127.0.0.1");
+        endpoint.setPort(8093);
+        existingLearner.setInternal(endpoint);
+        existingLearner.setVersion("1.0.0");
+
+        metadata.setLearner(Arrays.asList(existingLearner));
+        raftStateMachine.setRaftLeaderMetadata(metadata);
+
+        // Create a learner update with same host/port
+        Node updatedLearner = new Node();
+        updatedLearner.setRole(ClusterRole.LEARNER);
+        updatedLearner.setGroup(TEST_GROUP);
+        Node.Endpoint updatedEndpoint = new Node.Endpoint();
+        updatedEndpoint.setHost("127.0.0.1");
+        updatedEndpoint.setPort(8093);
+        updatedLearner.setInternal(updatedEndpoint);
+        updatedLearner.setVersion("2.0.0");
+
+        // Apply the update
+        raftStateMachine.changeNodeMetadata(updatedLearner);
+
+        // Verify the existing learner was updated
+        RaftClusterMetadata updatedMetadata = 
raftStateMachine.getRaftLeaderMetadata();
+        Node resultNode = updatedMetadata.getLearner().get(0);
+        assertEquals("2.0.0", resultNode.getVersion());
+        assertEquals(1, updatedMetadata.getLearner().size()); // Should still 
be 1, not 2
+    }
+}
diff --git 
a/server/src/test/java/org/apache/seata/server/raft/RaftSyncMessageTest.java 
b/server/src/test/java/org/apache/seata/server/cluster/raft/RaftSyncMessageTest.java
similarity index 99%
rename from 
server/src/test/java/org/apache/seata/server/raft/RaftSyncMessageTest.java
rename to 
server/src/test/java/org/apache/seata/server/cluster/raft/RaftSyncMessageTest.java
index cdf2704f73..3175230ea0 100644
--- a/server/src/test/java/org/apache/seata/server/raft/RaftSyncMessageTest.java
+++ 
b/server/src/test/java/org/apache/seata/server/cluster/raft/RaftSyncMessageTest.java
@@ -14,7 +14,7 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.apache.seata.server.raft;
+package org.apache.seata.server.cluster.raft;
 
 import org.apache.seata.common.exception.SeataRuntimeException;
 import org.apache.seata.common.metadata.ClusterRole;
diff --git 
a/server/src/test/java/org/apache/seata/server/raft/TestSecurity.java 
b/server/src/test/java/org/apache/seata/server/cluster/raft/TestSecurity.java
similarity index 95%
copy from server/src/test/java/org/apache/seata/server/raft/TestSecurity.java
copy to 
server/src/test/java/org/apache/seata/server/cluster/raft/TestSecurity.java
index 3e32c7e269..d17b43857d 100644
--- a/server/src/test/java/org/apache/seata/server/raft/TestSecurity.java
+++ 
b/server/src/test/java/org/apache/seata/server/cluster/raft/TestSecurity.java
@@ -14,7 +14,7 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.apache.seata.server.raft;
+package org.apache.seata.server.cluster.raft;
 
 public class TestSecurity implements java.io.Serializable {
 
diff --git 
a/server/src/test/java/org/apache/seata/server/cluster/raft/context/SeataClusterContextTest.java
 
b/server/src/test/java/org/apache/seata/server/cluster/raft/context/SeataClusterContextTest.java
new file mode 100644
index 0000000000..3a9b773a7e
--- /dev/null
+++ 
b/server/src/test/java/org/apache/seata/server/cluster/raft/context/SeataClusterContextTest.java
@@ -0,0 +1,167 @@
+/*
+ * 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.seata.server.cluster.raft.context;
+
+import org.apache.seata.server.BaseSpringBootTest;
+import org.junit.jupiter.api.AfterEach;
+import org.junit.jupiter.api.Test;
+
+import java.util.Arrays;
+
+import static org.junit.jupiter.api.Assertions.assertEquals;
+import static org.junit.jupiter.api.Assertions.assertNotNull;
+import static org.junit.jupiter.api.Assertions.assertNull;
+
+/**
+ * Unit tests for SeataClusterContext covering context management logic.
+ */
+public class SeataClusterContextTest extends BaseSpringBootTest {
+
+    @AfterEach
+    public void cleanup() {
+        // Clean up context after each test
+        SeataClusterContext.unbindGroup();
+    }
+
+    @Test
+    public void testBindGroupWithCustomValue() {
+        String customGroup = "custom-group";
+        SeataClusterContext.bindGroup(customGroup);
+
+        String retrievedGroup = SeataClusterContext.getGroup();
+        assertEquals(customGroup, retrievedGroup);
+    }
+
+    @Test
+    public void testBindGroupWithDefaultValue() {
+        String defaultGroup = SeataClusterContext.bindGroup();
+
+        assertNotNull(defaultGroup);
+        assertEquals(defaultGroup, SeataClusterContext.getGroup());
+    }
+
+    @Test
+    public void testBindGroupReturnsDefaultGroup() {
+        String group = SeataClusterContext.bindGroup();
+
+        // Default group should be "default" based on DEFAULT_SEATA_GROUP
+        assertNotNull(group);
+        assertEquals("default", group);
+    }
+
+    @Test
+    public void testUnbindGroupRemovesContext() {
+        SeataClusterContext.bindGroup("test-group");
+        assertEquals("test-group", SeataClusterContext.getGroup());
+
+        SeataClusterContext.unbindGroup();
+        assertNull(SeataClusterContext.getGroup());
+    }
+
+    @Test
+    public void testGetGroupReturnsNullWhenNotBound() {
+        assertNull(SeataClusterContext.getGroup());
+    }
+
+    @Test
+    public void testBindGroupOverwritesPreviousValue() {
+        SeataClusterContext.bindGroup("first-group");
+        assertEquals("first-group", SeataClusterContext.getGroup());
+
+        SeataClusterContext.bindGroup("second-group");
+        assertEquals("second-group", SeataClusterContext.getGroup());
+    }
+
+    @Test
+    public void testBindCustomGroupThenBindDefault() {
+        SeataClusterContext.bindGroup("custom-group");
+        assertEquals("custom-group", SeataClusterContext.getGroup());
+
+        String defaultGroup = SeataClusterContext.bindGroup();
+        assertEquals(defaultGroup, SeataClusterContext.getGroup());
+        assertEquals("default", defaultGroup);
+    }
+
+    @Test
+    public void testBindDefaultGroupThenBindCustom() {
+        SeataClusterContext.bindGroup();
+        assertEquals("default", SeataClusterContext.getGroup());
+
+        SeataClusterContext.bindGroup("custom-group");
+        assertEquals("custom-group", SeataClusterContext.getGroup());
+    }
+
+    @Test
+    public void testUnbindMultipleTimes() {
+        SeataClusterContext.bindGroup("test-group");
+
+        SeataClusterContext.unbindGroup();
+        assertNull(SeataClusterContext.getGroup());
+
+        // Unbinding again should not throw exception
+        SeataClusterContext.unbindGroup();
+        assertNull(SeataClusterContext.getGroup());
+    }
+
+    @Test
+    public void testBindEmptyString() {
+        SeataClusterContext.bindGroup("");
+        assertEquals("", SeataClusterContext.getGroup());
+    }
+
+    @Test
+    public void testBindGroupWithWhitespace() {
+        SeataClusterContext.bindGroup("  group-with-spaces  ");
+        assertEquals("  group-with-spaces  ", SeataClusterContext.getGroup());
+    }
+
+    @Test
+    public void testContextIsolationBetweenOperations() {
+        // First operation
+        SeataClusterContext.bindGroup("operation1");
+        String group1 = SeataClusterContext.getGroup();
+        assertEquals("operation1", group1);
+        SeataClusterContext.unbindGroup();
+
+        // Second operation should start clean
+        assertNull(SeataClusterContext.getGroup());
+        SeataClusterContext.bindGroup("operation2");
+        String group2 = SeataClusterContext.getGroup();
+        assertEquals("operation2", group2);
+    }
+
+    @Test
+    public void testKeyGroupConstant() {
+        assertEquals("TX_GROUP", SeataClusterContext.KEY_GROUP);
+    }
+
+    @Test
+    public void testBindGroupWithSpecialCharacters() {
+        String specialGroup = "group-with-@#$%^&*()";
+        SeataClusterContext.bindGroup(specialGroup);
+        assertEquals(specialGroup, SeataClusterContext.getGroup());
+    }
+
+    @Test
+    public void testBindGroupWithVeryLongName() {
+        char[] chars = new char[100];
+        Arrays.fill(chars, 'a');
+        String longGroup = "very-long-group-name-" + new String(chars);
+        SeataClusterContext.bindGroup(longGroup);
+        assertEquals(longGroup, SeataClusterContext.getGroup());
+    }
+}
diff --git 
a/server/src/test/java/org/apache/seata/server/raft/execute/BranchSessionExecuteTest.java
 
b/server/src/test/java/org/apache/seata/server/cluster/raft/execute/BranchSessionExecuteTest.java
similarity index 99%
rename from 
server/src/test/java/org/apache/seata/server/raft/execute/BranchSessionExecuteTest.java
rename to 
server/src/test/java/org/apache/seata/server/cluster/raft/execute/BranchSessionExecuteTest.java
index 3a6ba2653e..cbcaf1445a 100644
--- 
a/server/src/test/java/org/apache/seata/server/raft/execute/BranchSessionExecuteTest.java
+++ 
b/server/src/test/java/org/apache/seata/server/cluster/raft/execute/BranchSessionExecuteTest.java
@@ -14,7 +14,7 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.apache.seata.server.raft.execute;
+package org.apache.seata.server.cluster.raft.execute;
 
 import org.apache.seata.common.XID;
 import org.apache.seata.common.store.LockMode;
diff --git 
a/server/src/test/java/org/apache/seata/server/raft/execute/GlobalSessionExecuteTest.java
 
b/server/src/test/java/org/apache/seata/server/cluster/raft/execute/GlobalSessionExecuteTest.java
similarity index 99%
rename from 
server/src/test/java/org/apache/seata/server/raft/execute/GlobalSessionExecuteTest.java
rename to 
server/src/test/java/org/apache/seata/server/cluster/raft/execute/GlobalSessionExecuteTest.java
index 2f144b21b0..25c7840649 100644
--- 
a/server/src/test/java/org/apache/seata/server/raft/execute/GlobalSessionExecuteTest.java
+++ 
b/server/src/test/java/org/apache/seata/server/cluster/raft/execute/GlobalSessionExecuteTest.java
@@ -14,7 +14,7 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.apache.seata.server.raft.execute;
+package org.apache.seata.server.cluster.raft.execute;
 
 import org.apache.seata.common.store.LockMode;
 import org.apache.seata.common.store.SessionMode;
diff --git 
a/server/src/test/java/org/apache/seata/server/raft/execute/LockExecuteTest.java
 
b/server/src/test/java/org/apache/seata/server/cluster/raft/execute/LockExecuteTest.java
similarity index 99%
rename from 
server/src/test/java/org/apache/seata/server/raft/execute/LockExecuteTest.java
rename to 
server/src/test/java/org/apache/seata/server/cluster/raft/execute/LockExecuteTest.java
index 0c45b9363e..2fbcb74f4e 100644
--- 
a/server/src/test/java/org/apache/seata/server/raft/execute/LockExecuteTest.java
+++ 
b/server/src/test/java/org/apache/seata/server/cluster/raft/execute/LockExecuteTest.java
@@ -14,7 +14,7 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.apache.seata.server.raft.execute;
+package org.apache.seata.server.cluster.raft.execute;
 
 import org.apache.seata.common.store.LockMode;
 import org.apache.seata.common.store.SessionMode;
diff --git 
a/server/src/test/java/org/apache/seata/server/raft/TestSecurity.java 
b/server/src/test/java/org/apache/seata/server/cluster/raft/processor/PutNodeInfoRequestProcessorTest.java
similarity index 53%
rename from server/src/test/java/org/apache/seata/server/raft/TestSecurity.java
rename to 
server/src/test/java/org/apache/seata/server/cluster/raft/processor/PutNodeInfoRequestProcessorTest.java
index 3e32c7e269..053324e9a4 100644
--- a/server/src/test/java/org/apache/seata/server/raft/TestSecurity.java
+++ 
b/server/src/test/java/org/apache/seata/server/cluster/raft/processor/PutNodeInfoRequestProcessorTest.java
@@ -14,19 +14,27 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.apache.seata.server.raft;
+package org.apache.seata.server.cluster.raft.processor;
 
-public class TestSecurity implements java.io.Serializable {
+import 
org.apache.seata.server.cluster.raft.processor.request.PutNodeMetadataRequest;
+import org.junit.jupiter.api.Test;
 
-    private static final long serialVersionUID = 543214259201495900L;
+import static org.junit.jupiter.api.Assertions.*;
 
-    String a = "test";
+public class PutNodeInfoRequestProcessorTest {
 
-    public String getA() {
-        return a;
+    @Test
+    public void testConstructor() {
+        PutNodeInfoRequestProcessor processor = new 
PutNodeInfoRequestProcessor();
+        assertNotNull(processor);
     }
 
-    public void setA(String a) {
-        this.a = a;
+    @Test
+    public void testInterest() {
+        PutNodeInfoRequestProcessor processor = new 
PutNodeInfoRequestProcessor();
+        String interest = processor.interest();
+
+        assertNotNull(interest);
+        assertEquals(PutNodeMetadataRequest.class.getName(), interest);
     }
 }
diff --git 
a/server/src/test/java/org/apache/seata/server/cluster/raft/processor/request/PutNodeMetadataRequestTest.java
 
b/server/src/test/java/org/apache/seata/server/cluster/raft/processor/request/PutNodeMetadataRequestTest.java
new file mode 100644
index 0000000000..29aff86b0d
--- /dev/null
+++ 
b/server/src/test/java/org/apache/seata/server/cluster/raft/processor/request/PutNodeMetadataRequestTest.java
@@ -0,0 +1,60 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.seata.server.cluster.raft.processor.request;
+
+import org.apache.seata.common.metadata.Node;
+import org.junit.jupiter.api.Test;
+
+import static org.junit.jupiter.api.Assertions.*;
+
+public class PutNodeMetadataRequestTest {
+
+    @Test
+    public void testDefaultConstructor() {
+        PutNodeMetadataRequest request = new PutNodeMetadataRequest();
+        assertNotNull(request);
+        assertNull(request.getNode());
+    }
+
+    @Test
+    public void testConstructorWithNode() {
+        Node node = new Node();
+        PutNodeMetadataRequest request = new PutNodeMetadataRequest(node);
+
+        assertNotNull(request);
+        assertEquals(node, request.getNode());
+    }
+
+    @Test
+    public void testSetAndGetNode() {
+        PutNodeMetadataRequest request = new PutNodeMetadataRequest();
+        Node node = new Node();
+        request.setNode(node);
+
+        assertEquals(node, request.getNode());
+    }
+
+    @Test
+    public void testSetNodeToNull() {
+        Node node = new Node();
+        PutNodeMetadataRequest request = new PutNodeMetadataRequest(node);
+        assertNotNull(request.getNode());
+
+        request.setNode(null);
+        assertNull(request.getNode());
+    }
+}
diff --git 
a/server/src/test/java/org/apache/seata/server/cluster/raft/processor/response/PutNodeMetadataResponseTest.java
 
b/server/src/test/java/org/apache/seata/server/cluster/raft/processor/response/PutNodeMetadataResponseTest.java
new file mode 100644
index 0000000000..36a9a91d50
--- /dev/null
+++ 
b/server/src/test/java/org/apache/seata/server/cluster/raft/processor/response/PutNodeMetadataResponseTest.java
@@ -0,0 +1,65 @@
+/*
+ * 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.seata.server.cluster.raft.processor.response;
+
+import org.junit.jupiter.api.Test;
+
+import static org.junit.jupiter.api.Assertions.*;
+
+public class PutNodeMetadataResponseTest {
+
+    @Test
+    public void testConstructorWithTrue() {
+        PutNodeMetadataResponse response = new PutNodeMetadataResponse(true);
+        assertTrue(response.isSuccess());
+    }
+
+    @Test
+    public void testConstructorWithFalse() {
+        PutNodeMetadataResponse response = new PutNodeMetadataResponse(false);
+        assertFalse(response.isSuccess());
+    }
+
+    @Test
+    public void testSetSuccess() {
+        PutNodeMetadataResponse response = new PutNodeMetadataResponse(false);
+        assertFalse(response.isSuccess());
+
+        response.setSuccess(true);
+        assertTrue(response.isSuccess());
+    }
+
+    @Test
+    public void testToString() {
+        PutNodeMetadataResponse response = new PutNodeMetadataResponse(true);
+        String str = response.toString();
+
+        assertNotNull(str);
+        assertTrue(str.contains("success"));
+        assertTrue(str.contains("true"));
+    }
+
+    @Test
+    public void testToStringWithFalse() {
+        PutNodeMetadataResponse response = new PutNodeMetadataResponse(false);
+        String str = response.toString();
+
+        assertNotNull(str);
+        assertTrue(str.contains("success"));
+        assertTrue(str.contains("false"));
+    }
+}
diff --git 
a/server/src/test/java/org/apache/seata/server/cluster/raft/serializer/JacksonSerializerTest.java
 
b/server/src/test/java/org/apache/seata/server/cluster/raft/serializer/JacksonSerializerTest.java
new file mode 100644
index 0000000000..067537c451
--- /dev/null
+++ 
b/server/src/test/java/org/apache/seata/server/cluster/raft/serializer/JacksonSerializerTest.java
@@ -0,0 +1,80 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.seata.server.cluster.raft.serializer;
+
+import org.apache.seata.server.cluster.raft.sync.msg.RaftBaseMsg;
+import org.apache.seata.server.cluster.raft.sync.msg.RaftSyncMsgType;
+import org.apache.seata.server.cluster.raft.sync.msg.dto.BranchTransactionDTO;
+import org.apache.seata.server.cluster.raft.sync.msg.dto.GlobalTransactionDTO;
+import org.junit.jupiter.api.Test;
+
+import static org.junit.jupiter.api.Assertions.*;
+
+public class JacksonSerializerTest {
+
+    private final JacksonSerializer serializer = new JacksonSerializer();
+
+    @Test
+    public void testSerializeAndDeserializeSimpleObject() {
+        RaftBaseMsg original = new RaftBaseMsg();
+        original.setMsgType(RaftSyncMsgType.ADD_GLOBAL_SESSION);
+        original.setGroup("test-group");
+
+        byte[] bytes = serializer.serialize(original);
+        assertNotNull(bytes);
+        assertTrue(bytes.length > 0);
+
+        RaftBaseMsg deserialized = serializer.deserialize(bytes);
+        assertNotNull(deserialized);
+        assertEquals(original.getMsgType(), deserialized.getMsgType());
+        assertEquals(original.getGroup(), deserialized.getGroup());
+    }
+
+    @Test
+    public void testSerializeAndDeserializeBranchTransactionDTO() {
+        BranchTransactionDTO original = new BranchTransactionDTO("xid:123", 
456L);
+        original.setLockKey("table:1,2,3");
+
+        byte[] bytes = serializer.serialize(original);
+        assertNotNull(bytes);
+
+        BranchTransactionDTO deserialized = serializer.deserialize(bytes);
+        assertNotNull(deserialized);
+        assertEquals(original.getXid(), deserialized.getXid());
+        assertEquals(original.getBranchId(), deserialized.getBranchId());
+        assertEquals(original.getLockKey(), deserialized.getLockKey());
+    }
+
+    @Test
+    public void testSerializeAndDeserializeGlobalTransactionDTO() {
+        GlobalTransactionDTO original = new GlobalTransactionDTO("xid:789");
+
+        byte[] bytes = serializer.serialize(original);
+        assertNotNull(bytes);
+
+        GlobalTransactionDTO deserialized = serializer.deserialize(bytes);
+        assertNotNull(deserialized);
+        assertEquals(original.getXid(), deserialized.getXid());
+    }
+
+    @Test
+    public void testSerializeNull() {
+        assertThrows(RuntimeException.class, () -> {
+            serializer.serialize(null);
+        });
+    }
+}
diff --git 
a/server/src/test/java/org/apache/seata/server/cluster/raft/sync/msg/RaftBaseMsgTest.java
 
b/server/src/test/java/org/apache/seata/server/cluster/raft/sync/msg/RaftBaseMsgTest.java
new file mode 100644
index 0000000000..3bc50d65e5
--- /dev/null
+++ 
b/server/src/test/java/org/apache/seata/server/cluster/raft/sync/msg/RaftBaseMsgTest.java
@@ -0,0 +1,74 @@
+/*
+ * 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.seata.server.cluster.raft.sync.msg;
+
+import org.junit.jupiter.api.Test;
+
+import java.io.ByteArrayInputStream;
+import java.io.ByteArrayOutputStream;
+import java.io.ObjectInputStream;
+import java.io.ObjectOutputStream;
+
+import static org.apache.seata.common.DefaultValues.DEFAULT_SEATA_GROUP;
+import static org.junit.jupiter.api.Assertions.*;
+
+public class RaftBaseMsgTest {
+
+    @Test
+    public void testDefaultValues() {
+        RaftBaseMsg msg = new RaftBaseMsg();
+        assertEquals(DEFAULT_SEATA_GROUP, msg.getGroup());
+        assertNull(msg.getMsgType());
+    }
+
+    @Test
+    public void testSetAndGetMsgType() {
+        RaftBaseMsg msg = new RaftBaseMsg();
+        msg.setMsgType(RaftSyncMsgType.ADD_GLOBAL_SESSION);
+        assertEquals(RaftSyncMsgType.ADD_GLOBAL_SESSION, msg.getMsgType());
+    }
+
+    @Test
+    public void testSetAndGetGroup() {
+        RaftBaseMsg msg = new RaftBaseMsg();
+        msg.setGroup("test-group");
+        assertEquals("test-group", msg.getGroup());
+    }
+
+    @Test
+    public void testSerialization() throws Exception {
+        RaftBaseMsg original = new RaftBaseMsg();
+        original.setMsgType(RaftSyncMsgType.ADD_BRANCH_SESSION);
+        original.setGroup("custom-group");
+
+        // Serialize
+        ByteArrayOutputStream bos = new ByteArrayOutputStream();
+        ObjectOutputStream oos = new ObjectOutputStream(bos);
+        oos.writeObject(original);
+        oos.flush();
+        byte[] serialized = bos.toByteArray();
+
+        // Deserialize
+        ByteArrayInputStream bis = new ByteArrayInputStream(serialized);
+        ObjectInputStream ois = new ObjectInputStream(bis);
+        RaftBaseMsg deserialized = (RaftBaseMsg) ois.readObject();
+
+        // Verify
+        assertEquals(original.getMsgType(), deserialized.getMsgType());
+        assertEquals(original.getGroup(), deserialized.getGroup());
+    }
+}
diff --git 
a/server/src/test/java/org/apache/seata/server/cluster/raft/sync/msg/RaftBranchSessionSyncMsgTest.java
 
b/server/src/test/java/org/apache/seata/server/cluster/raft/sync/msg/RaftBranchSessionSyncMsgTest.java
new file mode 100644
index 0000000000..cb91c49c4a
--- /dev/null
+++ 
b/server/src/test/java/org/apache/seata/server/cluster/raft/sync/msg/RaftBranchSessionSyncMsgTest.java
@@ -0,0 +1,105 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.seata.server.cluster.raft.sync.msg;
+
+import org.apache.seata.server.cluster.raft.sync.msg.dto.BranchTransactionDTO;
+import org.junit.jupiter.api.Test;
+
+import java.io.ByteArrayInputStream;
+import java.io.ByteArrayOutputStream;
+import java.io.ObjectInputStream;
+import java.io.ObjectOutputStream;
+
+import static org.apache.seata.common.DefaultValues.DEFAULT_SEATA_GROUP;
+import static org.junit.jupiter.api.Assertions.*;
+
+public class RaftBranchSessionSyncMsgTest {
+
+    @Test
+    public void testDefaultConstructor() {
+        RaftBranchSessionSyncMsg msg = new RaftBranchSessionSyncMsg();
+        assertNotNull(msg);
+        assertEquals(DEFAULT_SEATA_GROUP, msg.getGroup());
+        assertNull(msg.getBranchSession());
+        assertNull(msg.getMsgType());
+    }
+
+    @Test
+    public void testConstructorWithParameters() {
+        BranchTransactionDTO dto = new BranchTransactionDTO("xid:123", 456L);
+        RaftBranchSessionSyncMsg msg = new 
RaftBranchSessionSyncMsg(RaftSyncMsgType.ADD_BRANCH_SESSION, dto);
+
+        assertEquals(RaftSyncMsgType.ADD_BRANCH_SESSION, msg.getMsgType());
+        assertEquals(dto, msg.getBranchSession());
+    }
+
+    @Test
+    public void testSetAndGetBranchSession() {
+        RaftBranchSessionSyncMsg msg = new RaftBranchSessionSyncMsg();
+        BranchTransactionDTO dto = new BranchTransactionDTO("xid:789", 101L);
+        msg.setBranchSession(dto);
+
+        assertEquals(dto, msg.getBranchSession());
+    }
+
+    @Test
+    public void testSetAndGetGroup() {
+        RaftBranchSessionSyncMsg msg = new RaftBranchSessionSyncMsg();
+        msg.setGroup("custom-group");
+        assertEquals("custom-group", msg.getGroup());
+    }
+
+    @Test
+    public void testToString() {
+        BranchTransactionDTO dto = new BranchTransactionDTO("xid:123", 456L);
+        RaftBranchSessionSyncMsg msg = new 
RaftBranchSessionSyncMsg(RaftSyncMsgType.UPDATE_BRANCH_SESSION_STATUS, dto);
+
+        String str = msg.toString();
+        assertNotNull(str);
+        assertFalse(str.isEmpty());
+    }
+
+    @Test
+    public void testSerialization() throws Exception {
+        BranchTransactionDTO dto = new BranchTransactionDTO("xid:123", 456L);
+        dto.setLockKey("table:1,2,3");
+        RaftBranchSessionSyncMsg original = new 
RaftBranchSessionSyncMsg(RaftSyncMsgType.ADD_BRANCH_SESSION, dto);
+        original.setGroup("test-group");
+
+        // Serialize
+        ByteArrayOutputStream bos = new ByteArrayOutputStream();
+        ObjectOutputStream oos = new ObjectOutputStream(bos);
+        oos.writeObject(original);
+        oos.flush();
+        byte[] serialized = bos.toByteArray();
+
+        // Deserialize
+        ByteArrayInputStream bis = new ByteArrayInputStream(serialized);
+        ObjectInputStream ois = new ObjectInputStream(bis);
+        RaftBranchSessionSyncMsg deserialized = (RaftBranchSessionSyncMsg) 
ois.readObject();
+
+        // Verify
+        assertEquals(original.getMsgType(), deserialized.getMsgType());
+        assertEquals(original.getGroup(), deserialized.getGroup());
+        assertEquals(
+                original.getBranchSession().getXid(),
+                deserialized.getBranchSession().getXid());
+        assertEquals(
+                original.getBranchSession().getBranchId(),
+                deserialized.getBranchSession().getBranchId());
+    }
+}
diff --git 
a/server/src/test/java/org/apache/seata/server/cluster/raft/sync/msg/RaftClusterMetadataMsgTest.java
 
b/server/src/test/java/org/apache/seata/server/cluster/raft/sync/msg/RaftClusterMetadataMsgTest.java
new file mode 100644
index 0000000000..3d86259aa0
--- /dev/null
+++ 
b/server/src/test/java/org/apache/seata/server/cluster/raft/sync/msg/RaftClusterMetadataMsgTest.java
@@ -0,0 +1,89 @@
+/*
+ * 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.seata.server.cluster.raft.sync.msg;
+
+import org.apache.seata.server.cluster.raft.sync.msg.dto.RaftClusterMetadata;
+import org.junit.jupiter.api.Test;
+
+import java.io.ByteArrayInputStream;
+import java.io.ByteArrayOutputStream;
+import java.io.ObjectInputStream;
+import java.io.ObjectOutputStream;
+
+import static org.junit.jupiter.api.Assertions.*;
+
+public class RaftClusterMetadataMsgTest {
+
+    @Test
+    public void testDefaultConstructor() {
+        RaftClusterMetadataMsg msg = new RaftClusterMetadataMsg();
+        assertNotNull(msg);
+        assertNull(msg.getRaftClusterMetadata());
+    }
+
+    @Test
+    public void testConstructorWithMetadata() {
+        RaftClusterMetadata metadata = new RaftClusterMetadata(123L);
+        RaftClusterMetadataMsg msg = new RaftClusterMetadataMsg(metadata);
+
+        assertEquals(RaftSyncMsgType.REFRESH_CLUSTER_METADATA, 
msg.getMsgType());
+        assertEquals(metadata, msg.getRaftClusterMetadata());
+    }
+
+    @Test
+    public void testSetAndGetRaftClusterMetadata() {
+        RaftClusterMetadataMsg msg = new RaftClusterMetadataMsg();
+        RaftClusterMetadata metadata = new RaftClusterMetadata(456L);
+        msg.setRaftClusterMetadata(metadata);
+
+        assertEquals(metadata, msg.getRaftClusterMetadata());
+    }
+
+    @Test
+    public void testToString() {
+        RaftClusterMetadata metadata = new RaftClusterMetadata(789L);
+        RaftClusterMetadataMsg msg = new RaftClusterMetadataMsg(metadata);
+
+        String str = msg.toString();
+        assertNotNull(str);
+        assertFalse(str.isEmpty());
+    }
+
+    @Test
+    public void testSerialization() throws Exception {
+        RaftClusterMetadata metadata = new RaftClusterMetadata(123L);
+        RaftClusterMetadataMsg original = new RaftClusterMetadataMsg(metadata);
+
+        // Serialize
+        ByteArrayOutputStream bos = new ByteArrayOutputStream();
+        ObjectOutputStream oos = new ObjectOutputStream(bos);
+        oos.writeObject(original);
+        oos.flush();
+        byte[] serialized = bos.toByteArray();
+
+        // Deserialize
+        ByteArrayInputStream bis = new ByteArrayInputStream(serialized);
+        ObjectInputStream ois = new ObjectInputStream(bis);
+        RaftClusterMetadataMsg deserialized = (RaftClusterMetadataMsg) 
ois.readObject();
+
+        // Verify
+        assertEquals(original.getMsgType(), deserialized.getMsgType());
+        assertEquals(
+                original.getRaftClusterMetadata().getTerm(),
+                deserialized.getRaftClusterMetadata().getTerm());
+    }
+}
diff --git 
a/server/src/test/java/org/apache/seata/server/cluster/raft/sync/msg/RaftGlobalSessionSyncMsgTest.java
 
b/server/src/test/java/org/apache/seata/server/cluster/raft/sync/msg/RaftGlobalSessionSyncMsgTest.java
new file mode 100644
index 0000000000..f4eb1d47ae
--- /dev/null
+++ 
b/server/src/test/java/org/apache/seata/server/cluster/raft/sync/msg/RaftGlobalSessionSyncMsgTest.java
@@ -0,0 +1,91 @@
+/*
+ * 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.seata.server.cluster.raft.sync.msg;
+
+import org.apache.seata.server.cluster.raft.sync.msg.dto.GlobalTransactionDTO;
+import org.junit.jupiter.api.Test;
+
+import java.io.ByteArrayInputStream;
+import java.io.ByteArrayOutputStream;
+import java.io.ObjectInputStream;
+import java.io.ObjectOutputStream;
+
+import static org.junit.jupiter.api.Assertions.*;
+
+public class RaftGlobalSessionSyncMsgTest {
+
+    @Test
+    public void testDefaultConstructor() {
+        RaftGlobalSessionSyncMsg msg = new RaftGlobalSessionSyncMsg();
+        assertNotNull(msg);
+        assertNull(msg.getGlobalSession());
+        assertNull(msg.getMsgType());
+    }
+
+    @Test
+    public void testConstructorWithParameters() {
+        GlobalTransactionDTO dto = new GlobalTransactionDTO("xid:123456");
+        RaftGlobalSessionSyncMsg msg = new 
RaftGlobalSessionSyncMsg(RaftSyncMsgType.ADD_GLOBAL_SESSION, dto);
+
+        assertEquals(RaftSyncMsgType.ADD_GLOBAL_SESSION, msg.getMsgType());
+        assertEquals(dto, msg.getGlobalSession());
+    }
+
+    @Test
+    public void testSetAndGetGlobalSession() {
+        RaftGlobalSessionSyncMsg msg = new RaftGlobalSessionSyncMsg();
+        GlobalTransactionDTO dto = new GlobalTransactionDTO("xid:789");
+        msg.setGlobalSession(dto);
+
+        assertEquals(dto, msg.getGlobalSession());
+    }
+
+    @Test
+    public void testToString() {
+        GlobalTransactionDTO dto = new GlobalTransactionDTO("xid:123");
+        RaftGlobalSessionSyncMsg msg = new 
RaftGlobalSessionSyncMsg(RaftSyncMsgType.REMOVE_GLOBAL_SESSION, dto);
+
+        String str = msg.toString();
+        assertNotNull(str);
+        assertFalse(str.isEmpty());
+    }
+
+    @Test
+    public void testSerialization() throws Exception {
+        GlobalTransactionDTO dto = new GlobalTransactionDTO("xid:123456");
+        RaftGlobalSessionSyncMsg original =
+                new 
RaftGlobalSessionSyncMsg(RaftSyncMsgType.UPDATE_GLOBAL_SESSION_STATUS, dto);
+
+        // Serialize
+        ByteArrayOutputStream bos = new ByteArrayOutputStream();
+        ObjectOutputStream oos = new ObjectOutputStream(bos);
+        oos.writeObject(original);
+        oos.flush();
+        byte[] serialized = bos.toByteArray();
+
+        // Deserialize
+        ByteArrayInputStream bis = new ByteArrayInputStream(serialized);
+        ObjectInputStream ois = new ObjectInputStream(bis);
+        RaftGlobalSessionSyncMsg deserialized = (RaftGlobalSessionSyncMsg) 
ois.readObject();
+
+        // Verify
+        assertEquals(original.getMsgType(), deserialized.getMsgType());
+        assertEquals(
+                original.getGlobalSession().getXid(),
+                deserialized.getGlobalSession().getXid());
+    }
+}
diff --git 
a/server/src/test/java/org/apache/seata/server/cluster/raft/sync/msg/RaftVGroupSyncMsgTest.java
 
b/server/src/test/java/org/apache/seata/server/cluster/raft/sync/msg/RaftVGroupSyncMsgTest.java
new file mode 100644
index 0000000000..d7047b4073
--- /dev/null
+++ 
b/server/src/test/java/org/apache/seata/server/cluster/raft/sync/msg/RaftVGroupSyncMsgTest.java
@@ -0,0 +1,69 @@
+/*
+ * 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.seata.server.cluster.raft.sync.msg;
+
+import org.apache.seata.core.store.MappingDO;
+import org.junit.jupiter.api.Test;
+
+import static org.junit.jupiter.api.Assertions.*;
+
+public class RaftVGroupSyncMsgTest {
+
+    @Test
+    public void testDefaultConstructor() {
+        RaftVGroupSyncMsg msg = new RaftVGroupSyncMsg();
+        assertNotNull(msg);
+        assertNull(msg.getMappingDO());
+        assertNull(msg.getMsgType());
+    }
+
+    @Test
+    public void testConstructorWithParameters() {
+        MappingDO mappingDO = new MappingDO();
+        mappingDO.setVGroup("vgroup");
+        mappingDO.setCluster("cluster");
+        RaftVGroupSyncMsg msg = new RaftVGroupSyncMsg(mappingDO, 
RaftSyncMsgType.ADD_VGROUP_MAPPING);
+
+        assertEquals(RaftSyncMsgType.ADD_VGROUP_MAPPING, msg.getMsgType());
+        assertEquals(mappingDO, msg.getMappingDO());
+    }
+
+    @Test
+    public void testSetAndGetMappingDO() {
+        RaftVGroupSyncMsg msg = new RaftVGroupSyncMsg();
+        MappingDO mappingDO = new MappingDO();
+        mappingDO.setVGroup("vgroup2");
+        mappingDO.setCluster("cluster2");
+        msg.setMappingDO(mappingDO);
+
+        assertEquals(mappingDO, msg.getMappingDO());
+    }
+
+    @Test
+    public void testGettersAndSetters() {
+        RaftVGroupSyncMsg msg = new RaftVGroupSyncMsg();
+        MappingDO mappingDO = new MappingDO();
+        mappingDO.setVGroup("test-vgroup");
+        mappingDO.setCluster("test-cluster");
+        msg.setMappingDO(mappingDO);
+        msg.setMsgType(RaftSyncMsgType.UPDATE_VGROUP_MAPPING);
+
+        assertEquals(RaftSyncMsgType.UPDATE_VGROUP_MAPPING, msg.getMsgType());
+        assertEquals("test-vgroup", msg.getMappingDO().getVGroup());
+        assertEquals("test-cluster", msg.getMappingDO().getCluster());
+    }
+}
diff --git 
a/server/src/test/java/org/apache/seata/server/cluster/raft/sync/msg/dto/BranchTransactionDTOTest.java
 
b/server/src/test/java/org/apache/seata/server/cluster/raft/sync/msg/dto/BranchTransactionDTOTest.java
new file mode 100644
index 0000000000..74adc50026
--- /dev/null
+++ 
b/server/src/test/java/org/apache/seata/server/cluster/raft/sync/msg/dto/BranchTransactionDTOTest.java
@@ -0,0 +1,78 @@
+/*
+ * 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.seata.server.cluster.raft.sync.msg.dto;
+
+import org.junit.jupiter.api.Test;
+
+import java.io.ByteArrayInputStream;
+import java.io.ByteArrayOutputStream;
+import java.io.ObjectInputStream;
+import java.io.ObjectOutputStream;
+
+import static org.junit.jupiter.api.Assertions.*;
+
+public class BranchTransactionDTOTest {
+
+    @Test
+    public void testDefaultConstructor() {
+        BranchTransactionDTO dto = new BranchTransactionDTO();
+        assertNotNull(dto);
+        assertNull(dto.getLockKey());
+    }
+
+    @Test
+    public void testConstructorWithParameters() {
+        String xid = "192.168.1.1:8091:12345678";
+        long branchId = 123456L;
+        BranchTransactionDTO dto = new BranchTransactionDTO(xid, branchId);
+
+        assertEquals(xid, dto.getXid());
+        assertEquals(branchId, dto.getBranchId());
+        assertNull(dto.getLockKey());
+    }
+
+    @Test
+    public void testSetAndGetLockKey() {
+        BranchTransactionDTO dto = new BranchTransactionDTO();
+        String lockKey = "table:1,2,3";
+        dto.setLockKey(lockKey);
+        assertEquals(lockKey, dto.getLockKey());
+    }
+
+    @Test
+    public void testSerialization() throws Exception {
+        BranchTransactionDTO original = new BranchTransactionDTO("xid:123", 
456L);
+        original.setLockKey("table:lock:keys");
+
+        // Serialize
+        ByteArrayOutputStream bos = new ByteArrayOutputStream();
+        ObjectOutputStream oos = new ObjectOutputStream(bos);
+        oos.writeObject(original);
+        oos.flush();
+        byte[] serialized = bos.toByteArray();
+
+        // Deserialize
+        ByteArrayInputStream bis = new ByteArrayInputStream(serialized);
+        ObjectInputStream ois = new ObjectInputStream(bis);
+        BranchTransactionDTO deserialized = (BranchTransactionDTO) 
ois.readObject();
+
+        // Verify
+        assertEquals(original.getXid(), deserialized.getXid());
+        assertEquals(original.getBranchId(), deserialized.getBranchId());
+        assertEquals(original.getLockKey(), deserialized.getLockKey());
+    }
+}
diff --git 
a/server/src/test/java/org/apache/seata/server/cluster/raft/sync/msg/dto/GlobalTransactionDTOTest.java
 
b/server/src/test/java/org/apache/seata/server/cluster/raft/sync/msg/dto/GlobalTransactionDTOTest.java
new file mode 100644
index 0000000000..aae894697c
--- /dev/null
+++ 
b/server/src/test/java/org/apache/seata/server/cluster/raft/sync/msg/dto/GlobalTransactionDTOTest.java
@@ -0,0 +1,63 @@
+/*
+ * 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.seata.server.cluster.raft.sync.msg.dto;
+
+import org.junit.jupiter.api.Test;
+
+import java.io.ByteArrayInputStream;
+import java.io.ByteArrayOutputStream;
+import java.io.ObjectInputStream;
+import java.io.ObjectOutputStream;
+
+import static org.junit.jupiter.api.Assertions.*;
+
+public class GlobalTransactionDTOTest {
+
+    @Test
+    public void testDefaultConstructor() {
+        GlobalTransactionDTO dto = new GlobalTransactionDTO();
+        assertNotNull(dto);
+    }
+
+    @Test
+    public void testConstructorWithXid() {
+        String xid = "192.168.1.1:8091:12345678";
+        GlobalTransactionDTO dto = new GlobalTransactionDTO(xid);
+
+        assertEquals(xid, dto.getXid());
+    }
+
+    @Test
+    public void testSerialization() throws Exception {
+        GlobalTransactionDTO original = new GlobalTransactionDTO("xid:123456");
+
+        // Serialize
+        ByteArrayOutputStream bos = new ByteArrayOutputStream();
+        ObjectOutputStream oos = new ObjectOutputStream(bos);
+        oos.writeObject(original);
+        oos.flush();
+        byte[] serialized = bos.toByteArray();
+
+        // Deserialize
+        ByteArrayInputStream bis = new ByteArrayInputStream(serialized);
+        ObjectInputStream ois = new ObjectInputStream(bis);
+        GlobalTransactionDTO deserialized = (GlobalTransactionDTO) 
ois.readObject();
+
+        // Verify
+        assertEquals(original.getXid(), deserialized.getXid());
+    }
+}
diff --git 
a/server/src/test/java/org/apache/seata/server/cluster/raft/sync/msg/dto/RaftClusterMetadataTest.java
 
b/server/src/test/java/org/apache/seata/server/cluster/raft/sync/msg/dto/RaftClusterMetadataTest.java
new file mode 100644
index 0000000000..4c078b0637
--- /dev/null
+++ 
b/server/src/test/java/org/apache/seata/server/cluster/raft/sync/msg/dto/RaftClusterMetadataTest.java
@@ -0,0 +1,104 @@
+/*
+ * 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.seata.server.cluster.raft.sync.msg.dto;
+
+import org.apache.seata.common.metadata.Node;
+import org.junit.jupiter.api.Test;
+
+import java.util.Arrays;
+
+import static org.junit.jupiter.api.Assertions.*;
+
+public class RaftClusterMetadataTest {
+
+    @Test
+    public void testDefaultConstructor() {
+        RaftClusterMetadata metadata = new RaftClusterMetadata();
+        assertNotNull(metadata);
+        assertNull(metadata.getLeader());
+        assertNotNull(metadata.getFollowers());
+        assertNotNull(metadata.getLearner());
+        assertEquals(0, metadata.getTerm());
+    }
+
+    @Test
+    public void testConstructorWithTerm() {
+        long term = 12345L;
+        RaftClusterMetadata metadata = new RaftClusterMetadata(term);
+        assertEquals(term, metadata.getTerm());
+    }
+
+    @Test
+    public void testSetAndGetLeader() {
+        RaftClusterMetadata metadata = new RaftClusterMetadata();
+        Node leader = new Node();
+        metadata.setLeader(leader);
+        assertEquals(leader, metadata.getLeader());
+    }
+
+    @Test
+    public void testSetAndGetFollowers() {
+        RaftClusterMetadata metadata = new RaftClusterMetadata();
+        Node follower1 = new Node();
+        Node follower2 = new Node();
+        metadata.setFollowers(Arrays.asList(follower1, follower2));
+        assertEquals(2, metadata.getFollowers().size());
+    }
+
+    @Test
+    public void testSetAndGetLearner() {
+        RaftClusterMetadata metadata = new RaftClusterMetadata();
+        Node learner1 = new Node();
+        metadata.setLearner(Arrays.asList(learner1));
+        assertEquals(1, metadata.getLearner().size());
+    }
+
+    @Test
+    public void testSetAndGetTerm() {
+        RaftClusterMetadata metadata = new RaftClusterMetadata();
+        metadata.setTerm(999L);
+        assertEquals(999L, metadata.getTerm());
+    }
+
+    @Test
+    public void testToString() {
+        RaftClusterMetadata metadata = new RaftClusterMetadata(123L);
+        String str = metadata.toString();
+        assertNotNull(str);
+        assertFalse(str.isEmpty());
+    }
+
+    @Test
+    public void testNodeListManagement() {
+        RaftClusterMetadata metadata = new RaftClusterMetadata(100L);
+
+        Node leader = new Node();
+        metadata.setLeader(leader);
+        assertNotNull(metadata.getLeader());
+
+        Node follower1 = new Node();
+        Node follower2 = new Node();
+        metadata.setFollowers(Arrays.asList(follower1, follower2));
+        assertEquals(2, metadata.getFollowers().size());
+
+        Node learner = new Node();
+        metadata.setLearner(Arrays.asList(learner));
+        assertEquals(1, metadata.getLearner().size());
+
+        assertEquals(100L, metadata.getTerm());
+    }
+}
diff --git 
a/server/src/test/java/org/apache/seata/server/cluster/raft/util/RaftTaskUtilTest.java
 
b/server/src/test/java/org/apache/seata/server/cluster/raft/util/RaftTaskUtilTest.java
new file mode 100644
index 0000000000..4432733a72
--- /dev/null
+++ 
b/server/src/test/java/org/apache/seata/server/cluster/raft/util/RaftTaskUtilTest.java
@@ -0,0 +1,208 @@
+/*
+ * 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.seata.server.cluster.raft.util;
+
+import org.apache.seata.core.exception.GlobalTransactionException;
+import org.apache.seata.core.exception.TransactionException;
+import org.apache.seata.core.exception.TransactionExceptionCode;
+import org.junit.jupiter.api.Test;
+
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.ExecutionException;
+
+import static org.junit.jupiter.api.Assertions.assertEquals;
+import static org.junit.jupiter.api.Assertions.assertFalse;
+import static org.junit.jupiter.api.Assertions.assertThrows;
+import static org.junit.jupiter.api.Assertions.assertTrue;
+
+/**
+ * Unit tests for RaftTaskUtil focusing on future handling and exception 
management.
+ */
+public class RaftTaskUtilTest {
+
+    @Test
+    public void testFutureGetWithSuccessfulCompletion() throws 
TransactionException {
+        CompletableFuture<Boolean> future = 
CompletableFuture.completedFuture(true);
+
+        boolean result = RaftTaskUtil.futureGet(future);
+
+        assertTrue(result);
+    }
+
+    @Test
+    public void testFutureGetWithFalseResult() throws TransactionException {
+        CompletableFuture<Boolean> future = 
CompletableFuture.completedFuture(false);
+
+        boolean result = RaftTaskUtil.futureGet(future);
+
+        assertFalse(result);
+    }
+
+    @Test
+    public void testFutureGetWithInterruptedException() {
+        CompletableFuture<Boolean> future = new CompletableFuture<>();
+        future.completeExceptionally(new InterruptedException("Test 
interruption"));
+
+        GlobalTransactionException exception =
+                assertThrows(GlobalTransactionException.class, () -> 
RaftTaskUtil.futureGet(future));
+
+        assertEquals(TransactionExceptionCode.FailedWriteSession, 
exception.getCode());
+        assertTrue(exception.getMessage().contains("Fail to store global 
session"));
+    }
+
+    @Test
+    public void testFutureGetWithTransactionExceptionInExecutionException() {
+        CompletableFuture<Boolean> future = new CompletableFuture<>();
+        TransactionException cause =
+                new 
TransactionException(TransactionExceptionCode.BranchRegisterFailed, "Branch 
registration failed");
+        future.completeExceptionally(cause);
+
+        TransactionException exception = 
assertThrows(TransactionException.class, () -> RaftTaskUtil.futureGet(future));
+
+        assertEquals(TransactionExceptionCode.BranchRegisterFailed, 
exception.getCode());
+        assertTrue(exception.getMessage().contains("Branch registration 
failed"));
+    }
+
+    @Test
+    public void 
testFutureGetWithGlobalTransactionExceptionInExecutionException() {
+        CompletableFuture<Boolean> future = new CompletableFuture<>();
+        GlobalTransactionException cause = new GlobalTransactionException(
+                TransactionExceptionCode.GlobalTransactionNotExist, "Global 
transaction does not exist");
+        future.completeExceptionally(cause);
+
+        GlobalTransactionException exception =
+                assertThrows(GlobalTransactionException.class, () -> 
RaftTaskUtil.futureGet(future));
+
+        assertEquals(TransactionExceptionCode.GlobalTransactionNotExist, 
exception.getCode());
+        assertTrue(exception.getMessage().contains("Global transaction does 
not exist"));
+    }
+
+    @Test
+    public void testFutureGetWithNonTransactionExceptionInExecutionException() 
{
+        CompletableFuture<Boolean> future = new CompletableFuture<>();
+        RuntimeException cause = new RuntimeException("Unexpected error");
+        future.completeExceptionally(cause);
+
+        GlobalTransactionException exception =
+                assertThrows(GlobalTransactionException.class, () -> 
RaftTaskUtil.futureGet(future));
+
+        assertEquals(TransactionExceptionCode.FailedWriteSession, 
exception.getCode());
+        assertTrue(exception.getMessage().contains("Fail to store global 
session"));
+    }
+
+    @Test
+    public void testFutureGetWithExecutionExceptionWithNullCause() {
+        CompletableFuture<Boolean> future = new CompletableFuture<>();
+        future.completeExceptionally(new ExecutionException("Error message", 
null));
+
+        GlobalTransactionException exception =
+                assertThrows(GlobalTransactionException.class, () -> 
RaftTaskUtil.futureGet(future));
+
+        assertEquals(TransactionExceptionCode.FailedWriteSession, 
exception.getCode());
+    }
+
+    @Test
+    public void testFutureGetWithMultipleCompletions() throws 
TransactionException {
+        CompletableFuture<Boolean> future = 
CompletableFuture.completedFuture(true);
+
+        // First get
+        boolean result1 = RaftTaskUtil.futureGet(future);
+        assertTrue(result1);
+
+        // Second get should return the same result
+        boolean result2 = RaftTaskUtil.futureGet(future);
+        assertTrue(result2);
+    }
+
+    @Test
+    public void testFutureGetPreservesTransactionExceptionCode() {
+        CompletableFuture<Boolean> future = new CompletableFuture<>();
+        TransactionException cause =
+                new 
TransactionException(TransactionExceptionCode.LockKeyConflict, "Lock conflict 
detected");
+        future.completeExceptionally(cause);
+
+        TransactionException exception = 
assertThrows(TransactionException.class, () -> RaftTaskUtil.futureGet(future));
+
+        assertEquals(TransactionExceptionCode.LockKeyConflict, 
exception.getCode());
+        assertEquals("Lock conflict detected", exception.getMessage());
+    }
+
+    @Test
+    public void testFutureGetWithDifferentTransactionExceptionTypes() {
+        // Test with various TransactionException subclasses
+        CompletableFuture<Boolean> future1 = new CompletableFuture<>();
+        future1.completeExceptionally(
+                new 
GlobalTransactionException(TransactionExceptionCode.FailedToSendBranchCommitRequest));
+
+        assertThrows(GlobalTransactionException.class, () -> 
RaftTaskUtil.futureGet(future1));
+
+        CompletableFuture<Boolean> future2 = new CompletableFuture<>();
+        future2.completeExceptionally(new 
TransactionException(TransactionExceptionCode.FailedToAddBranch));
+
+        assertThrows(TransactionException.class, () -> 
RaftTaskUtil.futureGet(future2));
+    }
+
+    @Test
+    public void testFutureGetWithAsyncCompletion() throws Exception {
+        CompletableFuture<Boolean> future = new CompletableFuture<>();
+
+        // Complete asynchronously
+        Thread completer = new Thread(() -> {
+            try {
+                Thread.sleep(100);
+                future.complete(true);
+            } catch (InterruptedException e) {
+                future.completeExceptionally(e);
+            }
+        });
+        completer.start();
+
+        boolean result = RaftTaskUtil.futureGet(future);
+
+        assertTrue(result);
+        completer.join();
+    }
+
+    @Test
+    public void testFutureGetExceptionMessageContainsDetails() {
+        CompletableFuture<Boolean> future = new CompletableFuture<>();
+        future.completeExceptionally(new InterruptedException("Specific 
interruption reason"));
+
+        GlobalTransactionException exception =
+                assertThrows(GlobalTransactionException.class, () -> 
RaftTaskUtil.futureGet(future));
+
+        // The exception message should contain reference to the failure
+        assertTrue(exception.getMessage().contains("Fail to store global 
session"));
+        assertTrue(exception.getMessage().contains("Specific interruption 
reason"));
+    }
+
+    @Test
+    public void testFutureGetWithNestedExecutionException() {
+        CompletableFuture<Boolean> future = new CompletableFuture<>();
+        ExecutionException nested =
+                new ExecutionException(new 
TransactionException(TransactionExceptionCode.BeginFailed, "Begin failed"));
+        future.completeExceptionally(nested);
+
+        // The ExecutionException's cause is another ExecutionException,
+        // which is not a TransactionException, so should be wrapped
+        GlobalTransactionException exception =
+                assertThrows(GlobalTransactionException.class, () -> 
RaftTaskUtil.futureGet(future));
+
+        // Should wrap the ExecutionException since it's not 
TransactionException
+        assertEquals(TransactionExceptionCode.FailedWriteSession, 
exception.getCode());
+    }
+}


---------------------------------------------------------------------
To unsubscribe, e-mail: [email protected]
For additional commands, e-mail: [email protected]

Reply via email to