From 423be85df8ff52efe2302081720d67e7c947af9f Mon Sep 17 00:00:00 2001 From: Nandakumar Vadivelu Date: Thu, 1 Feb 2024 22:27:19 +0530 Subject: [PATCH 001/397] RATIS-1976. Migrate ratis-test tests to Junit 5 - Part 1 (#1038) --- .../ratis/InstallSnapshotFromLeaderTests.java | 31 ++-- .../org/apache/ratis/RetryCacheTests.java | 22 +-- .../server/impl/LeaderElectionTests.java | 133 +++++++++--------- ratis-test/pom.xml | 10 ++ .../ratis/client/TestClientProtoUtils.java | 6 +- .../org/apache/ratis/conf/TestConfUtils.java | 2 +- .../apache/ratis/conf/TestRaftProperties.java | 45 +++--- .../DataStreamAsyncClusterTests.java | 12 +- .../ratis/datastream/DataStreamBaseTest.java | 12 +- .../datastream/DataStreamClusterTests.java | 14 +- .../ratis/datastream/DataStreamTestUtils.java | 48 +++---- ...WithRpcTypeGrpcAndDataStreamTypeNetty.java | 1 - .../datastream/TestDataStreamDisabled.java | 11 +- ...WithRpcTypeGrpcAndDataStreamTypeNetty.java | 10 +- ...ataStreamChainTopologyWithGrpcCluster.java | 4 +- ...DataStreamStarTopologyWithGrpcCluster.java | 4 +- .../TestNettyDataStreamWithMock.java | 10 +- .../TestNettyDataStreamWithNettyCluster.java | 5 +- .../apache/ratis/grpc/TestGrpcFactory.java | 8 +- .../ratis/grpc/TestGrpcMessageMetrics.java | 6 +- .../grpc/TestLeaderElectionWithGrpc.java | 2 +- .../ratis/grpc/TestLeaderInstallSnapshot.java | 23 ++- .../ratis/grpc/TestLogAppenderWithGrpc.java | 35 +++-- .../grpc/TestRaftOutputStreamWithGrpc.java | 6 +- .../ratis/grpc/TestRaftServerWithGrpc.java | 94 +++++++------ .../ratis/grpc/TestRaftSnapshotWithGrpc.java | 8 +- .../apache/ratis/grpc/TestRaftWithGrpc.java | 42 +++--- .../ratis/grpc/TestRetryCacheWithGrpc.java | 14 +- .../grpc/server/TestGrpcServerMetrics.java | 27 ++-- .../grpc/util/GrpcZeroCopyTestServer.java | 10 +- .../ratis/grpc/util/TestGrpcZeroCopy.java | 16 +-- .../util/TestStreamObserverWithTimeout.java | 10 +- .../netty/TestLeaderElectionWithNetty.java | 2 +- 33 files changed, 351 insertions(+), 332 deletions(-) diff --git a/ratis-server/src/test/java/org/apache/ratis/InstallSnapshotFromLeaderTests.java b/ratis-server/src/test/java/org/apache/ratis/InstallSnapshotFromLeaderTests.java index 15dafb88c8..9d6a2f1834 100644 --- a/ratis-server/src/test/java/org/apache/ratis/InstallSnapshotFromLeaderTests.java +++ b/ratis-server/src/test/java/org/apache/ratis/InstallSnapshotFromLeaderTests.java @@ -39,8 +39,7 @@ import org.apache.ratis.util.JavaUtils; import org.apache.ratis.util.LifeCycle; import org.apache.ratis.util.SizeInBytes; -import org.junit.Assert; -import org.junit.Test; +import org.junit.jupiter.api.Assertions; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -74,14 +73,12 @@ public abstract class InstallSnapshotFromLeaderTests { for (RaftServer.Division follower : cluster.getFollowers()) { final SnapshotInfo info = follower.getStateMachine().getLatestSnapshot(); - Assert.assertNotNull(info); - Assert.assertEquals(3, info.getFiles().size()); + Assertions.assertNotNull(info); + Assertions.assertEquals(3, info.getFiles().size()); } }, 10, ONE_SECOND, "check snapshot", LOG); } finally { @@ -182,8 +179,8 @@ public long takeSnapshot() { return RaftLog.INVALID_LOG_INDEX; } - Assert.assertTrue(file1.exists()); - Assert.assertTrue(file2.exists()); + Assertions.assertTrue(file1.exists()); + Assertions.assertTrue(file2.exists()); return super.takeSnapshot(); } @@ -199,7 +196,7 @@ public SnapshotInfo getLatestSnapshot() { files.add(new FileInfo( file2.toPath(), null)); - Assert.assertEquals(2, files.size()); + Assertions.assertEquals(2, files.size()); SnapshotInfo info = super.getLatestSnapshot(); if (info == null) { @@ -224,8 +221,8 @@ public synchronized void initialize(RaftServer server, RaftGroupId groupId, Raft this.snapshotDir = new File(root, "snapshot"); this.tmpDir = new File(root, "tmp"); FileUtils.deleteFully(root); - Assert.assertTrue(this.snapshotDir.mkdirs()); - Assert.assertTrue(this.tmpDir.mkdirs()); + Assertions.assertTrue(this.snapshotDir.mkdirs()); + Assertions.assertTrue(this.tmpDir.mkdirs()); this.root.deleteOnExit(); } @@ -246,13 +243,13 @@ public long takeSnapshot() { try { FileUtils.deleteFully(snapshotRealDir); FileUtils.deleteFully(snapshotTmpDir); - Assert.assertTrue(snapshotTmpDir.mkdirs()); + Assertions.assertTrue(snapshotTmpDir.mkdirs()); final File snapshotFile1 = new File(snapshotTmpDir, "deer"); final File snapshotFile2 = new File(snapshotTmpDir, "loves"); final File snapshotFile3 = new File(snapshotTmpDir, "vegetable"); - Assert.assertTrue(snapshotFile1.createNewFile()); - Assert.assertTrue(snapshotFile2.createNewFile()); - Assert.assertTrue(snapshotFile3.createNewFile()); + Assertions.assertTrue(snapshotFile1.createNewFile()); + Assertions.assertTrue(snapshotFile2.createNewFile()); + Assertions.assertTrue(snapshotFile3.createNewFile()); FileUtils.move(snapshotTmpDir, snapshotRealDir); } catch (IOException ioe) { LOG.error("create snapshot data file failed", ioe); diff --git a/ratis-server/src/test/java/org/apache/ratis/RetryCacheTests.java b/ratis-server/src/test/java/org/apache/ratis/RetryCacheTests.java index 18561ee65c..f81ac9556b 100644 --- a/ratis-server/src/test/java/org/apache/ratis/RetryCacheTests.java +++ b/ratis-server/src/test/java/org/apache/ratis/RetryCacheTests.java @@ -35,8 +35,8 @@ import org.apache.ratis.util.JavaUtils; import org.apache.ratis.util.Slf4jUtils; import org.apache.ratis.util.TimeDuration; -import org.junit.Assert; -import org.junit.Test; +import org.junit.jupiter.api.Assertions; +import org.junit.jupiter.api.Test; import org.slf4j.event.Level; import java.util.Arrays; @@ -84,9 +84,9 @@ void runTestBasicRetry(CLUSTER cluster) throws Exception { } public static void assertReply(RaftClientReply reply, RaftClient client, long callId) { - Assert.assertEquals(client.getId(), reply.getClientId()); - Assert.assertEquals(callId, reply.getCallId()); - Assert.assertTrue(reply.isSuccess()); + Assertions.assertEquals(client.getId(), reply.getClientId()); + Assertions.assertEquals(callId, reply.getCallId()); + Assertions.assertTrue(reply.isSuccess()); } public void assertServer(MiniRaftCluster cluster, ClientId clientId, long callId, long oldLastApplied) throws Exception { @@ -97,10 +97,10 @@ public void assertServer(MiniRaftCluster cluster, ClientId clientId, long callId if (server.getInfo().getLastAppliedIndex() < leaderApplied) { Thread.sleep(1000); } - Assert.assertEquals(2, server.getRetryCache().getStatistics().size()); - Assert.assertNotNull(RetryCacheTestUtil.get(server, clientId, callId)); + Assertions.assertEquals(2, server.getRetryCache().getStatistics().size()); + Assertions.assertNotNull(RetryCacheTestUtil.get(server, clientId, callId)); // make sure there is only one log entry committed - Assert.assertEquals(1, count(server.getRaftLog(), oldLastApplied + 1)); + Assertions.assertEquals(1, count(server.getRaftLog(), oldLastApplied + 1)); } } @@ -145,10 +145,10 @@ void runTestRetryOnNewLeader(CLUSTER cluster) throws Exception { final RaftPeerId newLeaderId = JavaUtils.attemptRepeatedly(() -> { final RaftPeerId id = RaftTestUtil.waitForLeader(cluster).getId(); - Assert.assertNotEquals(leaderId, id); + Assertions.assertNotEquals(leaderId, id); return id; }, 10, TimeDuration.valueOf(100, TimeUnit.MILLISECONDS), "wait for a leader different than " + leaderId, LOG); - Assert.assertNotEquals(leaderId, newLeaderId); + Assertions.assertNotEquals(leaderId, newLeaderId); // same clientId and callId in the request r = cluster.newRaftClientRequest(client.getId(), newLeaderId, callId, new SimpleMessage("message")); @@ -164,7 +164,7 @@ void runTestRetryOnNewLeader(CLUSTER cluster) throws Exception { } // check the new leader and make sure the retry did not get committed - Assert.assertEquals(0, count(cluster.getLeader().getRaftLog(), oldLastApplied + 1)); + Assertions.assertEquals(0, count(cluster.getLeader().getRaftLog(), oldLastApplied + 1)); } } } diff --git a/ratis-server/src/test/java/org/apache/ratis/server/impl/LeaderElectionTests.java b/ratis-server/src/test/java/org/apache/ratis/server/impl/LeaderElectionTests.java index 6453e8e944..391541e64a 100644 --- a/ratis-server/src/test/java/org/apache/ratis/server/impl/LeaderElectionTests.java +++ b/ratis-server/src/test/java/org/apache/ratis/server/impl/LeaderElectionTests.java @@ -44,8 +44,8 @@ import org.apache.ratis.util.TimeDuration; import org.apache.ratis.util.Timestamp; import org.apache.ratis.util.function.CheckedBiConsumer; -import org.junit.Assert; -import org.junit.Test; +import org.junit.jupiter.api.Assertions; +import org.junit.jupiter.api.Test; import org.slf4j.Logger; import java.io.IOException; @@ -138,18 +138,18 @@ public void testLeaderNotCountListenerForMajority() throws Exception { void runTestLeaderNotCountListenerForMajority(CLUSTER cluster) throws Exception { final RaftServer.Division leader = waitForLeader(cluster); - Assert.assertEquals(2, ((RaftConfigurationImpl)cluster.getLeader().getRaftConf()).getMajorityCount()); + Assertions.assertEquals(2, ((RaftConfigurationImpl)cluster.getLeader().getRaftConf()).getMajorityCount()); try (RaftClient client = cluster.createClient(leader.getId())) { client.io().send(new RaftTestUtil.SimpleMessage("message")); List listeners = cluster.getListeners() .stream().map(RaftServer.Division::getPeer).collect(Collectors.toList()); - Assert.assertEquals(2, listeners.size()); + Assertions.assertEquals(2, listeners.size()); RaftClientReply reply = client.admin().setConfiguration(cluster.getPeers()); - Assert.assertTrue(reply.isSuccess()); + Assertions.assertTrue(reply.isSuccess()); Collection peer = leader.getRaftConf().getAllPeers(RaftProtos.RaftPeerRole.LISTENER); - Assert.assertEquals(0, peer.size()); + Assertions.assertEquals(0, peer.size()); } - Assert.assertEquals(3, ((RaftConfigurationImpl)cluster.getLeader().getRaftConf()).getMajorityCount()); + Assertions.assertEquals(3, ((RaftConfigurationImpl)cluster.getLeader().getRaftConf()).getMajorityCount()); } @Test @@ -167,7 +167,7 @@ void runTestListenerNotStartLeaderElection(CLUSTER cluster) throws Exception { isolate(cluster, listenerId); maxTimeout.sleep(); maxTimeout.sleep(); - Assert.assertEquals(RaftProtos.RaftPeerRole.LISTENER, listener.getInfo().getCurrentRole()); + Assertions.assertEquals(RaftProtos.RaftPeerRole.LISTENER, listener.getInfo().getCurrentRole()); } finally { deIsolate(cluster, listener.getId()); } @@ -183,18 +183,18 @@ public void testTransferLeader() throws Exception { client.io().send(new RaftTestUtil.SimpleMessage("message")); List followers = cluster.getFollowers(); - Assert.assertEquals(2, followers.size()); + Assertions.assertEquals(2, followers.size()); RaftServer.Division newLeader = followers.get(0); RaftClientReply reply = client.admin().transferLeadership(newLeader.getId(), 20000); - Assert.assertTrue(reply.isSuccess()); + Assertions.assertTrue(reply.isSuccess()); final RaftServer.Division currLeader = waitForLeader(cluster); - Assert.assertEquals(newLeader.getId(), currLeader.getId()); + Assertions.assertEquals(newLeader.getId(), currLeader.getId()); reply = client.io().send(new RaftTestUtil.SimpleMessage("message")); - Assert.assertEquals(newLeader.getId().toString(), reply.getReplierId()); - Assert.assertTrue(reply.isSuccess()); + Assertions.assertEquals(newLeader.getId().toString(), reply.getReplierId()); + Assertions.assertTrue(reply.isSuccess()); } cluster.shutdown(); @@ -211,24 +211,24 @@ public void testYieldLeaderToHigherPriority() throws Exception { client.io().send(new RaftTestUtil.SimpleMessage("message")); List followers = cluster.getFollowers(); - Assert.assertEquals(2, followers.size()); + Assertions.assertEquals(2, followers.size()); RaftServer.Division newLeader = followers.get(0); List peers = cluster.getPeers(); List peersWithNewPriority = getPeersWithPriority(peers, newLeader.getPeer()); RaftClientReply reply = client.admin().setConfiguration(peersWithNewPriority.toArray(new RaftPeer[0])); - Assert.assertTrue(reply.isSuccess()); + Assertions.assertTrue(reply.isSuccess()); // Wait the old leader to step down. // TODO: make it more deterministic. TimeDuration.valueOf(1, TimeUnit.SECONDS).sleep(); final RaftServer.Division currLeader = waitForLeader(cluster); - Assert.assertEquals(newLeader.getId(), currLeader.getId()); + Assertions.assertEquals(newLeader.getId(), currLeader.getId()); reply = client.io().send(new RaftTestUtil.SimpleMessage("message")); - Assert.assertEquals(newLeader.getId().toString(), reply.getReplierId()); - Assert.assertTrue(reply.isSuccess()); + Assertions.assertEquals(newLeader.getId().toString(), reply.getReplierId()); + Assertions.assertTrue(reply.isSuccess()); } cluster.shutdown(); @@ -243,7 +243,7 @@ public void testTransferLeaderTimeout() throws Exception { final RaftServer.Division leader = waitForLeader(cluster); try (RaftClient client = cluster.createClient(leader.getId())) { List followers = cluster.getFollowers(); - Assert.assertEquals(followers.size(), 2); + Assertions.assertEquals(followers.size(), 2); RaftServer.Division newLeader = followers.get(0); // isolate new leader, so that transfer leadership will timeout @@ -259,9 +259,9 @@ public void testTransferLeaderTimeout() throws Exception { client.admin().transferLeadership(newLeader.getId(), timeoutMs); } catch (TransferLeadershipException e) { long cost = System.currentTimeMillis() - start; - Assert.assertTrue(cost > timeoutMs); - Assert.assertTrue(e.getMessage().contains("Failed to transfer leadership to")); - Assert.assertTrue(e.getMessage().contains(TransferLeadership.Result.Type.TIMED_OUT.toString())); + Assertions.assertTrue(cost > timeoutMs); + Assertions.assertTrue(e.getMessage().contains("Failed to transfer leadership to")); + Assertions.assertTrue(e.getMessage().contains(TransferLeadership.Result.Type.TIMED_OUT.toString())); } return true; @@ -275,17 +275,17 @@ public void testTransferLeaderTimeout() throws Exception { try { client.io().send(new RaftTestUtil.SimpleMessage("message")); } catch (LeaderSteppingDownException e) { - Assert.assertTrue(e.getMessage().contains("is stepping down")); + Assertions.assertTrue(e.getMessage().contains("is stepping down")); } return null; }, 5, TimeDuration.ONE_SECOND, "check leader steppingDown", RaftServer.LOG); - Assert.assertTrue(transferTimeoutFuture.get()); + Assertions.assertTrue(transferTimeoutFuture.get()); // after transfer timeout, leader should accept request RaftClientReply reply = client.io().send(new RaftTestUtil.SimpleMessage("message")); - Assert.assertEquals(leader.getId().toString(), reply.getReplierId()); - Assert.assertTrue(reply.isSuccess()); + Assertions.assertEquals(leader.getId().toString(), reply.getReplierId()); + Assertions.assertTrue(reply.isSuccess()); deIsolate(cluster, newLeader.getId()); } @@ -323,7 +323,7 @@ static void enforceLeader(MiniRaftCluster cluster, final String newLeader, Logge LOG.info(cluster.printServers()); final RaftServer.Division leader = cluster.getLeader(); - Assert.assertEquals(newLeader, leader.getId().toString()); + Assertions.assertEquals(newLeader, leader.getId().toString()); } @Test @@ -352,7 +352,8 @@ public void testLateServerStart() throws Exception { .orElseThrow(() -> new IllegalStateException("No leader yet")), 10, ONE_SECOND, "getLeaderId", LOG); LOG.info(cluster.printServers()); - Assert.assertEquals(leader.getId(), lastServerLeaderId); + Assertions.assertEquals(leader.getId(), lastServerLeaderId); + cluster.shutdown(); } protected void testDisconnectLeader() throws Exception { @@ -365,8 +366,8 @@ protected void testDisconnectLeader() throws Exception { Thread.sleep(1000); isolate(cluster, leader.getId()); RaftClientReply reply = client.io().send(new RaftTestUtil.SimpleMessage("message")); - Assert.assertNotEquals(reply.getReplierId(), leader.getId().toString()); - Assert.assertTrue(reply.isSuccess()); + Assertions.assertNotEquals(reply.getReplierId(), leader.getId().toString()); + Assertions.assertTrue(reply.isSuccess()); } finally { deIsolate(cluster, leader.getId()); } @@ -397,15 +398,15 @@ public void testAddListener() throws Exception { try (RaftClient client = cluster.createClient(leader.getId())) { client.io().send(new RaftTestUtil.SimpleMessage("message")); List servers = cluster.getPeers(); - Assert.assertEquals(servers.size(), 3); + Assertions.assertEquals(servers.size(), 3); MiniRaftCluster.PeerChanges changes = cluster.addNewPeers(1, true, false, RaftProtos.RaftPeerRole.LISTENER); RaftClientReply reply = client.admin().setConfiguration(servers, Arrays.asList(changes.newPeers)); - Assert.assertTrue(reply.isSuccess()); + Assertions.assertTrue(reply.isSuccess()); Collection listener = leader.getRaftConf().getAllPeers(RaftProtos.RaftPeerRole.LISTENER); - Assert.assertEquals(1, listener.size()); - Assert.assertEquals(changes.newPeers[0].getId(), new ArrayList<>(listener).get(0).getId()); + Assertions.assertEquals(1, listener.size()); + Assertions.assertEquals(changes.newPeers[0].getId(), new ArrayList<>(listener).get(0).getId()); } cluster.shutdown(); } @@ -419,18 +420,18 @@ public void testAddFollowerWhenExistsListener() throws Exception { try (RaftClient client = cluster.createClient(leader.getId())) { client.io().send(new RaftTestUtil.SimpleMessage("message")); List servers = cluster.getPeers(); - Assert.assertEquals(4, servers.size()); + Assertions.assertEquals(4, servers.size()); List listener = new ArrayList<>( leader.getRaftConf().getAllPeers(RaftProtos.RaftPeerRole.LISTENER)); - Assert.assertEquals(1, listener.size()); + Assertions.assertEquals(1, listener.size()); MiniRaftCluster.PeerChanges changes = cluster.addNewPeers(1, true, false); ArrayList newPeers = new ArrayList<>(Arrays.asList(changes.newPeers)); newPeers.addAll(leader.getRaftConf().getAllPeers(RaftProtos.RaftPeerRole.FOLLOWER)); RaftClientReply reply = client.admin().setConfiguration(newPeers, listener); - Assert.assertTrue(reply.isSuccess()); - Assert.assertEquals(4, + Assertions.assertTrue(reply.isSuccess()); + Assertions.assertEquals(4, leader.getRaftConf().getAllPeers(RaftProtos.RaftPeerRole.FOLLOWER).size()); - Assert.assertEquals(1, + Assertions.assertEquals(1, leader.getRaftConf().getAllPeers(RaftProtos.RaftPeerRole.LISTENER).size()); } cluster.shutdown(); @@ -444,13 +445,13 @@ public void testRemoveListener() throws Exception { final RaftServer.Division leader = waitForLeader(cluster); try (RaftClient client = cluster.createClient(leader.getId())) { client.io().send(new RaftTestUtil.SimpleMessage("message")); - Assert.assertEquals(1, cluster.getListeners().size()); + Assertions.assertEquals(1, cluster.getListeners().size()); List servers = cluster.getFollowers().stream().map(RaftServer.Division::getPeer).collect( Collectors.toList()); servers.add(leader.getPeer()); RaftClientReply reply = client.admin().setConfiguration(servers); - Assert.assertTrue(reply.isSuccess()); - Assert.assertEquals(0, leader.getRaftConf().getAllPeers(RaftProtos.RaftPeerRole.LISTENER).size()); + Assertions.assertTrue(reply.isSuccess()); + Assertions.assertEquals(0, leader.getRaftConf().getAllPeers(RaftProtos.RaftPeerRole.LISTENER).size()); } cluster.shutdown(); } @@ -465,15 +466,15 @@ public void testChangeFollowerToListener() throws Exception { client.io().send(new RaftTestUtil.SimpleMessage("message")); List followers = cluster.getFollowers().stream().map( RaftServer.Division::getPeer).collect(Collectors.toList()); - Assert.assertEquals(2, followers.size()); + Assertions.assertEquals(2, followers.size()); List listeners = new ArrayList<>(); listeners.add(followers.get(1)); followers.remove(1); RaftClientReply reply = client.admin().setConfiguration(followers, listeners); - Assert.assertTrue(reply.isSuccess()); + Assertions.assertTrue(reply.isSuccess()); Collection peer = leader.getRaftConf().getAllPeers(RaftProtos.RaftPeerRole.LISTENER); - Assert.assertEquals(1, peer.size()); - Assert.assertEquals(listeners.get(0).getId(), new ArrayList<>(peer).get(0).getId()); + Assertions.assertEquals(1, peer.size()); + Assertions.assertEquals(listeners.get(0).getId(), new ArrayList<>(peer).get(0).getId()); } cluster.shutdown(); } @@ -488,11 +489,11 @@ public void testChangeListenerToFollower() throws Exception { client.io().send(new RaftTestUtil.SimpleMessage("message")); List listeners = cluster.getListeners() .stream().map(RaftServer.Division::getPeer).collect(Collectors.toList()); - Assert.assertEquals(listeners.size(), 1); + Assertions.assertEquals(listeners.size(), 1); RaftClientReply reply = client.admin().setConfiguration(cluster.getPeers()); - Assert.assertTrue(reply.isSuccess()); + Assertions.assertTrue(reply.isSuccess()); Collection peer = leader.getRaftConf().getAllPeers(RaftProtos.RaftPeerRole.LISTENER); - Assert.assertEquals(0, peer.size()); + Assertions.assertEquals(0, peer.size()); } cluster.shutdown(); } @@ -571,7 +572,7 @@ public void testPreVote() { isolate(cluster, follower.getId()); // send message so that the isolated follower's log lag the others RaftClientReply reply = client.io().send(new RaftTestUtil.SimpleMessage("message")); - Assert.assertTrue(reply.isSuccess()); + Assertions.assertTrue(reply.isSuccess()); final long savedTerm = leader.getInfo().getCurrentTerm(); LOG.info("Wait follower {} timeout and trigger pre-vote", follower.getId()); @@ -586,7 +587,7 @@ public void testPreVote() { assertEquals(savedTerm, leader.getInfo().getCurrentTerm()); reply = client.io().send(new RaftTestUtil.SimpleMessage("message")); - Assert.assertTrue(reply.isSuccess()); + Assertions.assertTrue(reply.isSuccess()); } cluster.shutdown(); @@ -606,7 +607,7 @@ void runTestListenerRejectRequestVote(CLUSTER cluster) throws IOException, Inter final RaftProtos.RequestVoteRequestProto r = ServerProtoUtils.toRequestVoteRequestProto( leader.getMemberId(), listener.getId(), leader.getRaftLog().getLastEntryTermIndex().getTerm() + 1, lastEntry, true); RaftProtos.RequestVoteReplyProto listenerReply = listener.getRaftServer().requestVote(r); - Assert.assertFalse(listenerReply.getServerReply().getSuccess()); + Assertions.assertFalse(listenerReply.getServerReply().getSuccess()); } @@ -620,23 +621,23 @@ void runTestPauseResumeLeaderElection(CLUSTER cluster) throws IOException, Inter final RaftServer.Division leader = RaftTestUtil.waitForLeader(cluster); final RaftPeerId leaderId = leader.getId(); final List followers = cluster.getFollowers(); - Assert.assertTrue(followers.size() >= 1); + Assertions.assertTrue(followers.size() >= 1); final RaftServerImpl f1 = (RaftServerImpl)followers.get(0); try (final RaftClient client = cluster.createClient()) { pauseLeaderReply = client.getLeaderElectionManagementApi(f1.getId()).pause(); - Assert.assertTrue(pauseLeaderReply.isSuccess()); + Assertions.assertTrue(pauseLeaderReply.isSuccess()); client.io().send(new RaftTestUtil.SimpleMessage("message")); RaftServer.Division newLeader = followers.get(0); List peers = cluster.getPeers(); List peersWithNewPriority = getPeersWithPriority(peers, newLeader.getPeer()); RaftClientReply reply = client.admin().setConfiguration(peersWithNewPriority.toArray(new RaftPeer[0])); - Assert.assertTrue(reply.isSuccess()); - JavaUtils.attempt(() -> Assert.assertEquals(leaderId, leader.getId()), + Assertions.assertTrue(reply.isSuccess()); + JavaUtils.attempt(() -> Assertions.assertEquals(leaderId, leader.getId()), 20, HUNDRED_MILLIS, "check leader id", LOG); final RaftClientReply resumeLeaderReply = client.getLeaderElectionManagementApi(f1.getId()).resume(); - Assert.assertTrue(resumeLeaderReply.isSuccess()); - JavaUtils.attempt(() -> Assert.assertEquals(f1.getId(), cluster.getLeader().getId()), + Assertions.assertTrue(resumeLeaderReply.isSuccess()); + JavaUtils.attempt(() -> Assertions.assertEquals(f1.getId(), cluster.getLeader().getId()), 20, HUNDRED_MILLIS, "check new leader", LOG); } } @@ -662,15 +663,15 @@ void runTestLeaderLease(CLUSTER cluster, long leaseTimeoutMs) throws Exception { try (final RaftClient client = cluster.createClient(leader.getId())) { client.io().send(new RaftTestUtil.SimpleMessage("message")); - Assert.assertTrue(leader.getInfo().isLeader()); - Assert.assertTrue(leader.getInfo().isLeaderReady()); + Assertions.assertTrue(leader.getInfo().isLeader()); + Assertions.assertTrue(leader.getInfo().isLeaderReady()); RaftServerTestUtil.assertLeaderLease(leader, true); isolate(cluster, leader.getId()); Thread.sleep(leaseTimeoutMs); - Assert.assertTrue(leader.getInfo().isLeader()); - Assert.assertTrue(leader.getInfo().isLeaderReady()); + Assertions.assertTrue(leader.getInfo().isLeader()); + Assertions.assertTrue(leader.getInfo().isLeaderReady()); RaftServerTestUtil.assertLeaderLease(leader, false); } finally { deIsolate(cluster, leader.getId()); @@ -690,8 +691,8 @@ void runTestLeaderLeaseDuringReconfiguration(CLUSTER cluster, long leaseTimeoutM try (final RaftClient client = cluster.createClient(leader.getId())) { client.io().send(new RaftTestUtil.SimpleMessage("message")); - Assert.assertTrue(leader.getInfo().isLeader()); - Assert.assertTrue(leader.getInfo().isLeaderReady()); + Assertions.assertTrue(leader.getInfo().isLeader()); + Assertions.assertTrue(leader.getInfo().isLeaderReady()); RaftServerTestUtil.assertLeaderLease(leader, true); final List followers = cluster.getFollowers(); @@ -712,8 +713,8 @@ void runTestLeaderLeaseDuringReconfiguration(CLUSTER cluster, long leaseTimeoutM Thread.sleep(leaseTimeoutMs); - Assert.assertTrue(leader.getInfo().isLeader()); - Assert.assertTrue(leader.getInfo().isLeaderReady()); + Assertions.assertTrue(leader.getInfo().isLeader()); + Assertions.assertTrue(leader.getInfo().isLeaderReady()); RaftServerTestUtil.assertLeaderLease(leader, false); } finally { diff --git a/ratis-test/pom.xml b/ratis-test/pom.xml index 2021e06ad4..5990ae057f 100644 --- a/ratis-test/pom.xml +++ b/ratis-test/pom.xml @@ -136,11 +136,21 @@ junit test + + org.junit.jupiter + junit-jupiter-engine + test + org.junit.jupiter junit-jupiter-api test + + org.junit.jupiter + junit-jupiter-params + test + org.mockito mockito-core diff --git a/ratis-test/src/test/java/org/apache/ratis/client/TestClientProtoUtils.java b/ratis-test/src/test/java/org/apache/ratis/client/TestClientProtoUtils.java index 9d85320539..56bf94a84e 100644 --- a/ratis-test/src/test/java/org/apache/ratis/client/TestClientProtoUtils.java +++ b/ratis-test/src/test/java/org/apache/ratis/client/TestClientProtoUtils.java @@ -29,8 +29,8 @@ import org.apache.ratis.util.SizeInBytes; import org.apache.ratis.util.TimeDuration; import org.apache.ratis.util.Timestamp; -import org.junit.Assert; -import org.junit.Test; +import org.junit.jupiter.api.Assertions; +import org.junit.jupiter.api.Test; import java.io.IOException; import java.util.concurrent.TimeUnit; @@ -71,7 +71,7 @@ void runTestToRaftClientRequestProto(int n, SizeInBytes messageSize) final RaftClientRequest computed = ClientProtoUtils.toRaftClientRequest(proto); final TimeDuration r = startTime.elapsedTime().subtract(p); - Assert.assertEquals(request.getMessage().getContent(), computed.getMessage().getContent()); + Assertions.assertEquals(request.getMessage().getContent(), computed.getMessage().getContent()); toProto = toProto.add(p); toRequest = toRequest.add(r); diff --git a/ratis-test/src/test/java/org/apache/ratis/conf/TestConfUtils.java b/ratis-test/src/test/java/org/apache/ratis/conf/TestConfUtils.java index 1600da18dc..67c02cd5f6 100644 --- a/ratis-test/src/test/java/org/apache/ratis/conf/TestConfUtils.java +++ b/ratis-test/src/test/java/org/apache/ratis/conf/TestConfUtils.java @@ -23,7 +23,7 @@ import org.apache.ratis.grpc.GrpcConfigKeys; import org.apache.ratis.netty.NettyConfigKeys; import org.apache.ratis.server.RaftServerConfigKeys; -import org.junit.Test; +import org.junit.jupiter.api.Test; public class TestConfUtils extends BaseTest { @Test diff --git a/ratis-test/src/test/java/org/apache/ratis/conf/TestRaftProperties.java b/ratis-test/src/test/java/org/apache/ratis/conf/TestRaftProperties.java index c65d00f43e..29bfc321df 100644 --- a/ratis-test/src/test/java/org/apache/ratis/conf/TestRaftProperties.java +++ b/ratis-test/src/test/java/org/apache/ratis/conf/TestRaftProperties.java @@ -17,8 +17,10 @@ */ package org.apache.ratis.conf; -import org.junit.Assert; -import org.junit.Test; + +import org.junit.jupiter.api.Assertions; +import org.junit.jupiter.api.Test; +import org.junit.jupiter.api.Timeout; public class TestRaftProperties { enum Type {APPEND_ENTRIES} @@ -29,71 +31,72 @@ static class Request_Vote { static final String KEY = "key"; static void setUnderscoreValue(RaftProperties p, String valueWithUnderscore) { - Assert.assertTrue(valueWithUnderscore.contains("_")); + Assertions.assertTrue(valueWithUnderscore.contains("_")); p.set(KEY, valueWithUnderscore); } static void setNonUnderscoreValue(RaftProperties p, String valueWithoutUnderscore) { - Assert.assertFalse(valueWithoutUnderscore.contains("_")); + Assertions.assertFalse(valueWithoutUnderscore.contains("_")); p.set(KEY, valueWithoutUnderscore); } - @Test(timeout = 1000) + @Test + @Timeout(value = 1000) public void testUnderscore() { final RaftProperties p = new RaftProperties(); { // boolean - Assert.assertNull(p.getBoolean(KEY, null)); + Assertions.assertNull(p.getBoolean(KEY, null)); setNonUnderscoreValue(p, "true"); - Assert.assertTrue(p.getBoolean(KEY, null)); + Assertions.assertTrue(p.getBoolean(KEY, null)); setNonUnderscoreValue(p, "false"); - Assert.assertFalse(p.getBoolean(KEY, null)); + Assertions.assertFalse(p.getBoolean(KEY, null)); setUnderscoreValue(p, "fa_lse"); - Assert.assertNull(p.getBoolean(KEY, null)); + Assertions.assertNull(p.getBoolean(KEY, null)); p.unset(KEY); } { //int final Integer expected = 1000000; - Assert.assertNull(p.getInt(KEY, null)); + Assertions.assertNull(p.getInt(KEY, null)); setUnderscoreValue(p, "1_000_000"); - Assert.assertEquals(expected, p.getInt(KEY, null)); + Assertions.assertEquals(expected, p.getInt(KEY, null)); setNonUnderscoreValue(p, "1000000"); - Assert.assertEquals(expected, p.getInt(KEY, null)); + Assertions.assertEquals(expected, p.getInt(KEY, null)); p.unset(KEY); } { // long final Long expected = 1_000_000_000_000L; - Assert.assertNull(p.getLong(KEY, null)); + Assertions.assertNull(p.getLong(KEY, null)); setUnderscoreValue(p, "1_000_000_000_000"); - Assert.assertEquals(expected, p.getLong(KEY, null)); + Assertions.assertEquals(expected, p.getLong(KEY, null)); setNonUnderscoreValue(p, "1000000000000"); - Assert.assertEquals(expected, p.getLong(KEY, null)); + Assertions.assertEquals(expected, p.getLong(KEY, null)); p.unset(KEY); } { // File final String expected = "1_000_000"; - Assert.assertNull(p.getFile(KEY, null)); + Assertions.assertNull(p.getFile(KEY, null)); setUnderscoreValue(p, expected); - Assert.assertEquals(expected, p.getFile(KEY, null).getName()); + Assertions.assertEquals(expected, p.getFile(KEY, null).getName()); p.unset(KEY); } { // class final Type expected = Type.APPEND_ENTRIES; - Assert.assertNull(p.getEnum(KEY, Type.class, null)); + Assertions.assertNull(p.getEnum(KEY, Type.class, null)); setUnderscoreValue(p, expected.name()); - Assert.assertEquals(expected, p.getEnum(KEY, Type.class, null)); + Assertions.assertEquals(expected, p.getEnum(KEY, Type.class, null)); p.unset(KEY); } { // enum final Class expected = Request_Vote.class; - Assert.assertNull(p.getClass(KEY, null)); + Assertions.assertNull(p.getClass(KEY, null)); setUnderscoreValue(p, expected.getName()); - Assert.assertEquals(expected, p.getClass(KEY, null)); + Assertions.assertEquals(expected, p.getClass(KEY, null)); p.unset(KEY); } } diff --git a/ratis-test/src/test/java/org/apache/ratis/datastream/DataStreamAsyncClusterTests.java b/ratis-test/src/test/java/org/apache/ratis/datastream/DataStreamAsyncClusterTests.java index 8c315070e5..2fcf500e2c 100644 --- a/ratis-test/src/test/java/org/apache/ratis/datastream/DataStreamAsyncClusterTests.java +++ b/ratis-test/src/test/java/org/apache/ratis/datastream/DataStreamAsyncClusterTests.java @@ -34,8 +34,8 @@ import org.apache.ratis.util.Slf4jUtils; import org.apache.ratis.util.TimeDuration; import org.apache.ratis.util.function.CheckedBiFunction; -import org.junit.Assert; -import org.junit.Test; +import org.junit.jupiter.api.Assertions; +import org.junit.jupiter.api.Test; import org.slf4j.event.Level; import java.io.IOException; @@ -130,14 +130,14 @@ void runTestDataStream(CLUSTER cluster, boolean stepDownLeader, CheckedBiFunctio // wait for all servers to catch up try (RaftClient client = cluster.createClient()) { RaftClientReply reply = client.async().watch(maxIndex, ReplicationLevel.ALL).join(); - Assert.assertTrue(reply.isSuccess()); + Assertions.assertTrue(reply.isSuccess()); } // assert all streams are linked for (RaftServer proxy : cluster.getServers()) { final RaftServer.Division impl = proxy.getDivision(cluster.getGroupId()); final MultiDataStreamStateMachine stateMachine = (MultiDataStreamStateMachine) impl.getStateMachine(); for (SingleDataStream s : stateMachine.getStreams()) { - Assert.assertFalse(s.getDataChannel().isOpen()); + Assertions.assertFalse(s.getDataChannel().isOpen()); DataStreamTestUtils.assertLogEntry(impl, s); } } @@ -150,7 +150,7 @@ Long runTestDataStream( futures.add(CompletableFuture.supplyAsync( () -> runTestDataStream(cluster, numStreams, bufferSize, bufferNum, stepDownLeader), executor)); } - Assert.assertEquals(numClients, futures.size()); + Assertions.assertEquals(numClients, futures.size()); return futures.stream() .map(CompletableFuture::join) .max(Long::compareTo) @@ -174,7 +174,7 @@ long runTestDataStream(CLUSTER cluster, int numStreams, int bufferSize, int buff futures.add(CompletableFuture.supplyAsync(() -> DataStreamTestUtils.writeAndCloseAndAssertReplies( servers, leader, out, bufferSize, bufferNum, client.getId(), stepDownLeader).join(), executor)); } - Assert.assertEquals(numStreams, futures.size()); + Assertions.assertEquals(numStreams, futures.size()); return futures.stream() .map(CompletableFuture::join) .map(RaftClientReply::getLogIndex) diff --git a/ratis-test/src/test/java/org/apache/ratis/datastream/DataStreamBaseTest.java b/ratis-test/src/test/java/org/apache/ratis/datastream/DataStreamBaseTest.java index 3f241a8b9c..70e26af249 100644 --- a/ratis-test/src/test/java/org/apache/ratis/datastream/DataStreamBaseTest.java +++ b/ratis-test/src/test/java/org/apache/ratis/datastream/DataStreamBaseTest.java @@ -34,7 +34,7 @@ import org.apache.ratis.server.RaftServer; import org.apache.ratis.server.impl.RaftServerTestUtil; import org.apache.ratis.util.CollectionUtils; -import org.junit.Assert; +import org.junit.jupiter.api.Assertions; import java.io.IOException; import java.util.ArrayList; @@ -132,10 +132,10 @@ void runTestMockCluster(ClientId clientId, int bufferSize, int bufferNum, .stream(null, DataStreamTestUtils.getRoutingTableChainTopology(peers, getPrimaryServer().getPeer())); if (headerException != null) { final DataStreamReply headerReply = out.getHeaderFuture().join(); - Assert.assertFalse(headerReply.isSuccess()); + Assertions.assertFalse(headerReply.isSuccess()); final RaftClientReply clientReply = ClientProtoUtils.toRaftClientReply( ((DataStreamReplyByteBuffer)headerReply).slice()); - Assert.assertTrue(clientReply.getException().getMessage().contains(headerException.getMessage())); + Assertions.assertTrue(clientReply.getException().getMessage().contains(headerException.getMessage())); return; } @@ -143,11 +143,11 @@ void runTestMockCluster(ClientId clientId, int bufferSize, int bufferNum, CollectionUtils.as(servers, Server::getRaftServer), null, out, bufferSize, bufferNum, client.getId(), false).join(); if (expectedException != null) { - Assert.assertFalse(clientReply.isSuccess()); - Assert.assertTrue(clientReply.getException().getMessage().contains( + Assertions.assertFalse(clientReply.isSuccess()); + Assertions.assertTrue(clientReply.getException().getMessage().contains( expectedException.getMessage())); } else { - Assert.assertTrue(clientReply.isSuccess()); + Assertions.assertTrue(clientReply.isSuccess()); } } } diff --git a/ratis-test/src/test/java/org/apache/ratis/datastream/DataStreamClusterTests.java b/ratis-test/src/test/java/org/apache/ratis/datastream/DataStreamClusterTests.java index 352d98e650..f99ff56236 100644 --- a/ratis-test/src/test/java/org/apache/ratis/datastream/DataStreamClusterTests.java +++ b/ratis-test/src/test/java/org/apache/ratis/datastream/DataStreamClusterTests.java @@ -36,8 +36,8 @@ import org.apache.ratis.util.FileUtils; import org.apache.ratis.util.Timestamp; import org.apache.ratis.util.function.CheckedConsumer; -import org.junit.Assert; -import org.junit.Test; +import org.junit.jupiter.api.Assertions; +import org.junit.jupiter.api.Test; import java.io.File; import java.nio.channels.FileChannel; @@ -115,9 +115,9 @@ void runTestInvalidPrimaryInRoutingTable(CLUSTER cluster) throws Exception { } } - Assert.assertNotNull( - "Cannot find peer other than the primary", notPrimary); - Assert.assertNotEquals(primaryServer, notPrimary); + Assertions.assertNotNull(notPrimary, + "Cannot find peer other than the primary"); + Assertions.assertNotEquals(primaryServer, notPrimary); try (RaftClient client = cluster.createClient(primaryServer)) { RoutingTable routingTableWithWrongPrimary = @@ -156,7 +156,7 @@ static CheckedConsumer transferToWritableByteCh public void accept(DataStreamOutputImpl out) throws Exception { try (FileChannel in = FileUtils.newFileChannel(f, StandardOpenOption.READ)) { final long transferred = in.transferTo(0, size, out.getWritableByteChannel()); - Assert.assertEquals(size, transferred); + Assertions.assertEquals(size, transferred); } } @@ -196,7 +196,7 @@ void assertLogEntry(CLUSTER cluster, RaftClientRequest request) throws Exception final RaftServer.Division impl = proxy.getDivision(cluster.getGroupId()); final MultiDataStreamStateMachine stateMachine = (MultiDataStreamStateMachine) impl.getStateMachine(); final SingleDataStream s = stateMachine.getSingleDataStream(request); - Assert.assertFalse(s.getDataChannel().isOpen()); + Assertions.assertFalse(s.getDataChannel().isOpen()); DataStreamTestUtils.assertLogEntry(impl, s); } } diff --git a/ratis-test/src/test/java/org/apache/ratis/datastream/DataStreamTestUtils.java b/ratis-test/src/test/java/org/apache/ratis/datastream/DataStreamTestUtils.java index 738cb0359c..e4a930f1d1 100644 --- a/ratis-test/src/test/java/org/apache/ratis/datastream/DataStreamTestUtils.java +++ b/ratis-test/src/test/java/org/apache/ratis/datastream/DataStreamTestUtils.java @@ -49,7 +49,7 @@ import org.apache.ratis.util.CollectionUtils; import org.apache.ratis.util.FileUtils; import org.apache.ratis.util.JavaUtils; -import org.junit.Assert; +import org.junit.jupiter.api.Assertions; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -87,7 +87,7 @@ static ByteBuffer initBuffer(int offset, int size) { buffer.put(pos2byte(offset + j)); } buffer.flip(); - Assert.assertEquals(length, buffer.remaining()); + Assertions.assertEquals(length, buffer.remaining()); return buffer; } @@ -117,7 +117,7 @@ public int read(ByteBuffer dst) { FileUtils.createDirectories(f.getParentFile()); try(FileChannel out = FileUtils.newFileChannel(f, StandardOpenOption.CREATE, StandardOpenOption.WRITE)) { final long transferred = out.transferFrom(source, 0, size); - Assert.assertEquals(size, transferred); + Assertions.assertEquals(size, transferred); } } @@ -253,7 +253,7 @@ public int write(ByteBuffer src) { } final int remaining = src.remaining(); for (; src.remaining() > 0; ) { - Assert.assertEquals(pos2byte(bytesWritten), src.get()); + Assertions.assertEquals(pos2byte(bytesWritten), src.get()); bytesWritten += 1; } return remaining; @@ -302,9 +302,9 @@ static int writeAndAssertReplies(DataStreamOutputImpl out, int bufferSize, int b } static void assertSuccessReply(Type expectedType, long expectedBytesWritten, DataStreamReply reply) { - Assert.assertTrue(reply.isSuccess()); - Assert.assertEquals(expectedBytesWritten, reply.getBytesWritten()); - Assert.assertEquals(expectedType, reply.getType()); + Assertions.assertTrue(reply.isSuccess()); + Assertions.assertEquals(expectedBytesWritten, reply.getBytesWritten()); + Assertions.assertEquals(expectedType, reply.getType()); } static CompletableFuture writeAndCloseAndAssertReplies( @@ -328,26 +328,26 @@ static CompletableFuture writeAndCloseAndAssertReplies( static void assertHeader(RaftServer server, RaftClientRequest header, int dataSize, boolean stepDownLeader) throws Exception { // check header - Assert.assertEquals(RaftClientRequest.dataStreamRequestType(), header.getType()); + Assertions.assertEquals(RaftClientRequest.dataStreamRequestType(), header.getType()); // check stream final MultiDataStreamStateMachine stateMachine = (MultiDataStreamStateMachine) server.getDivision(header.getRaftGroupId()).getStateMachine(); final SingleDataStream stream = stateMachine.getSingleDataStream(header); final MyDataChannel channel = stream.getDataChannel(); - Assert.assertEquals(dataSize, channel.getBytesWritten()); - Assert.assertEquals(dataSize, channel.getForcedPosition()); + Assertions.assertEquals(dataSize, channel.getBytesWritten()); + Assertions.assertEquals(dataSize, channel.getForcedPosition()); // check writeRequest final RaftClientRequest writeRequest = stream.getWriteRequest(); - Assert.assertEquals(RaftClientRequest.dataStreamRequestType(), writeRequest.getType()); + Assertions.assertEquals(RaftClientRequest.dataStreamRequestType(), writeRequest.getType()); assertRaftClientMessage(header, null, writeRequest, header.getClientId(), stepDownLeader); } static CompletableFuture assertCloseReply(DataStreamOutputImpl out, DataStreamReply dataStreamReply, long bytesWritten, RaftPeerId leader, ClientId clientId, boolean stepDownLeader) { // Test close idempotent - Assert.assertSame(dataStreamReply, out.closeAsync().join()); - Assert.assertEquals(dataStreamReply.getClientId(), clientId); + Assertions.assertSame(dataStreamReply, out.closeAsync().join()); + Assertions.assertEquals(dataStreamReply.getClientId(), clientId); BaseTest.testFailureCase("writeAsync should fail", () -> out.writeAsync(DataStreamRequestByteBuffer.EMPTY_BYTE_BUFFER).join(), CompletionException.class, (Logger) null, AlreadyClosedException.class); @@ -359,7 +359,7 @@ static CompletableFuture assertCloseReply(DataStreamOutputImpl if (reply.isSuccess()) { final ByteString bytes = reply.getMessage().getContent(); if (!bytes.equals(MOCK)) { - Assert.assertEquals(bytesWritten2ByteString(bytesWritten), bytes); + Assertions.assertEquals(bytesWritten2ByteString(bytesWritten), bytes); } } @@ -372,13 +372,13 @@ static CompletableFuture assertCloseReply(DataStreamOutputImpl static void assertRaftClientMessage( RaftClientMessage expected, RaftPeerId expectedServerId, RaftClientMessage computed, ClientId expectedClientId, boolean stepDownLeader) { - Assert.assertNotNull(computed); - Assert.assertEquals(expectedClientId, computed.getClientId()); + Assertions.assertNotNull(computed); + Assertions.assertEquals(expectedClientId, computed.getClientId()); if (!stepDownLeader) { - Assert.assertEquals( + Assertions.assertEquals( Optional.ofNullable(expectedServerId).orElseGet(expected::getServerId), computed.getServerId()); } - Assert.assertEquals(expected.getRaftGroupId(), computed.getRaftGroupId()); + Assertions.assertEquals(expected.getRaftGroupId(), computed.getRaftGroupId()); } static LogEntryProto searchLogEntry(ClientInvocationId invocationId, RaftLog log) throws Exception { @@ -394,12 +394,12 @@ static LogEntryProto searchLogEntry(ClientInvocationId invocationId, RaftLog log } static void assertLogEntry(LogEntryProto logEntry, RaftClientRequest request) { - Assert.assertNotNull(logEntry); - Assert.assertTrue(logEntry.hasStateMachineLogEntry()); + Assertions.assertNotNull(logEntry); + Assertions.assertTrue(logEntry.hasStateMachineLogEntry()); final StateMachineLogEntryProto s = logEntry.getStateMachineLogEntry(); - Assert.assertEquals(StateMachineLogEntryProto.Type.DATASTREAM, s.getType()); - Assert.assertEquals(request.getCallId(), s.getCallId()); - Assert.assertEquals(request.getClientId().toByteString(), s.getClientId()); + Assertions.assertEquals(StateMachineLogEntryProto.Type.DATASTREAM, s.getType()); + Assertions.assertEquals(request.getCallId(), s.getCallId()); + Assertions.assertEquals(request.getClientId().toByteString(), s.getClientId()); } static void assertLogEntry(RaftServer.Division division, SingleDataStream stream) throws Exception { @@ -408,6 +408,6 @@ static void assertLogEntry(RaftServer.Division division, SingleDataStream stream assertLogEntry(entryFromStream, request); final LogEntryProto entryFromLog = searchLogEntry(ClientInvocationId.valueOf(request), division.getRaftLog()); - Assert.assertEquals(entryFromStream, entryFromLog); + Assertions.assertEquals(entryFromStream, entryFromLog); } } diff --git a/ratis-test/src/test/java/org/apache/ratis/datastream/MiniRaftClusterWithRpcTypeGrpcAndDataStreamTypeNetty.java b/ratis-test/src/test/java/org/apache/ratis/datastream/MiniRaftClusterWithRpcTypeGrpcAndDataStreamTypeNetty.java index 3396ada9b3..b9e20fb82e 100644 --- a/ratis-test/src/test/java/org/apache/ratis/datastream/MiniRaftClusterWithRpcTypeGrpcAndDataStreamTypeNetty.java +++ b/ratis-test/src/test/java/org/apache/ratis/datastream/MiniRaftClusterWithRpcTypeGrpcAndDataStreamTypeNetty.java @@ -17,7 +17,6 @@ */ package org.apache.ratis.datastream; -import org.apache.ratis.security.TlsConf; import org.apache.ratis.server.impl.MiniRaftCluster; import org.apache.ratis.RaftConfigKeys; import org.apache.ratis.conf.Parameters; diff --git a/ratis-test/src/test/java/org/apache/ratis/datastream/TestDataStreamDisabled.java b/ratis-test/src/test/java/org/apache/ratis/datastream/TestDataStreamDisabled.java index 168a1b02dc..613bb69752 100644 --- a/ratis-test/src/test/java/org/apache/ratis/datastream/TestDataStreamDisabled.java +++ b/ratis-test/src/test/java/org/apache/ratis/datastream/TestDataStreamDisabled.java @@ -26,14 +26,15 @@ import org.apache.ratis.protocol.RaftGroup; import org.apache.ratis.protocol.RaftGroupId; import org.apache.ratis.protocol.RaftPeer; -import org.junit.Assert; -import org.junit.Test; +import org.junit.jupiter.api.Assertions; +import org.junit.jupiter.api.Test; +@SuppressWarnings({"try"}) public class TestDataStreamDisabled extends BaseTest { @Test public void testDataStreamDisabled() throws Exception { final RaftProperties properties = new RaftProperties(); - Assert.assertEquals(SupportedDataStreamType.DISABLED, RaftConfigKeys.DataStream.type(properties, LOG::info)); + Assertions.assertEquals(SupportedDataStreamType.DISABLED, RaftConfigKeys.DataStream.type(properties, LOG::info)); final RaftPeer server = RaftPeer.newBuilder().setId("s0").build(); @@ -44,9 +45,9 @@ public void testDataStreamDisabled() throws Exception { .setProperties(properties) .build(); DataStreamOutput out = client.getDataStreamApi().stream()) { - Assert.fail("Unexpected object: " + out); + Assertions.fail("Unexpected object: " + out); } catch (UnsupportedOperationException e) { - Assert.assertTrue(e.getMessage().contains( + Assertions.assertTrue(e.getMessage().contains( DisabledDataStreamClientFactory.class.getName() + "$1 does not support streamAsync")); } } diff --git a/ratis-test/src/test/java/org/apache/ratis/datastream/TestDataStreamSslWithRpcTypeGrpcAndDataStreamTypeNetty.java b/ratis-test/src/test/java/org/apache/ratis/datastream/TestDataStreamSslWithRpcTypeGrpcAndDataStreamTypeNetty.java index 8e423ab293..8e6d892c83 100644 --- a/ratis-test/src/test/java/org/apache/ratis/datastream/TestDataStreamSslWithRpcTypeGrpcAndDataStreamTypeNetty.java +++ b/ratis-test/src/test/java/org/apache/ratis/datastream/TestDataStreamSslWithRpcTypeGrpcAndDataStreamTypeNetty.java @@ -24,7 +24,7 @@ import org.apache.ratis.security.TlsConf; import org.apache.ratis.util.JavaUtils; import org.apache.ratis.util.Slf4jUtils; -import org.junit.Ignore; +import org.junit.jupiter.api.Disabled; import org.slf4j.event.Level; import java.util.function.Supplier; @@ -55,22 +55,22 @@ public MiniRaftClusterWithRpcTypeGrpcAndDataStreamTypeNetty.Factory getFactory() return new MiniRaftClusterWithRpcTypeGrpcAndDataStreamTypeNetty.Factory(getParameters()); } - @Ignore + @Disabled @Override public void testStreamWrites() { } - @Ignore + @Disabled @Override public void testStreamWithInvalidRoutingTable() { } - @Ignore + @Disabled @Override public void testMultipleStreamsMultipleServers() { } - @Ignore + @Disabled @Override public void testMultipleStreamsMultipleServersStepDownLeader() { } diff --git a/ratis-test/src/test/java/org/apache/ratis/datastream/TestNettyDataStreamChainTopologyWithGrpcCluster.java b/ratis-test/src/test/java/org/apache/ratis/datastream/TestNettyDataStreamChainTopologyWithGrpcCluster.java index 31b28b4c2d..778ee8225c 100644 --- a/ratis-test/src/test/java/org/apache/ratis/datastream/TestNettyDataStreamChainTopologyWithGrpcCluster.java +++ b/ratis-test/src/test/java/org/apache/ratis/datastream/TestNettyDataStreamChainTopologyWithGrpcCluster.java @@ -22,13 +22,13 @@ import org.apache.ratis.netty.NettyConfigKeys; import org.apache.ratis.util.SizeInBytes; import org.apache.ratis.util.TimeDuration; -import org.junit.Before; +import org.junit.jupiter.api.BeforeEach; public class TestNettyDataStreamChainTopologyWithGrpcCluster extends DataStreamAsyncClusterTests implements MiniRaftClusterWithRpcTypeGrpcAndDataStreamTypeNetty.FactoryGet { - @Before + @BeforeEach public void setup() { final RaftProperties p = getProperties(); RaftClientConfigKeys.DataStream.setRequestTimeout(p, TimeDuration.ONE_MINUTE); diff --git a/ratis-test/src/test/java/org/apache/ratis/datastream/TestNettyDataStreamStarTopologyWithGrpcCluster.java b/ratis-test/src/test/java/org/apache/ratis/datastream/TestNettyDataStreamStarTopologyWithGrpcCluster.java index 45247d489a..bd80d6b6b5 100644 --- a/ratis-test/src/test/java/org/apache/ratis/datastream/TestNettyDataStreamStarTopologyWithGrpcCluster.java +++ b/ratis-test/src/test/java/org/apache/ratis/datastream/TestNettyDataStreamStarTopologyWithGrpcCluster.java @@ -25,7 +25,7 @@ import org.apache.ratis.protocol.RoutingTable; import org.apache.ratis.util.SizeInBytes; import org.apache.ratis.util.TimeDuration; -import org.junit.Before; +import org.junit.jupiter.api.BeforeEach; import java.util.Collection; import java.util.List; @@ -35,7 +35,7 @@ public class TestNettyDataStreamStarTopologyWithGrpcCluster extends DataStreamAsyncClusterTests implements MiniRaftClusterWithRpcTypeGrpcAndDataStreamTypeNetty.FactoryGet { - @Before + @BeforeEach public void setup() { final RaftProperties p = getProperties(); RaftClientConfigKeys.DataStream.setRequestTimeout(p, TimeDuration.ONE_MINUTE); diff --git a/ratis-test/src/test/java/org/apache/ratis/datastream/TestNettyDataStreamWithMock.java b/ratis-test/src/test/java/org/apache/ratis/datastream/TestNettyDataStreamWithMock.java index 27a1ee102f..503f8cf66e 100644 --- a/ratis-test/src/test/java/org/apache/ratis/datastream/TestNettyDataStreamWithMock.java +++ b/ratis-test/src/test/java/org/apache/ratis/datastream/TestNettyDataStreamWithMock.java @@ -36,14 +36,14 @@ import org.apache.ratis.server.RaftServer; import org.apache.ratis.server.impl.RaftServerTestUtil; import org.apache.ratis.util.NetUtils; -import org.junit.Before; -import org.junit.Ignore; -import org.junit.Test; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.Disabled; +import org.junit.jupiter.api.Test; import org.mockito.Mockito; import static org.mockito.Mockito.mock; import static org.mockito.Mockito.when; -@Ignore +@Disabled public class TestNettyDataStreamWithMock extends DataStreamBaseTest { static RaftPeer newRaftPeer(RaftServer server) { return RaftPeer.newBuilder() @@ -53,7 +53,7 @@ static RaftPeer newRaftPeer(RaftServer server) { .build(); } - @Before + @BeforeEach public void setup() { properties = new RaftProperties(); RaftConfigKeys.DataStream.setType(properties, SupportedDataStreamType.NETTY); diff --git a/ratis-test/src/test/java/org/apache/ratis/datastream/TestNettyDataStreamWithNettyCluster.java b/ratis-test/src/test/java/org/apache/ratis/datastream/TestNettyDataStreamWithNettyCluster.java index 90af31425f..1a29d014e4 100644 --- a/ratis-test/src/test/java/org/apache/ratis/datastream/TestNettyDataStreamWithNettyCluster.java +++ b/ratis-test/src/test/java/org/apache/ratis/datastream/TestNettyDataStreamWithNettyCluster.java @@ -17,9 +17,10 @@ */ package org.apache.ratis.datastream; -import org.junit.Ignore; -@Ignore("Ignored by runzhiwang, because NettyClientRpc does not support sendRequestAsync") +import org.junit.jupiter.api.Disabled; + +@Disabled("Ignored by runzhiwang, because NettyClientRpc does not support sendRequestAsync") public class TestNettyDataStreamWithNettyCluster extends DataStreamClusterTests implements MiniRaftClusterWithRpcTypeNettyAndDataStreamTypeNetty.FactoryGet { diff --git a/ratis-test/src/test/java/org/apache/ratis/grpc/TestGrpcFactory.java b/ratis-test/src/test/java/org/apache/ratis/grpc/TestGrpcFactory.java index 76fbcee5eb..99a395d8a1 100644 --- a/ratis-test/src/test/java/org/apache/ratis/grpc/TestGrpcFactory.java +++ b/ratis-test/src/test/java/org/apache/ratis/grpc/TestGrpcFactory.java @@ -18,16 +18,14 @@ package org.apache.ratis.grpc; import org.apache.ratis.BaseTest; -import org.apache.ratis.util.JavaUtils; -import org.junit.Assert; -import org.junit.Test; +import org.junit.jupiter.api.Assertions; +import org.junit.jupiter.api.Test; public class TestGrpcFactory extends BaseTest { @Test public void testUseCacheForAllThreads() { // trigger GrpcFactory static initializer final boolean value = GrpcFactory.checkPooledByteBufAllocatorUseCacheForAllThreads(LOG::info); - Assert.assertFalse(value); - LOG.info("value is {}", value); + Assertions.assertFalse(value); } } \ No newline at end of file diff --git a/ratis-test/src/test/java/org/apache/ratis/grpc/TestGrpcMessageMetrics.java b/ratis-test/src/test/java/org/apache/ratis/grpc/TestGrpcMessageMetrics.java index 2abba79300..aee13223b8 100644 --- a/ratis-test/src/test/java/org/apache/ratis/grpc/TestGrpcMessageMetrics.java +++ b/ratis-test/src/test/java/org/apache/ratis/grpc/TestGrpcMessageMetrics.java @@ -29,8 +29,8 @@ import org.apache.ratis.server.impl.RaftServerTestUtil; import org.apache.ratis.util.JavaUtils; import org.apache.ratis.util.TimeDuration; -import org.junit.Assert; -import org.junit.Test; +import org.junit.jupiter.api.Assertions; +import org.junit.jupiter.api.Test; import java.util.concurrent.CompletableFuture; import java.util.concurrent.TimeUnit; @@ -68,6 +68,6 @@ static void assertMessageCount(RaftServer.Division server) { GrpcService service = (GrpcService) RaftServerTestUtil.getServerRpc(server); RatisMetricRegistry registry = service.getServerInterceptor().getMetrics().getRegistry(); String counter_prefix = serverId + "_" + "ratis.grpc.RaftServerProtocolService"; - Assert.assertTrue(registry.counter(counter_prefix + "_" + "requestVote" + "_OK_completed_total").getCount() > 0); + Assertions.assertTrue(registry.counter(counter_prefix + "_" + "requestVote" + "_OK_completed_total").getCount() > 0); } } \ No newline at end of file diff --git a/ratis-test/src/test/java/org/apache/ratis/grpc/TestLeaderElectionWithGrpc.java b/ratis-test/src/test/java/org/apache/ratis/grpc/TestLeaderElectionWithGrpc.java index 7730cb1166..ef6bc2a866 100644 --- a/ratis-test/src/test/java/org/apache/ratis/grpc/TestLeaderElectionWithGrpc.java +++ b/ratis-test/src/test/java/org/apache/ratis/grpc/TestLeaderElectionWithGrpc.java @@ -19,7 +19,7 @@ import org.apache.ratis.server.impl.BlockRequestHandlingInjection; import org.apache.ratis.server.impl.LeaderElectionTests; -import org.junit.Test; +import org.junit.jupiter.api.Test; public class TestLeaderElectionWithGrpc extends LeaderElectionTests diff --git a/ratis-test/src/test/java/org/apache/ratis/grpc/TestLeaderInstallSnapshot.java b/ratis-test/src/test/java/org/apache/ratis/grpc/TestLeaderInstallSnapshot.java index 5f7a40f0f4..22c590c9dd 100644 --- a/ratis-test/src/test/java/org/apache/ratis/grpc/TestLeaderInstallSnapshot.java +++ b/ratis-test/src/test/java/org/apache/ratis/grpc/TestLeaderInstallSnapshot.java @@ -18,23 +18,32 @@ package org.apache.ratis.grpc; import org.apache.ratis.InstallSnapshotFromLeaderTests; -import org.junit.runner.RunWith; -import org.junit.runners.Parameterized; +import org.junit.jupiter.params.ParameterizedTest; +import org.junit.jupiter.params.provider.MethodSource; import java.util.Arrays; import java.util.Collection; -@RunWith(Parameterized.class) public class TestLeaderInstallSnapshot extends InstallSnapshotFromLeaderTests implements MiniRaftClusterWithGrpc.FactoryGet { - public TestLeaderInstallSnapshot(Boolean separateHeartbeat) { + public static Collection data() { + return Arrays.asList((new Boolean[][] {{Boolean.FALSE}, {Boolean.TRUE}})); + } + + @ParameterizedTest + @MethodSource("data") + public void testMultiFileInstallSnapshot(Boolean separateHeartbeat) throws Exception { GrpcConfigKeys.Server.setHeartbeatChannel(getProperties(), separateHeartbeat); + super.testMultiFileInstallSnapshot(); } - @Parameterized.Parameters - public static Collection data() { - return Arrays.asList((new Boolean[][] {{Boolean.FALSE}, {Boolean.TRUE}})); + @ParameterizedTest + @MethodSource("data") + public void testSeparateSnapshotInstallPath(Boolean separateHeartbeat) throws Exception { + GrpcConfigKeys.Server.setHeartbeatChannel(getProperties(), separateHeartbeat); + super.testSeparateSnapshotInstallPath(); } + } diff --git a/ratis-test/src/test/java/org/apache/ratis/grpc/TestLogAppenderWithGrpc.java b/ratis-test/src/test/java/org/apache/ratis/grpc/TestLogAppenderWithGrpc.java index c0d102f957..107cd7ba9a 100644 --- a/ratis-test/src/test/java/org/apache/ratis/grpc/TestLogAppenderWithGrpc.java +++ b/ratis-test/src/test/java/org/apache/ratis/grpc/TestLogAppenderWithGrpc.java @@ -33,10 +33,9 @@ import org.apache.ratis.statemachine.StateMachine; import org.apache.ratis.util.JavaUtils; import org.apache.ratis.util.Slf4jUtils; -import org.junit.Assert; -import org.junit.Test; -import org.junit.runner.RunWith; -import org.junit.runners.Parameterized; +import org.junit.jupiter.api.Assertions; +import org.junit.jupiter.params.ParameterizedTest; +import org.junit.jupiter.params.provider.MethodSource; import org.slf4j.event.Level; import java.io.IOException; @@ -47,7 +46,6 @@ import static org.apache.ratis.RaftTestUtil.waitForLeader; -@RunWith(Parameterized.class) public class TestLogAppenderWithGrpc extends LogAppenderTests implements MiniRaftClusterWithGrpc.FactoryGet { @@ -55,17 +53,14 @@ public class TestLogAppenderWithGrpc Slf4jUtils.setLogLevel(FollowerInfo.LOG, Level.DEBUG); } - public TestLogAppenderWithGrpc(Boolean separateHeartbeat) { - GrpcConfigKeys.Server.setHeartbeatChannel(getProperties(), separateHeartbeat); - } - - @Parameterized.Parameters public static Collection data() { return Arrays.asList((new Boolean[][] {{Boolean.FALSE}, {Boolean.TRUE}})); } - @Test - public void testPendingLimits() throws IOException, InterruptedException { + @ParameterizedTest + @MethodSource("data") + public void testPendingLimits(Boolean separateHeartbeat) throws IOException, InterruptedException { + GrpcConfigKeys.Server.setHeartbeatChannel(getProperties(), separateHeartbeat); int maxAppends = 10; RaftProperties properties = new RaftProperties(); properties.setClass(MiniRaftCluster.STATEMACHINE_CLASS_KEY, @@ -94,7 +89,7 @@ public void testPendingLimits() throws IOException, InterruptedException { JavaUtils.attempt(() -> { for (long nextIndex : leader.getInfo().getFollowerNextIndices()) { // Verify nextIndex does not progress due to pendingRequests limit - Assert.assertEquals(initialNextIndex + maxAppends, nextIndex); + Assertions.assertEquals(initialNextIndex + maxAppends, nextIndex); } }, 10, ONE_SECOND, "matching nextIndex", LOG); for (RaftServer.Division server : cluster.getFollowers()) { @@ -107,8 +102,10 @@ public void testPendingLimits() throws IOException, InterruptedException { } } - @Test - public void testRestartLogAppender() throws Exception { + @ParameterizedTest + @MethodSource("data") + public void testRestartLogAppender(Boolean separateHeartbeat) throws Exception { + GrpcConfigKeys.Server.setHeartbeatChannel(getProperties(), separateHeartbeat); runWithNewCluster(2, this::runTestRestartLogAppender); } @@ -120,7 +117,7 @@ private void runTestRestartLogAppender(MiniRaftClusterWithGrpc cluster) throws E try(RaftClient client = cluster.createClient(leader.getId())) { for(int i = 0; i < 10; i++) { final RaftClientReply reply = client.io().send(new RaftTestUtil.SimpleMessage("m" + ++messageCount)); - Assert.assertTrue(reply.isSuccess()); + Assertions.assertTrue(reply.isSuccess()); } } @@ -128,7 +125,7 @@ private void runTestRestartLogAppender(MiniRaftClusterWithGrpc cluster) throws E final GrpcServerMetrics leaderMetrics = new GrpcServerMetrics(leader.getMemberId().toString()); final String counter = String.format(GrpcServerMetrics.RATIS_GRPC_METRICS_LOG_APPENDER_INCONSISTENCY, cluster.getFollowers().iterator().next().getMemberId().getPeerId()); - Assert.assertEquals(0L, leaderMetrics.getRegistry().counter(counter).getCount()); + Assertions.assertEquals(0L, leaderMetrics.getRegistry().counter(counter).getCount()); // restart LogAppender RaftServerTestUtil.restartLogAppenders(leader); @@ -137,7 +134,7 @@ private void runTestRestartLogAppender(MiniRaftClusterWithGrpc cluster) throws E try(RaftClient client = cluster.createClient(leader.getId())) { for(int i = 0; i < 10; i++) { final RaftClientReply reply = client.io().send(new RaftTestUtil.SimpleMessage("m" + ++messageCount)); - Assert.assertTrue(reply.isSuccess()); + Assertions.assertTrue(reply.isSuccess()); } } @@ -148,7 +145,7 @@ private void runTestRestartLogAppender(MiniRaftClusterWithGrpc cluster) throws E // assert INCONSISTENCY counter >= 1 // If old LogAppender die before new LogAppender start, INCONSISTENCY equal to 1, // else INCONSISTENCY greater than 1 - Assert.assertTrue(newleaderMetrics.getRegistry().counter(counter).getCount() >= 1L); + Assertions.assertTrue(newleaderMetrics.getRegistry().counter(counter).getCount() >= 1L); } } } diff --git a/ratis-test/src/test/java/org/apache/ratis/grpc/TestRaftOutputStreamWithGrpc.java b/ratis-test/src/test/java/org/apache/ratis/grpc/TestRaftOutputStreamWithGrpc.java index fb35d958ab..2f1ef3f124 100644 --- a/ratis-test/src/test/java/org/apache/ratis/grpc/TestRaftOutputStreamWithGrpc.java +++ b/ratis-test/src/test/java/org/apache/ratis/grpc/TestRaftOutputStreamWithGrpc.java @@ -18,12 +18,10 @@ package org.apache.ratis.grpc; import org.apache.ratis.OutputStreamBaseTest; +import org.junit.jupiter.api.Timeout; +@Timeout(value = 100) public class TestRaftOutputStreamWithGrpc extends OutputStreamBaseTest implements MiniRaftClusterWithGrpc.FactoryGet { - @Override - public int getGlobalTimeoutSeconds() { - return 100; - } } diff --git a/ratis-test/src/test/java/org/apache/ratis/grpc/TestRaftServerWithGrpc.java b/ratis-test/src/test/java/org/apache/ratis/grpc/TestRaftServerWithGrpc.java index 0af1d87cce..05d772c17e 100644 --- a/ratis-test/src/test/java/org/apache/ratis/grpc/TestRaftServerWithGrpc.java +++ b/ratis-test/src/test/java/org/apache/ratis/grpc/TestRaftServerWithGrpc.java @@ -26,7 +26,6 @@ import org.apache.ratis.protocol.ClientInvocationId; import org.apache.ratis.server.RetryCache; import org.apache.ratis.util.JavaUtils; -import org.slf4j.event.Level; import org.apache.ratis.conf.Parameters; import org.apache.ratis.security.SecurityTestUtils; import org.apache.ratis.server.storage.RaftStorage; @@ -62,12 +61,11 @@ import org.apache.ratis.util.ProtoUtils; import org.apache.ratis.util.SizeInBytes; import org.apache.ratis.util.TimeDuration; -import org.junit.Assert; -import org.junit.Before; -import org.junit.Test; -import org.junit.runner.RunWith; -import org.junit.runners.Parameterized; - +import org.junit.jupiter.api.Assertions; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.params.ParameterizedTest; +import org.junit.jupiter.params.provider.MethodSource; +import org.slf4j.event.Level; import javax.net.ssl.KeyManager; import javax.net.ssl.TrustManager; import java.io.IOException; @@ -80,30 +78,26 @@ import java.util.concurrent.TimeUnit; import java.util.concurrent.atomic.AtomicLong; -@RunWith(Parameterized.class) public class TestRaftServerWithGrpc extends BaseTest implements MiniRaftClusterWithGrpc.FactoryGet { { Slf4jUtils.setLogLevel(GrpcClientProtocolClient.LOG, Level.TRACE); } - public TestRaftServerWithGrpc(Boolean separateHeartbeat) { - GrpcConfigKeys.Server.setHeartbeatChannel(getProperties(), separateHeartbeat); - } - - @Parameterized.Parameters public static Collection data() { return Arrays.asList((new Boolean[][] {{Boolean.FALSE}, {Boolean.TRUE}})); } - @Before + @BeforeEach public void setup() { final RaftProperties p = getProperties(); p.setClass(MiniRaftCluster.STATEMACHINE_CLASS_KEY, SimpleStateMachine4Testing.class, StateMachine.class); RaftClientConfigKeys.Rpc.setRequestTimeout(p, TimeDuration.valueOf(1, TimeUnit.SECONDS)); } - @Test - public void testServerRestartOnException() throws Exception { + @ParameterizedTest + @MethodSource("data") + public void testServerRestartOnException(Boolean separateHeartbeat) throws Exception { + GrpcConfigKeys.Server.setHeartbeatChannel(getProperties(), separateHeartbeat); runWithNewCluster(1, this::runTestServerRestartOnException); } @@ -149,8 +143,10 @@ void runTestServerRestartOnException(MiniRaftClusterWithGrpc cluster) throws Exc cluster.getServerFactory(leaderId).newRaftServerRpc(cluster.getServer(leaderId)); } - @Test - public void testUnsupportedMethods() throws Exception { + @ParameterizedTest + @MethodSource("data") + public void testUnsupportedMethods(Boolean separateHeartbeat) throws Exception { + GrpcConfigKeys.Server.setHeartbeatChannel(getProperties(), separateHeartbeat); runWithNewCluster(1, this::runTestUnsupportedMethods); } @@ -167,8 +163,10 @@ void runTestUnsupportedMethods(MiniRaftClusterWithGrpc cluster) throws Exception UnsupportedOperationException.class); } - @Test - public void testLeaderRestart() throws Exception { + @ParameterizedTest + @MethodSource("data") + public void testLeaderRestart(Boolean separateHeartbeat) throws Exception { + GrpcConfigKeys.Server.setHeartbeatChannel(getProperties(), separateHeartbeat); runWithNewCluster(3, this::runTestLeaderRestart); } @@ -178,7 +176,7 @@ void runTestLeaderRestart(MiniRaftClusterWithGrpc cluster) throws Exception { try (final RaftClient client = cluster.createClient()) { // send a request to make sure leader is ready final CompletableFuture f = client.async().send(new SimpleMessage("testing")); - Assert.assertTrue(f.get().isSuccess()); + Assertions.assertTrue(f.get().isSuccess()); } try (final RaftClient client = cluster.createClient()) { @@ -189,14 +187,14 @@ void runTestLeaderRestart(MiniRaftClusterWithGrpc cluster) throws Exception { { // send a request using rpc directly final RaftClientRequest request = newRaftClientRequest(client, seqNum.incrementAndGet()); - Assert.assertEquals(client.getId(), request.getClientId()); + Assertions.assertEquals(client.getId(), request.getClientId()); final CompletableFuture f = rpc.sendRequestAsync(request); final RaftClientReply reply = f.get(); - Assert.assertTrue(reply.isSuccess()); + Assertions.assertTrue(reply.isSuccess()); RaftClientTestUtil.handleReply(request, reply, client); invocationId = ClientInvocationId.valueOf(request.getClientId(), request.getCallId()); final RetryCache.Entry entry = leader.getRetryCache().getIfPresent(invocationId); - Assert.assertNotNull(entry); + Assertions.assertNotNull(entry); LOG.info("cache entry {}", entry); } @@ -206,13 +204,13 @@ void runTestLeaderRestart(MiniRaftClusterWithGrpc cluster) throws Exception { final RaftClientRequest requestBlocked = newRaftClientRequest(client, seqNum.incrementAndGet()); final CompletableFuture futureBlocked = rpc.sendRequestAsync(requestBlocked); - JavaUtils.attempt(() -> Assert.assertNull(leader.getRetryCache().getIfPresent(invocationId)), + JavaUtils.attempt(() -> Assertions.assertNull(leader.getRetryCache().getIfPresent(invocationId)), 10, HUNDRED_MILLIS, "invalidate cache entry", LOG); LOG.info("cache entry not found for {}", invocationId); // change leader RaftTestUtil.changeLeader(cluster, leader.getId()); - Assert.assertNotEquals(RaftPeerRole.LEADER, leader.getInfo().getCurrentRole()); + Assertions.assertNotEquals(RaftPeerRole.LEADER, leader.getInfo().getCurrentRole()); // the blocked request should fail testFailureCase("request should fail", futureBlocked::get, @@ -229,13 +227,17 @@ void runTestLeaderRestart(MiniRaftClusterWithGrpc cluster) throws Exception { } - @Test - public void testRaftClientMetrics() throws Exception { + @ParameterizedTest + @MethodSource("data") + public void testRaftClientMetrics(Boolean separateHeartbeat) throws Exception { + GrpcConfigKeys.Server.setHeartbeatChannel(getProperties(), separateHeartbeat); runWithNewCluster(3, this::testRaftClientRequestMetrics); } - @Test - public void testRaftServerMetrics() throws Exception { + @ParameterizedTest + @MethodSource("data") + public void testRaftServerMetrics(Boolean separateHeartbeat) throws Exception { + GrpcConfigKeys.Server.setHeartbeatChannel(getProperties(), separateHeartbeat); final RaftProperties p = getProperties(); RaftServerConfigKeys.Write.setElementLimit(p, 10); RaftServerConfigKeys.Write.setByteLimit(p, SizeInBytes.valueOf("1MB")); @@ -252,7 +254,7 @@ void testRequestMetrics(MiniRaftClusterWithGrpc cluster) throws Exception { try (RaftClient client = cluster.createClient()) { // send a request to make sure leader is ready final CompletableFuture< RaftClientReply > f = client.async().send(new SimpleMessage("testing")); - Assert.assertTrue(f.get().isSuccess()); + Assertions.assertTrue(f.get().isSuccess()); } SimpleStateMachine4Testing stateMachine = SimpleStateMachine4Testing.get(cluster.getLeader()); @@ -285,14 +287,14 @@ void testRequestMetrics(MiniRaftClusterWithGrpc cluster) throws Exception { client = cluster.createClient(cluster.getLeader().getId(), RetryPolicies.noRetry()); final SizeInBytes size = SizeInBytes.valueOf("1025kb"); final ByteString bytes = randomByteString(size.getSizeInt()); - Assert.assertEquals(size.getSizeInt(), bytes.size()); + Assertions.assertEquals(size.getSizeInt(), bytes.size()); client.async().send(new SimpleMessage(size + "-message", bytes)); clients.add(client); RaftTestUtil.waitFor(() -> getRaftServerMetrics(cluster.getLeader()) .getNumRequestsByteSizeLimitHits().getCount() == 1, 300, 5000); - Assert.assertEquals(2, getRaftServerMetrics(cluster.getLeader()) + Assertions.assertEquals(2, getRaftServerMetrics(cluster.getLeader()) .getNumResourceLimitHits().getCount()); } finally { for (RaftClient client : clients) { @@ -326,36 +328,36 @@ void testRaftClientRequestMetrics(MiniRaftClusterWithGrpc cluster) throws IOExce try (final RaftClient client = cluster.createClient()) { final CompletableFuture f1 = client.async().send(new SimpleMessage("testing")); - Assert.assertTrue(f1.get().isSuccess()); + Assertions.assertTrue(f1.get().isSuccess()); final DefaultTimekeeperImpl write = (DefaultTimekeeperImpl) registry.timer(RAFT_CLIENT_WRITE_REQUEST); - JavaUtils.attempt(() -> Assert.assertTrue(write.getTimer().getCount() > 0), + JavaUtils.attempt(() -> Assertions.assertTrue(write.getTimer().getCount() > 0), 3, TimeDuration.ONE_SECOND, "writeTimer metrics", LOG); final CompletableFuture f2 = client.async().sendReadOnly(new SimpleMessage("testing")); - Assert.assertTrue(f2.get().isSuccess()); + Assertions.assertTrue(f2.get().isSuccess()); final DefaultTimekeeperImpl read = (DefaultTimekeeperImpl) registry.timer(RAFT_CLIENT_READ_REQUEST); - JavaUtils.attempt(() -> Assert.assertTrue(read.getTimer().getCount() > 0), + JavaUtils.attempt(() -> Assertions.assertTrue(read.getTimer().getCount() > 0), 3, TimeDuration.ONE_SECOND, "readTimer metrics", LOG); final CompletableFuture f3 = client.async().sendStaleRead(new SimpleMessage("testing"), 0, leader.getId()); - Assert.assertTrue(f3.get().isSuccess()); + Assertions.assertTrue(f3.get().isSuccess()); final DefaultTimekeeperImpl staleRead = (DefaultTimekeeperImpl) registry.timer(RAFT_CLIENT_STALE_READ_REQUEST); - JavaUtils.attempt(() -> Assert.assertTrue(staleRead.getTimer().getCount() > 0), + JavaUtils.attempt(() -> Assertions.assertTrue(staleRead.getTimer().getCount() > 0), 3, TimeDuration.ONE_SECOND, "staleReadTimer metrics", LOG); final CompletableFuture f4 = client.async().watch(0, RaftProtos.ReplicationLevel.ALL); - Assert.assertTrue(f4.get().isSuccess()); + Assertions.assertTrue(f4.get().isSuccess()); final DefaultTimekeeperImpl watchAll = (DefaultTimekeeperImpl) registry.timer( String.format(RAFT_CLIENT_WATCH_REQUEST, "-ALL")); - JavaUtils.attempt(() -> Assert.assertTrue(watchAll.getTimer().getCount() > 0), + JavaUtils.attempt(() -> Assertions.assertTrue(watchAll.getTimer().getCount() > 0), 3, TimeDuration.ONE_SECOND, "watchAllTimer metrics", LOG); final CompletableFuture f5 = client.async().watch(0, RaftProtos.ReplicationLevel.MAJORITY); - Assert.assertTrue(f5.get().isSuccess()); + Assertions.assertTrue(f5.get().isSuccess()); final DefaultTimekeeperImpl watch = (DefaultTimekeeperImpl) registry.timer( String.format(RAFT_CLIENT_WATCH_REQUEST, "")); - JavaUtils.attempt(() -> Assert.assertTrue(watch.getTimer().getCount() > 0), + JavaUtils.attempt(() -> Assertions.assertTrue(watch.getTimer().getCount() > 0), 3, TimeDuration.ONE_SECOND, "watchTimer metrics", LOG); } } @@ -366,8 +368,10 @@ static RaftClientRequest newRaftClientRequest(RaftClient client, long seqNum) { RaftClientRequest.writeRequestType(), ProtoUtils.toSlidingWindowEntry(seqNum, seqNum == 1L)); } - @Test - public void testTlsWithKeyAndTrustManager() throws Exception { + @ParameterizedTest + @MethodSource("data") + public void testTlsWithKeyAndTrustManager(Boolean separateHeartbeat) throws Exception { + GrpcConfigKeys.Server.setHeartbeatChannel(getProperties(), separateHeartbeat); final RaftProperties p = getProperties(); RaftServerConfigKeys.Write.setElementLimit(p, 10); RaftServerConfigKeys.Write.setByteLimit(p, SizeInBytes.valueOf("1MB")); diff --git a/ratis-test/src/test/java/org/apache/ratis/grpc/TestRaftSnapshotWithGrpc.java b/ratis-test/src/test/java/org/apache/ratis/grpc/TestRaftSnapshotWithGrpc.java index 2d8524f26c..82318c43ee 100644 --- a/ratis-test/src/test/java/org/apache/ratis/grpc/TestRaftSnapshotWithGrpc.java +++ b/ratis-test/src/test/java/org/apache/ratis/grpc/TestRaftSnapshotWithGrpc.java @@ -26,7 +26,7 @@ import org.apache.ratis.metrics.RatisMetricRegistry; import org.apache.ratis.server.RaftServer; import org.apache.ratis.statemachine.RaftSnapshotBaseTest; -import org.junit.Assert; +import org.junit.jupiter.api.Assertions; public class TestRaftSnapshotWithGrpc extends RaftSnapshotBaseTest { @Override @@ -39,10 +39,10 @@ protected void verifyInstallSnapshotMetric(RaftServer.Division leader) { MetricRegistryInfo info = new MetricRegistryInfo(leader.getMemberId().toString(), "ratis_grpc", "log_appender", "Metrics for Ratis Grpc Log Appender"); Optional metricRegistry = MetricRegistries.global().get(info); - Assert.assertTrue(metricRegistry.isPresent()); + Assertions.assertTrue(metricRegistry.isPresent()); final LongCounter installSnapshotCounter = metricRegistry.get().counter("num_install_snapshot"); - Assert.assertNotNull(installSnapshotCounter); - Assert.assertTrue(installSnapshotCounter.getCount() >= 1); + Assertions.assertNotNull(installSnapshotCounter); + Assertions.assertTrue(installSnapshotCounter.getCount() >= 1); } } diff --git a/ratis-test/src/test/java/org/apache/ratis/grpc/TestRaftWithGrpc.java b/ratis-test/src/test/java/org/apache/ratis/grpc/TestRaftWithGrpc.java index 046453d582..b93621137d 100644 --- a/ratis-test/src/test/java/org/apache/ratis/grpc/TestRaftWithGrpc.java +++ b/ratis-test/src/test/java/org/apache/ratis/grpc/TestRaftWithGrpc.java @@ -30,10 +30,9 @@ import org.apache.ratis.statemachine.StateMachine; import org.apache.ratis.util.JavaUtils; import org.apache.ratis.util.TimeDuration; -import org.junit.Assert; -import org.junit.Test; -import org.junit.runner.RunWith; -import org.junit.runners.Parameterized; +import org.junit.jupiter.api.Assertions; +import org.junit.jupiter.params.ParameterizedTest; +import org.junit.jupiter.params.provider.MethodSource; import java.util.Arrays; import java.util.Collection; @@ -42,7 +41,6 @@ import static org.apache.ratis.RaftTestUtil.waitForLeader; -@RunWith(Parameterized.class) public class TestRaftWithGrpc extends RaftBasicTests implements MiniRaftClusterWithGrpc.FactoryGet { @@ -52,29 +50,29 @@ public class TestRaftWithGrpc SimpleStateMachine4Testing.class, StateMachine.class); } - public TestRaftWithGrpc(Boolean separateHeartbeat) { - GrpcConfigKeys.Server.setHeartbeatChannel(getProperties(), separateHeartbeat); - } - - @Parameterized.Parameters public static Collection data() { return Arrays.asList((new Boolean[][] {{Boolean.FALSE}, {Boolean.TRUE}})); } - @Override - @Test - public void testWithLoad() throws Exception { + @ParameterizedTest + @MethodSource("data") + public void testWithLoad(Boolean separateHeartbeat) throws Exception { + GrpcConfigKeys.Server.setHeartbeatChannel(getProperties(), separateHeartbeat); super.testWithLoad(); BlockRequestHandlingInjection.getInstance().unblockAll(); } - @Test - public void testRequestTimeout() throws Exception { + @ParameterizedTest + @MethodSource("data") + public void testRequestTimeout(Boolean separateHeartbeat) throws Exception { + GrpcConfigKeys.Server.setHeartbeatChannel(getProperties(), separateHeartbeat); runWithNewCluster(NUM_SERVERS, cluster -> testRequestTimeout(false, cluster, LOG)); } - @Test - public void testUpdateViaHeartbeat() throws Exception { + @ParameterizedTest + @MethodSource("data") + public void testUpdateViaHeartbeat(Boolean separateHeartbeat) throws Exception { + GrpcConfigKeys.Server.setHeartbeatChannel(getProperties(), separateHeartbeat); runWithNewCluster(NUM_SERVERS, this::runTestUpdateViaHeartbeat); } @@ -91,7 +89,7 @@ void runTestUpdateViaHeartbeat(MiniRaftClusterWithGrpc cluster) throws Exception replyFuture = client.async().send(new RaftTestUtil.SimpleMessage("abc")); TimeDuration.valueOf(5 , TimeUnit.SECONDS).sleep(); // replyFuture should not be completed until append request is unblocked. - Assert.assertFalse(replyFuture.isDone()); + Assertions.assertFalse(replyFuture.isDone()); // unblock append request. cluster.getServerAliveStream() .filter(impl -> !impl.getInfo().isLeader()) @@ -107,9 +105,9 @@ void runTestUpdateViaHeartbeat(MiniRaftClusterWithGrpc cluster) throws Exception final LogEntryHeader[] leaderEntries = leaderLog.getEntries(0, Long.MAX_VALUE); final RaftLog followerLog = raftServer.getRaftLog(); - Assert.assertEquals(leaderNextIndex, followerLog.getNextIndex()); + Assertions.assertEquals(leaderNextIndex, followerLog.getNextIndex()); final LogEntryHeader[] serverEntries = followerLog.getEntries(0, Long.MAX_VALUE); - Assert.assertArrayEquals(serverEntries, leaderEntries); + Assertions.assertArrayEquals(serverEntries, leaderEntries); }, 10, HUNDRED_MILLIS, "assertRaftLog-" + raftServer.getId(), LOG))); // Wait for heartbeats from leader to be received by followers @@ -119,8 +117,8 @@ void runTestUpdateViaHeartbeat(MiniRaftClusterWithGrpc cluster) throws Exception final long leaderNextIndex = leaderLog.getNextIndex(); // FollowerInfo in the leader state should have updated next and match index. final long followerMatchIndex = logAppender.getFollower().getMatchIndex(); - Assert.assertTrue(followerMatchIndex >= leaderNextIndex - 1); - Assert.assertEquals(followerMatchIndex + 1, logAppender.getFollower().getNextIndex()); + Assertions.assertTrue(followerMatchIndex >= leaderNextIndex - 1); + Assertions.assertEquals(followerMatchIndex + 1, logAppender.getFollower().getNextIndex()); }, 10, HUNDRED_MILLIS, "assertRaftLog-" + logAppender.getFollower(), LOG))); } } diff --git a/ratis-test/src/test/java/org/apache/ratis/grpc/TestRetryCacheWithGrpc.java b/ratis-test/src/test/java/org/apache/ratis/grpc/TestRetryCacheWithGrpc.java index a39a4d1ef4..879b9eabdb 100644 --- a/ratis-test/src/test/java/org/apache/ratis/grpc/TestRetryCacheWithGrpc.java +++ b/ratis-test/src/test/java/org/apache/ratis/grpc/TestRetryCacheWithGrpc.java @@ -37,8 +37,9 @@ import org.apache.ratis.util.JavaUtils; import org.apache.ratis.util.Slf4jUtils; import org.apache.ratis.util.TimeDuration; -import org.junit.Assert; -import org.junit.Test; +import org.junit.jupiter.api.Assertions; +import org.junit.jupiter.api.Test; +import org.junit.jupiter.api.Timeout; import org.slf4j.event.Level; import java.io.IOException; @@ -66,7 +67,7 @@ public void testInvalidateRepliedCalls() throws Exception { } static long assertReply(RaftClientReply reply) { - Assert.assertTrue(reply.isSuccess()); + Assertions.assertTrue(reply.isSuccess()); return reply.getCallId(); } @@ -90,7 +91,7 @@ void assertRetryCacheEntry(RaftClient client, long callId, boolean exist) throws void assertRetryCacheEntry(RaftClient client, long callId, boolean exist, boolean eventually) throws InterruptedException { Supplier lookup = () -> RetryCacheTestUtil.get(leader, client.getId(), callId); - Consumer assertion = exist ? Assert::assertNotNull : Assert::assertNull; + Consumer assertion = exist ? Assertions::assertNotNull : Assertions::assertNull; if (eventually) { JavaUtils.attempt(() -> assertion.accept(lookup.get()), 100, TimeDuration.ONE_MILLISECOND, "retry cache entry", null); @@ -144,7 +145,7 @@ void run() throws Exception { ONE_SECOND.sleep(); // No calls can be completed. for (CompletableFuture f : asyncCalls) { - Assert.assertFalse(f.isDone()); + Assertions.assertFalse(f.isDone()); } stateMachine.unblockApplyTransaction(); // No calls can be invalidated. @@ -170,7 +171,8 @@ void run() throws Exception { } } - @Test(timeout = 10000) + @Test + @Timeout(value = 10000) public void testRetryOnResourceUnavailableException() throws InterruptedException, IOException { RaftProperties properties = new RaftProperties(); diff --git a/ratis-test/src/test/java/org/apache/ratis/grpc/server/TestGrpcServerMetrics.java b/ratis-test/src/test/java/org/apache/ratis/grpc/server/TestGrpcServerMetrics.java index 04f8ded95d..3e6257683c 100644 --- a/ratis-test/src/test/java/org/apache/ratis/grpc/server/TestGrpcServerMetrics.java +++ b/ratis-test/src/test/java/org/apache/ratis/grpc/server/TestGrpcServerMetrics.java @@ -38,11 +38,12 @@ import org.apache.ratis.protocol.RaftGroupMemberId; import org.apache.ratis.protocol.RaftPeerId; import org.apache.ratis.thirdparty.com.codahale.metrics.Gauge; -import org.junit.Assert; -import org.junit.BeforeClass; -import org.junit.Test; +import org.junit.jupiter.api.Assertions; +import org.junit.jupiter.api.BeforeAll; +import org.junit.jupiter.api.Test; import org.mockito.Mockito; +@SuppressWarnings({"rawtypes"}) public class TestGrpcServerMetrics { private static GrpcServerMetrics grpcServerMetrics; private static RatisMetricRegistry ratisMetricRegistry; @@ -50,7 +51,7 @@ public class TestGrpcServerMetrics { private static RaftPeerId raftPeerId; private static RaftPeerId followerId; - @BeforeClass + @BeforeAll public static void setUp() throws Exception { raftGroupId = RaftGroupId.randomId(); raftPeerId = RaftPeerId.valueOf("TestId"); @@ -74,11 +75,11 @@ public void testGrpcLogAppenderLatencyTimer() throws Exception { final String format = RATIS_GRPC_METRICS_LOG_APPENDER_LATENCY + GrpcServerMetrics.getHeartbeatSuffix(heartbeat); final String name = String.format(format, followerId); final DefaultTimekeeperImpl t = (DefaultTimekeeperImpl) ratisMetricRegistry.timer(name); - Assert.assertEquals(0L, t.getTimer().getSnapshot().getMax()); + Assertions.assertEquals(0L, t.getTimer().getSnapshot().getMax()); req.startRequestTimer(); Thread.sleep(1000L); req.stopRequestTimer(); - Assert.assertTrue(t.getTimer().getSnapshot().getMax() > 1000L); + Assertions.assertTrue(t.getTimer().getSnapshot().getMax() > 1000L); } } @@ -89,7 +90,7 @@ public void testGrpcLogRequestTotal() { RATIS_GRPC_METRICS_REQUESTS_COUNT + GrpcServerMetrics .getHeartbeatSuffix(heartbeat)).getCount(); grpcServerMetrics.onRequestCreate(heartbeat); - Assert.assertEquals(reqTotal + 1, ratisMetricRegistry.counter( + Assertions.assertEquals(reqTotal + 1, ratisMetricRegistry.counter( RATIS_GRPC_METRICS_REQUESTS_COUNT + GrpcServerMetrics .getHeartbeatSuffix(heartbeat)).getCount()); } @@ -97,9 +98,9 @@ public void testGrpcLogRequestTotal() { @Test public void testGrpcLogRequestRetry() { - Assert.assertEquals(0L, ratisMetricRegistry.counter(RATIS_GRPC_METRICS_REQUEST_RETRY_COUNT).getCount()); + Assertions.assertEquals(0L, ratisMetricRegistry.counter(RATIS_GRPC_METRICS_REQUEST_RETRY_COUNT).getCount()); grpcServerMetrics.onRequestRetry(); - Assert.assertEquals(1L, ratisMetricRegistry.counter(RATIS_GRPC_METRICS_REQUEST_RETRY_COUNT).getCount()); + Assertions.assertEquals(1L, ratisMetricRegistry.counter(RATIS_GRPC_METRICS_REQUEST_RETRY_COUNT).getCount()); } @Test @@ -110,9 +111,9 @@ public void testGrpcLogPendingRequestCount() { pendingRequest::logRequestsSize); final String name = String.format(RATIS_GRPC_METRICS_LOG_APPENDER_PENDING_COUNT, raftPeerId); final Gauge gauge = ServerMetricsTestUtils.getGaugeWithName(name, grpcServerMetrics::getRegistry); - Assert.assertEquals(0, gauge.getValue()); + Assertions.assertEquals(0, gauge.getValue()); when(pendingRequest.logRequestsSize()).thenReturn(10); - Assert.assertEquals(10, gauge.getValue()); + Assertions.assertEquals(10, gauge.getValue()); } @Test @@ -133,8 +134,8 @@ public void testGrpcLogAppenderRequestCounters() { private void assertCounterIncremented(String counterVar, Consumer incFunction) { String counter = String.format(counterVar, raftPeerId.toString()); - Assert.assertEquals(0L, ratisMetricRegistry.counter(counter).getCount()); + Assertions.assertEquals(0L, ratisMetricRegistry.counter(counter).getCount()); incFunction.accept(raftPeerId.toString()); - Assert.assertEquals(1L, ratisMetricRegistry.counter(counter).getCount()); + Assertions.assertEquals(1L, ratisMetricRegistry.counter(counter).getCount()); } } diff --git a/ratis-test/src/test/java/org/apache/ratis/grpc/util/GrpcZeroCopyTestServer.java b/ratis-test/src/test/java/org/apache/ratis/grpc/util/GrpcZeroCopyTestServer.java index e1bfe4e222..a07872c066 100644 --- a/ratis-test/src/test/java/org/apache/ratis/grpc/util/GrpcZeroCopyTestServer.java +++ b/ratis-test/src/test/java/org/apache/ratis/grpc/util/GrpcZeroCopyTestServer.java @@ -32,7 +32,7 @@ import org.apache.ratis.thirdparty.io.grpc.stub.StreamObserver; import org.apache.ratis.util.IOUtils; import org.apache.ratis.util.TraditionalBinaryPrefix; -import org.junit.Assert; +import org.junit.jupiter.api.Assertions; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -115,10 +115,10 @@ Count getNonZeroCopyCount() { void assertCounts(int expectNumElements, long expectNumBytes) { LOG.info("ZeroCopyCount = {}", zeroCopyCount); LOG.info("nonZeroCopyCount = {}", nonZeroCopyCount); - Assert.assertEquals("zeroCopyCount.getNumElements()", expectNumElements, zeroCopyCount.getNumElements()); - Assert.assertEquals("zeroCopyCount.getNumBytes()", expectNumBytes, zeroCopyCount.getNumBytes()); - Assert.assertEquals("nonZeroCopyCount.getNumElements()", 0, nonZeroCopyCount.getNumElements()); - Assert.assertEquals("nonZeroCopyCount.getNumBytes()", 0, nonZeroCopyCount.getNumBytes()); + Assertions.assertEquals(expectNumElements, zeroCopyCount.getNumElements(), "zeroCopyCount.getNumElements()"); + Assertions.assertEquals(expectNumBytes, zeroCopyCount.getNumBytes()," zeroCopyCount.getNumBytes()"); + Assertions.assertEquals(0, nonZeroCopyCount.getNumElements(), "nonZeroCopyCount.getNumElements()"); + Assertions.assertEquals(0, nonZeroCopyCount.getNumBytes(), "nonZeroCopyCount.getNumBytes()"); } int start() throws IOException { diff --git a/ratis-test/src/test/java/org/apache/ratis/grpc/util/TestGrpcZeroCopy.java b/ratis-test/src/test/java/org/apache/ratis/grpc/util/TestGrpcZeroCopy.java index 032a9c1db5..a5f61ee269 100644 --- a/ratis-test/src/test/java/org/apache/ratis/grpc/util/TestGrpcZeroCopy.java +++ b/ratis-test/src/test/java/org/apache/ratis/grpc/util/TestGrpcZeroCopy.java @@ -26,8 +26,8 @@ import org.apache.ratis.thirdparty.io.netty.buffer.PooledByteBufAllocator; import org.apache.ratis.util.NetUtils; import org.apache.ratis.util.TraditionalBinaryPrefix; -import org.junit.Assert; -import org.junit.Test; +import org.junit.jupiter.api.Assertions; +import org.junit.jupiter.api.Test; import java.util.ArrayList; import java.util.List; @@ -60,8 +60,8 @@ static void verify(long seed, ByteString b) { random.nextBytes(array); final ByteString expected = UnsafeByteOperations.unsafeWrap(array, 0, remaining); final ByteString computed = b.substring(offset, offset + remaining); - Assert.assertEquals(expected.size(), computed.size()); - Assert.assertEquals(expected, computed); + Assertions.assertEquals(expected.size(), computed.size()); + Assertions.assertEquals(expected, computed); offset += remaining; } } @@ -99,7 +99,7 @@ public static boolean isReady() { /** Test a zero-copy marshaller is available from the versions of gRPC and Protobuf. */ @Test public void testReadiness() { - Assert.assertTrue(isReady()); + Assertions.assertTrue(isReady()); } @@ -134,7 +134,7 @@ void sendMessages(int n, GrpcZeroCopyTestClient client, GrpcZeroCopyTestServer s for (int i = 0; i < futures.size(); i++) { final String expected = GrpcZeroCopyTestServer.toReply(i, messages.get(i)); final String reply = futures.get(i).get(); - Assert.assertEquals("expected = " + expected + " != reply = " + reply, expected, reply); + Assertions.assertEquals(expected, reply, "expected = " + expected + " != reply = " + reply); server.assertCounts(numElements, numBytes); } } @@ -159,8 +159,8 @@ void sendBinaries(int n, GrpcZeroCopyTestClient client, GrpcZeroCopyTestServer s } final ByteString reply = future.get(); - Assert.assertEquals(4, reply.size()); - Assert.assertEquals(size, reply.asReadOnlyByteBuffer().getInt()); + Assertions.assertEquals(4, reply.size()); + Assertions.assertEquals(size, reply.asReadOnlyByteBuffer().getInt()); numElements++; numBytes += size; diff --git a/ratis-test/src/test/java/org/apache/ratis/grpc/util/TestStreamObserverWithTimeout.java b/ratis-test/src/test/java/org/apache/ratis/grpc/util/TestStreamObserverWithTimeout.java index d0c936aa40..b279736f39 100644 --- a/ratis-test/src/test/java/org/apache/ratis/grpc/util/TestStreamObserverWithTimeout.java +++ b/ratis-test/src/test/java/org/apache/ratis/grpc/util/TestStreamObserverWithTimeout.java @@ -25,8 +25,8 @@ import org.apache.ratis.util.StringUtils; import org.apache.ratis.util.TimeDuration; import org.apache.ratis.util.TimeoutTimer; -import org.junit.Assert; -import org.junit.Test; +import org.junit.jupiter.api.Assertions; +import org.junit.jupiter.api.Test; import org.slf4j.event.Level; import java.util.ArrayList; @@ -106,7 +106,7 @@ void runTestTimeout(int slow, Type type) throws Exception { for (; i < slow; i++) { final String expected = (i + warmup) + GrpcTestServer.GreeterImpl.toReplySuffix(messages.get(i)); final String reply = futures.get(i).get(); - Assert.assertEquals(expected, reply); + Assertions.assertEquals(expected, reply); LOG.info("{}) passed", (i + warmup)); } @@ -114,10 +114,10 @@ void runTestTimeout(int slow, Type type) throws Exception { final CompletableFuture f = futures.get(i); try { final String reply = f.get(); - Assert.fail((i + warmup) + ") reply = " + reply + ", " + Assertions.fail((i + warmup) + ") reply = " + reply + ", " + StringUtils.completableFuture2String(f, false)); } catch (ExecutionException e) { - LOG.info("GOOD! {}) {}, {}", (i + warmup), StringUtils.completableFuture2String(f, true), e); + LOG.info("GOOD! {}) {}", (i + warmup), StringUtils.completableFuture2String(f, true), e); } } } diff --git a/ratis-test/src/test/java/org/apache/ratis/netty/TestLeaderElectionWithNetty.java b/ratis-test/src/test/java/org/apache/ratis/netty/TestLeaderElectionWithNetty.java index f84bbb7360..f3b760bf22 100644 --- a/ratis-test/src/test/java/org/apache/ratis/netty/TestLeaderElectionWithNetty.java +++ b/ratis-test/src/test/java/org/apache/ratis/netty/TestLeaderElectionWithNetty.java @@ -19,7 +19,7 @@ import org.apache.ratis.server.impl.BlockRequestHandlingInjection; import org.apache.ratis.server.impl.LeaderElectionTests; -import org.junit.Test; +import org.junit.jupiter.api.Test; public class TestLeaderElectionWithNetty extends LeaderElectionTests From 023f83c057a779880baaae2f595d2d853a25e265 Mon Sep 17 00:00:00 2001 From: "Doroszlai, Attila" <6454655+adoroszlai@users.noreply.github.com> Date: Thu, 1 Feb 2024 19:11:13 +0100 Subject: [PATCH 002/397] RATIS-2023. Remove duplicate RefCountingMap (#1039) --- .../apache/ratis/util}/RefCountingMap.java | 20 +-- .../ratis/util}/TestRefCountingMap.java | 3 +- .../metrics/impl/MetricRegistriesImpl.java | 1 + .../dropwizard3/Dm3MetricRegistriesImpl.java | 1 + .../metrics/dropwizard3/RefCountingMap.java | 94 ----------- .../dropwizard3/TestRefCountingMap.java | 147 ------------------ 6 files changed, 14 insertions(+), 252 deletions(-) rename {ratis-metrics-default/src/main/java/org/apache/ratis/metrics/impl => ratis-common/src/main/java/org/apache/ratis/util}/RefCountingMap.java (88%) rename {ratis-metrics-default/src/test/java/org/apache/ratis/metrics/impl => ratis-common/src/test/java/org/apache/ratis/util}/TestRefCountingMap.java (98%) delete mode 100644 ratis-metrics-dropwizard3/src/main/java/org/apache/ratis/metrics/dropwizard3/RefCountingMap.java delete mode 100644 ratis-metrics-dropwizard3/src/test/java/org/apache/ratis/metrics/dropwizard3/TestRefCountingMap.java diff --git a/ratis-metrics-default/src/main/java/org/apache/ratis/metrics/impl/RefCountingMap.java b/ratis-common/src/main/java/org/apache/ratis/util/RefCountingMap.java similarity index 88% rename from ratis-metrics-default/src/main/java/org/apache/ratis/metrics/impl/RefCountingMap.java rename to ratis-common/src/main/java/org/apache/ratis/util/RefCountingMap.java index 49759781f8..a6a9eb81ab 100644 --- a/ratis-metrics-default/src/main/java/org/apache/ratis/metrics/impl/RefCountingMap.java +++ b/ratis-common/src/main/java/org/apache/ratis/util/RefCountingMap.java @@ -15,7 +15,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.ratis.metrics.impl; +package org.apache.ratis.util; import java.util.Collection; import java.util.Set; @@ -31,7 +31,7 @@ * call will increment the ref count, and each remove() will decrement it. The values are removed * from the map iff ref count == 0. */ -class RefCountingMap { +public final class RefCountingMap { private static class Payload { private final V value; private final AtomicInteger refCount = new AtomicInteger(); @@ -55,15 +55,15 @@ Payload decrement() { private final ConcurrentMap> map = new ConcurrentHashMap<>(); - V put(K k, Supplier supplier) { + public V put(K k, Supplier supplier) { return map.compute(k, (k1, old) -> old != null? old: new Payload<>(supplier.get())).increment(); } - static V get(Payload p) { + public static V get(Payload p) { return p == null ? null : p.get(); } - V get(K k) { + public V get(K k) { return get(map.get(k)); } @@ -72,23 +72,23 @@ V get(K k) { * @param k the key to remove * @return the value associated with the specified key or null if key is removed from map. */ - V remove(K k) { + public V remove(K k) { return get(map.computeIfPresent(k, (k1, v) -> v.decrement())); } - void clear() { + public void clear() { map.clear(); } - Set keySet() { + public Set keySet() { return map.keySet(); } - Collection values() { + public Collection values() { return map.values().stream().map(Payload::get).collect(Collectors.toList()); } - int size() { + public int size() { return map.size(); } } diff --git a/ratis-metrics-default/src/test/java/org/apache/ratis/metrics/impl/TestRefCountingMap.java b/ratis-common/src/test/java/org/apache/ratis/util/TestRefCountingMap.java similarity index 98% rename from ratis-metrics-default/src/test/java/org/apache/ratis/metrics/impl/TestRefCountingMap.java rename to ratis-common/src/test/java/org/apache/ratis/util/TestRefCountingMap.java index be6eb3037c..db6e9239ae 100644 --- a/ratis-metrics-default/src/test/java/org/apache/ratis/metrics/impl/TestRefCountingMap.java +++ b/ratis-common/src/test/java/org/apache/ratis/util/TestRefCountingMap.java @@ -15,7 +15,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.ratis.metrics.impl; +package org.apache.ratis.util; import static org.junit.jupiter.api.Assertions.assertEquals; import static org.junit.jupiter.api.Assertions.assertNotNull; @@ -26,6 +26,7 @@ import java.util.Set; import org.apache.ratis.thirdparty.com.google.common.collect.Lists; +import org.apache.ratis.util.RefCountingMap; import org.junit.jupiter.api.BeforeEach; import org.junit.jupiter.api.Test; diff --git a/ratis-metrics-default/src/main/java/org/apache/ratis/metrics/impl/MetricRegistriesImpl.java b/ratis-metrics-default/src/main/java/org/apache/ratis/metrics/impl/MetricRegistriesImpl.java index 17968ae9f8..088508fab9 100644 --- a/ratis-metrics-default/src/main/java/org/apache/ratis/metrics/impl/MetricRegistriesImpl.java +++ b/ratis-metrics-default/src/main/java/org/apache/ratis/metrics/impl/MetricRegistriesImpl.java @@ -28,6 +28,7 @@ import org.apache.ratis.metrics.MetricRegistries; import org.apache.ratis.metrics.MetricRegistryInfo; import org.apache.ratis.metrics.RatisMetricRegistry; +import org.apache.ratis.util.RefCountingMap; import org.apache.ratis.util.TimeDuration; import org.slf4j.Logger; import org.slf4j.LoggerFactory; diff --git a/ratis-metrics-dropwizard3/src/main/java/org/apache/ratis/metrics/dropwizard3/Dm3MetricRegistriesImpl.java b/ratis-metrics-dropwizard3/src/main/java/org/apache/ratis/metrics/dropwizard3/Dm3MetricRegistriesImpl.java index b26f2e27a4..a90c5a0cee 100644 --- a/ratis-metrics-dropwizard3/src/main/java/org/apache/ratis/metrics/dropwizard3/Dm3MetricRegistriesImpl.java +++ b/ratis-metrics-dropwizard3/src/main/java/org/apache/ratis/metrics/dropwizard3/Dm3MetricRegistriesImpl.java @@ -28,6 +28,7 @@ import org.apache.ratis.metrics.MetricRegistries; import org.apache.ratis.metrics.MetricRegistryInfo; import org.apache.ratis.metrics.RatisMetricRegistry; +import org.apache.ratis.util.RefCountingMap; import org.apache.ratis.util.TimeDuration; import org.slf4j.Logger; import org.slf4j.LoggerFactory; diff --git a/ratis-metrics-dropwizard3/src/main/java/org/apache/ratis/metrics/dropwizard3/RefCountingMap.java b/ratis-metrics-dropwizard3/src/main/java/org/apache/ratis/metrics/dropwizard3/RefCountingMap.java deleted file mode 100644 index 6c3ad6f8cd..0000000000 --- a/ratis-metrics-dropwizard3/src/main/java/org/apache/ratis/metrics/dropwizard3/RefCountingMap.java +++ /dev/null @@ -1,94 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ -package org.apache.ratis.metrics.dropwizard3; - -import java.util.Collection; -import java.util.Set; -import java.util.concurrent.ConcurrentHashMap; -import java.util.concurrent.ConcurrentMap; -import java.util.concurrent.atomic.AtomicInteger; -import java.util.function.Supplier; -import java.util.stream.Collectors; - -/** - * A map of K to V, but does ref counting for added and removed values. The values are - * not added directly, but instead requested from the given Supplier if ref count == 0. Each put() - * call will increment the ref count, and each remove() will decrement it. The values are removed - * from the map iff ref count == 0. - */ -class RefCountingMap { - private static class Payload { - private final V value; - private final AtomicInteger refCount = new AtomicInteger(); - - Payload(V v) { - this.value = v; - } - - V get() { - return value; - } - - V increment() { - return refCount.incrementAndGet() > 0? value: null; - } - - Payload decrement() { - return refCount.decrementAndGet() > 0? this: null; - } - } - - private final ConcurrentMap> map = new ConcurrentHashMap<>(); - - V put(K k, Supplier supplier) { - return map.compute(k, (k1, old) -> old != null? old: new Payload<>(supplier.get())).increment(); - } - - static V get(Payload p) { - return p == null ? null : p.get(); - } - - V get(K k) { - return get(map.get(k)); - } - - /** - * Decrements the ref count of k, and removes from map if ref count == 0. - * @param k the key to remove - * @return the value associated with the specified key or null if key is removed from map. - */ - V remove(K k) { - return get(map.computeIfPresent(k, (k1, v) -> v.decrement())); - } - - void clear() { - map.clear(); - } - - Set keySet() { - return map.keySet(); - } - - Collection values() { - return map.values().stream().map(Payload::get).collect(Collectors.toList()); - } - - int size() { - return map.size(); - } -} diff --git a/ratis-metrics-dropwizard3/src/test/java/org/apache/ratis/metrics/dropwizard3/TestRefCountingMap.java b/ratis-metrics-dropwizard3/src/test/java/org/apache/ratis/metrics/dropwizard3/TestRefCountingMap.java deleted file mode 100644 index 87b8bf0123..0000000000 --- a/ratis-metrics-dropwizard3/src/test/java/org/apache/ratis/metrics/dropwizard3/TestRefCountingMap.java +++ /dev/null @@ -1,147 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ -package org.apache.ratis.metrics.dropwizard3; - -import static org.junit.jupiter.api.Assertions.assertEquals; -import static org.junit.jupiter.api.Assertions.assertNotNull; -import static org.junit.jupiter.api.Assertions.assertNull; -import static org.junit.jupiter.api.Assertions.assertTrue; - -import java.util.Collection; -import java.util.Set; - -import org.apache.ratis.thirdparty.com.google.common.collect.Lists; -import org.junit.jupiter.api.BeforeEach; -import org.junit.jupiter.api.Test; - -public class TestRefCountingMap { - - private RefCountingMap map; - - @BeforeEach - public void setUp() { - map = new RefCountingMap<>(); - } - - @Test - public void testPutGet() { - map.put("foo", () -> "foovalue"); - - String v = map.get("foo"); - assertNotNull(v); - assertEquals("foovalue", v); - } - - @Test - public void testPutMulti() { - String v1 = map.put("foo", () -> "foovalue"); - String v2 = map.put("foo", () -> "foovalue2"); - String v3 = map.put("foo", () -> "foovalue3"); - - String v = map.get("foo"); - assertEquals("foovalue", v); - assertEquals(v, v1); - assertEquals(v, v2); - assertEquals(v, v3); - } - - @Test - public void testPutRemove() { - map.put("foo", () -> "foovalue"); - String v = map.remove("foo"); - assertNull(v); - v = map.get("foo"); - assertNull(v); - } - - @Test - public void testPutRemoveMulti() { - map.put("foo", () -> "foovalue"); - map.put("foo", () -> "foovalue2"); - map.put("foo", () -> "foovalue3"); - - // remove 1 - String v = map.remove("foo"); - assertEquals("foovalue", v); - - // remove 2 - v = map.remove("foo"); - assertEquals("foovalue", v); - - // remove 3 - v = map.remove("foo"); - assertNull(v); - v = map.get("foo"); - assertNull(v); - } - - @Test - public void testSize() { - assertEquals(0, map.size()); - - // put a key - map.put("foo", () -> "foovalue"); - assertEquals(1, map.size()); - - // put a different key - map.put("bar", () -> "foovalue2"); - assertEquals(2, map.size()); - - // put the same key again - map.put("bar", () -> "foovalue3"); - assertEquals(2, map.size()); // map should be same size - } - - @Test - public void testClear() { - map.put("foo", () -> "foovalue"); - map.put("bar", () -> "foovalue2"); - map.put("baz", () -> "foovalue3"); - - map.clear(); - - assertEquals(0, map.size()); - } - - - @Test - public void testKeySet() { - map.put("foo", () -> "foovalue"); - map.put("bar", () -> "foovalue2"); - map.put("baz", () -> "foovalue3"); - - Set keys = map.keySet(); - assertEquals(3, keys.size()); - - Lists.newArrayList("foo", "bar", "baz").forEach(v -> assertTrue(keys.contains(v))); - } - - @Test - public void testValues() { - map.put("foo", () -> "foovalue"); - map.put("foo", () -> "foovalue2"); - map.put("bar", () -> "foovalue3"); - map.put("baz", () -> "foovalue4"); - - Collection values = map.values(); - assertEquals(3, values.size()); - - Lists.newArrayList("foovalue", "foovalue3", "foovalue4") - .forEach(v -> assertTrue(values.contains(v))); - } -} From ba6e9033ef7c64fd26558c7b91e412f76e4ee277 Mon Sep 17 00:00:00 2001 From: Tsz-Wo Nicholas Sze Date: Tue, 6 Feb 2024 08:22:52 -0800 Subject: [PATCH 003/397] RATIS-2024. Refactor appendEntries code. (#1040) --- .../ratis/server/impl/RaftServerImpl.java | 167 +++++++----------- .../apache/ratis/server/impl/ServerState.java | 23 ++- .../impl/SnapshotInstallationHandler.java | 29 ++- .../ratis/server/util/ServerStringUtils.java | 8 +- 4 files changed, 93 insertions(+), 134 deletions(-) diff --git a/ratis-server/src/main/java/org/apache/ratis/server/impl/RaftServerImpl.java b/ratis-server/src/main/java/org/apache/ratis/server/impl/RaftServerImpl.java index 88f4f76406..1b59d21ad5 100644 --- a/ratis-server/src/main/java/org/apache/ratis/server/impl/RaftServerImpl.java +++ b/ratis-server/src/main/java/org/apache/ratis/server/impl/RaftServerImpl.java @@ -108,6 +108,11 @@ import org.apache.ratis.util.Timestamp; import org.apache.ratis.util.function.CheckedSupplier; +import static org.apache.ratis.server.impl.ServerImplUtils.effectiveCommitIndex; +import static org.apache.ratis.server.impl.ServerProtoUtils.toAppendEntriesReplyProto; +import static org.apache.ratis.server.util.ServerStringUtils.toAppendEntriesReplyString; +import static org.apache.ratis.server.util.ServerStringUtils.toAppendEntriesRequestString; + class RaftServerImpl implements RaftServer.Division, RaftServerProtocol, RaftServerAsynchronousProtocol, RaftClientProtocol, RaftClientAsynchronousProtocol { @@ -1471,18 +1476,24 @@ public AppendEntriesReplyProto appendEntries(AppendEntriesRequestProto r) public CompletableFuture appendEntriesAsync(AppendEntriesRequestProto r) throws IOException { final RaftRpcRequestProto request = r.getServerRequest(); - final List entries = r.getEntriesList(); final TermIndex previous = r.hasPreviousLog()? TermIndex.valueOf(r.getPreviousLog()) : null; - final RaftPeerId requestorId = RaftPeerId.valueOf(request.getRequestorId()); - - preAppendEntriesAsync(requestorId, ProtoUtils.toRaftGroupId(request.getRaftGroupId()), r.getLeaderTerm(), - previous, r.getLeaderCommit(), r.getInitializing(), entries); try { - return appendEntriesAsync(requestorId, r.getLeaderTerm(), previous, r.getLeaderCommit(), - request.getCallId(), r.getInitializing(), r.getCommitInfosList(), entries); + final RaftPeerId leaderId = RaftPeerId.valueOf(request.getRequestorId()); + final RaftGroupId leaderGroupId = ProtoUtils.toRaftGroupId(request.getRaftGroupId()); + + CodeInjectionForTesting.execute(APPEND_ENTRIES, getId(), leaderId, previous, r); + + assertLifeCycleState(LifeCycle.States.STARTING_OR_RUNNING); + if (!startComplete.get()) { + throw new ServerNotReadyException(getMemberId() + ": The server role is not yet initialized."); + } + assertGroup(leaderId, leaderGroupId); + validateEntries(r.getLeaderTerm(), previous, r.getEntriesList()); + + return appendEntriesAsync(leaderId, request.getCallId(), previous, r); } catch(Exception t) { - LOG.error("{}: Failed appendEntriesAsync {}", getMemberId(), r, t); - throw t; + LOG.error("{}: Failed appendEntries* {}", getMemberId(), toAppendEntriesRequestString(r), t); + throw IOUtils.asIOException(t); } } @@ -1526,24 +1537,6 @@ Optional updateLastRpcTime(FollowerState.UpdateType updateType) { } } - private void preAppendEntriesAsync(RaftPeerId leaderId, RaftGroupId leaderGroupId, long leaderTerm, - TermIndex previous, long leaderCommit, boolean initializing, List entries) throws IOException { - CodeInjectionForTesting.execute(APPEND_ENTRIES, getId(), - leaderId, leaderTerm, previous, leaderCommit, initializing, entries); - - assertLifeCycleState(LifeCycle.States.STARTING_OR_RUNNING); - if (!startComplete.get()) { - throw new ServerNotReadyException(getMemberId() + ": The server role is not yet initialized."); - } - assertGroup(leaderId, leaderGroupId); - - try { - validateEntries(leaderTerm, previous, entries); - } catch (IllegalArgumentException e) { - throw new IOException(e); - } - } - private long updateCommitInfoCache() { return commitInfoCache.update(getId(), state.getLog().getLastCommittedIndex()); } @@ -1552,18 +1545,14 @@ ExecutorService getServerExecutor() { return serverExecutor; } - @SuppressWarnings("checkstyle:parameternumber") - private CompletableFuture appendEntriesAsync( - RaftPeerId leaderId, long leaderTerm, TermIndex previous, long leaderCommit, long callId, boolean initializing, - List commitInfos, List entries) throws IOException { + private CompletableFuture appendEntriesAsync(RaftPeerId leaderId, long callId, + TermIndex previous, AppendEntriesRequestProto proto) throws IOException { + final List entries = proto.getEntriesList(); final boolean isHeartbeat = entries.isEmpty(); - logAppendEntries(isHeartbeat, - () -> getMemberId() + ": receive appendEntries(" + leaderId + ", " + leaderTerm + ", " - + previous + ", " + leaderCommit + ", " + initializing - + ", commits:" + ProtoUtils.toString(commitInfos) - + ", cId:" + callId - + ", entries: " + LogProtoUtils.toLogEntriesString(entries)); + logAppendEntries(isHeartbeat, () -> getMemberId() + ": appendEntries* " + + toAppendEntriesRequestString(proto)); + final long leaderTerm = proto.getLeaderTerm(); final long currentTerm; final long followerCommit = state.getLog().getLastCommittedIndex(); final Optional followerState; @@ -1571,17 +1560,12 @@ private CompletableFuture appendEntriesAsync( synchronized (this) { // Check life cycle state again to avoid the PAUSING/PAUSED state. assertLifeCycleState(LifeCycle.States.STARTING_OR_RUNNING); - final boolean recognized = state.recognizeLeader(leaderId, leaderTerm); currentTerm = state.getCurrentTerm(); + final boolean recognized = state.recognizeLeader(Op.APPEND_ENTRIES, leaderId, leaderTerm); if (!recognized) { - final AppendEntriesReplyProto reply = ServerProtoUtils.toAppendEntriesReplyProto( + return CompletableFuture.completedFuture(toAppendEntriesReplyProto( leaderId, getMemberId(), currentTerm, followerCommit, state.getNextIndex(), - AppendResult.NOT_LEADER, callId, RaftLog.INVALID_LOG_INDEX, isHeartbeat); - if (LOG.isDebugEnabled()) { - LOG.debug("{}: Not recognize {} (term={}) as leader, state: {} reply: {}", - getMemberId(), leaderId, leaderTerm, state, ServerStringUtils.toAppendEntriesReplyString(reply)); - } - return CompletableFuture.completedFuture(reply); + AppendResult.NOT_LEADER, callId, RaftLog.INVALID_LOG_INDEX, isHeartbeat)); } try { changeToFollowerAndPersistMetadata(leaderTerm, true, "appendEntries"); @@ -1590,7 +1574,7 @@ leaderId, getMemberId(), currentTerm, followerCommit, state.getNextIndex(), } state.setLeader(leaderId, "appendEntries"); - if (!initializing && lifeCycle.compareAndTransition(State.STARTING, State.RUNNING)) { + if (!proto.getInitializing() && lifeCycle.compareAndTransition(State.STARTING, State.RUNNING)) { role.startFollowerState(this, Op.APPEND_ENTRIES); } followerState = updateLastRpcTime(FollowerState.UpdateType.APPEND_START); @@ -1602,12 +1586,14 @@ leaderId, getMemberId(), currentTerm, followerCommit, state.getNextIndex(), // 3. There is a gap between the local log and the entries // In any of these scenarios, we should return an INCONSISTENCY reply // back to leader so that the leader can update this follower's next index. - - AppendEntriesReplyProto inconsistencyReply = checkInconsistentAppendEntries( - leaderId, currentTerm, followerCommit, previous, callId, isHeartbeat, entries); - if (inconsistencyReply != null) { + final long inconsistencyReplyNextIndex = checkInconsistentAppendEntries(previous, entries); + if (inconsistencyReplyNextIndex > RaftLog.INVALID_LOG_INDEX) { + final AppendEntriesReplyProto reply = toAppendEntriesReplyProto( + leaderId, getMemberId(), currentTerm, followerCommit, inconsistencyReplyNextIndex, + AppendResult.INCONSISTENCY, callId, RaftLog.INVALID_LOG_INDEX, isHeartbeat); + LOG.info("{}: appendEntries* reply {}", getMemberId(), toAppendEntriesReplyString(reply)); followerState.ifPresent(fs -> fs.updateLastRpcTime(FollowerState.UpdateType.APPEND_COMPLETE)); - return CompletableFuture.completedFuture(inconsistencyReply); + return CompletableFuture.completedFuture(reply); } state.updateConfiguration(entries); @@ -1615,7 +1601,7 @@ leaderId, getMemberId(), currentTerm, followerCommit, state.getNextIndex(), final List> futures = entries.isEmpty() ? Collections.emptyList() : state.getLog().append(entries); - commitInfos.forEach(commitInfoCache::update); + proto.getCommitInfosList().forEach(commitInfoCache::update); CodeInjectionForTesting.execute(LOG_SYNC, getId(), null); if (!isHeartbeat) { @@ -1625,49 +1611,27 @@ leaderId, getMemberId(), currentTerm, followerCommit, state.getNextIndex(), stateMachine.event().notifySnapshotInstalled(InstallSnapshotResult.SUCCESS, installedIndex, getPeer()); } } - return JavaUtils.allOf(futures).whenCompleteAsync( - (r, t) -> followerState.ifPresent(fs -> fs.updateLastRpcTime(FollowerState.UpdateType.APPEND_COMPLETE)), - serverExecutor - ).thenApply(v -> { - final AppendEntriesReplyProto reply; - synchronized(this) { - final long commitIndex = ServerImplUtils.effectiveCommitIndex(leaderCommit, previous, entries.size()); - state.updateCommitIndex(commitIndex, currentTerm, false); + + final long commitIndex = effectiveCommitIndex(proto.getLeaderCommit(), previous, entries.size()); + final long matchIndex = isHeartbeat? RaftLog.INVALID_LOG_INDEX: entries.get(entries.size() - 1).getIndex(); + return JavaUtils.allOf(futures).whenCompleteAsync((r, t) -> { + followerState.ifPresent(fs -> fs.updateLastRpcTime(FollowerState.UpdateType.APPEND_COMPLETE)); + timer.stop(); + }, getServerExecutor()).thenApply(v -> { + final boolean updated = state.updateCommitIndex(commitIndex, currentTerm, false); + if (updated) { updateCommitInfoCache(); - final long n; - final long matchIndex; - if (!isHeartbeat) { - LogEntryProto requestLastEntry = entries.get(entries.size() - 1); - n = requestLastEntry.getIndex() + 1; - matchIndex = requestLastEntry.getIndex(); - } else { - n = state.getLog().getNextIndex(); - matchIndex = RaftLog.INVALID_LOG_INDEX; - } - reply = ServerProtoUtils.toAppendEntriesReplyProto(leaderId, getMemberId(), currentTerm, - state.getLog().getLastCommittedIndex(), n, AppendResult.SUCCESS, callId, matchIndex, isHeartbeat); } - logAppendEntries(isHeartbeat, () -> getMemberId() + ": succeeded to handle AppendEntries. Reply: " - + ServerStringUtils.toAppendEntriesReplyString(reply)); - timer.stop(); // TODO: future never completes exceptionally? + final long nextIndex = isHeartbeat? state.getNextIndex(): matchIndex + 1; + final AppendEntriesReplyProto reply = toAppendEntriesReplyProto(leaderId, getMemberId(), + currentTerm, updated? commitIndex : state.getLog().getLastCommittedIndex(), + nextIndex, AppendResult.SUCCESS, callId, matchIndex, isHeartbeat); + logAppendEntries(isHeartbeat, () -> getMemberId() + + ": appendEntries* reply " + toAppendEntriesReplyString(reply)); return reply; }); } - private AppendEntriesReplyProto checkInconsistentAppendEntries(RaftPeerId leaderId, long currentTerm, - long followerCommit, TermIndex previous, long callId, boolean isHeartbeat, List entries) { - final long replyNextIndex = checkInconsistentAppendEntries(previous, entries); - if (replyNextIndex == -1) { - return null; - } - - final AppendEntriesReplyProto reply = ServerProtoUtils.toAppendEntriesReplyProto( - leaderId, getMemberId(), currentTerm, followerCommit, replyNextIndex, - AppendResult.INCONSISTENCY, callId, RaftLog.INVALID_LOG_INDEX, isHeartbeat); - LOG.info("{}: inconsistency entries. Reply:{}", getMemberId(), ServerStringUtils.toAppendEntriesReplyString(reply)); - return reply; - } - private long checkInconsistentAppendEntries(TermIndex previous, List entries) { // Check if a snapshot installation through state machine is in progress. final long installSnapshot = snapshotInstallationHandler.getInProgressInstallSnapshotIndex(); @@ -1698,7 +1662,7 @@ private long checkInconsistentAppendEntries(TermIndex previous, List current || curLeaderId == null) { - // If the request indicates a term that is greater than the current term - // or no leader has been set for the current term, make sure to update - // leader and term later - return true; + if (peerTerm == current && curLeaderId != null && !curLeaderId.equals(peerId)) { + LOG.warn("{}: Failed to recognize {} as leader for {} since current leader is {} (peerTerm = currentTerm = {})", + getMemberId(), peerId, op, curLeaderId, current); + return false; } - return curLeaderId.equals(peerLeaderId); + return true; } static int compareLog(TermIndex lastEntry, TermIndex candidateLastEntry) { diff --git a/ratis-server/src/main/java/org/apache/ratis/server/impl/SnapshotInstallationHandler.java b/ratis-server/src/main/java/org/apache/ratis/server/impl/SnapshotInstallationHandler.java index 9794314b83..7aae944a43 100644 --- a/ratis-server/src/main/java/org/apache/ratis/server/impl/SnapshotInstallationHandler.java +++ b/ratis-server/src/main/java/org/apache/ratis/server/impl/SnapshotInstallationHandler.java @@ -32,6 +32,7 @@ import org.apache.ratis.protocol.RaftPeer; import org.apache.ratis.protocol.RaftPeerId; import org.apache.ratis.server.RaftServerConfigKeys; +import org.apache.ratis.server.protocol.RaftServerProtocol; import org.apache.ratis.server.protocol.TermIndex; import org.apache.ratis.server.raftlog.LogProtoUtils; import org.apache.ratis.server.util.ServerStringUtils; @@ -49,6 +50,8 @@ import java.util.concurrent.atomic.AtomicLong; import java.util.concurrent.atomic.AtomicReference; +import static org.apache.ratis.server.impl.ServerProtoUtils.toInstallSnapshotReplyProto; +import static org.apache.ratis.server.impl.ServerProtoUtils.toServerRpcProto; import static org.apache.ratis.server.raftlog.RaftLog.INVALID_LOG_INDEX; class SnapshotInstallationHandler { @@ -142,7 +145,7 @@ private InstallSnapshotReplyProto installSnapshotImpl(InstallSnapshotRequestProt } // There is a mismatch between configurations on leader and follower. - final InstallSnapshotReplyProto failedReply = ServerProtoUtils.toInstallSnapshotReplyProto( + final InstallSnapshotReplyProto failedReply = toInstallSnapshotReplyProto( leaderId, getMemberId(), state.getCurrentTerm(), InstallSnapshotResult.CONF_MISMATCH); LOG.error("{}: Configuration Mismatch ({}): Leader {} has it set to {} but follower {} has it set to {}", getMemberId(), RaftServerConfigKeys.Log.Appender.INSTALL_SNAPSHOT_ENABLED_KEY, @@ -158,13 +161,11 @@ private InstallSnapshotReplyProto checkAndInstallSnapshot(InstallSnapshotRequest final TermIndex lastIncluded = TermIndex.valueOf(snapshotChunkRequest.getTermIndex()); final long lastIncludedIndex = lastIncluded.getIndex(); synchronized (server) { - final boolean recognized = state.recognizeLeader(leaderId, leaderTerm); + final boolean recognized = state.recognizeLeader(RaftServerProtocol.Op.INSTALL_SNAPSHOT, leaderId, leaderTerm); currentTerm = state.getCurrentTerm(); if (!recognized) { - final InstallSnapshotReplyProto reply = ServerProtoUtils.toInstallSnapshotReplyProto(leaderId, getMemberId(), + return toInstallSnapshotReplyProto(leaderId, getMemberId(), currentTerm, snapshotChunkRequest.getRequestIndex(), InstallSnapshotResult.NOT_LEADER); - LOG.warn("{}: Failed to recognize leader for installSnapshot chunk.", getMemberId()); - return reply; } server.changeToFollowerAndPersistMetadata(leaderTerm, true, "installSnapshot"); state.setLeader(leaderId, "installSnapshot"); @@ -193,7 +194,7 @@ private InstallSnapshotReplyProto checkAndInstallSnapshot(InstallSnapshotRequest if (snapshotChunkRequest.getDone()) { LOG.info("{}: successfully install the entire snapshot-{}", getMemberId(), lastIncludedIndex); } - return ServerProtoUtils.toInstallSnapshotReplyProto(leaderId, getMemberId(), + return toInstallSnapshotReplyProto(leaderId, getMemberId(), currentTerm, snapshotChunkRequest.getRequestIndex(), InstallSnapshotResult.SUCCESS); } @@ -205,13 +206,11 @@ private InstallSnapshotReplyProto notifyStateMachineToInstallSnapshot( request.getNotification().getFirstAvailableTermIndex()); final long firstAvailableLogIndex = firstAvailableLogTermIndex.getIndex(); synchronized (server) { - final boolean recognized = state.recognizeLeader(leaderId, leaderTerm); + final boolean recognized = state.recognizeLeader("notifyInstallSnapshot", leaderId, leaderTerm); currentTerm = state.getCurrentTerm(); if (!recognized) { - final InstallSnapshotReplyProto reply = ServerProtoUtils.toInstallSnapshotReplyProto(leaderId, getMemberId(), + return toInstallSnapshotReplyProto(leaderId, getMemberId(), currentTerm, InstallSnapshotResult.NOT_LEADER); - LOG.warn("{}: Failed to recognize leader for installSnapshot notification.", getMemberId()); - return reply; } server.changeToFollowerAndPersistMetadata(leaderTerm, true, "installSnapshot"); state.setLeader(leaderId, "installSnapshot"); @@ -229,7 +228,7 @@ private InstallSnapshotReplyProto notifyStateMachineToInstallSnapshot( inProgressInstallSnapshotIndex.compareAndSet(firstAvailableLogIndex, INVALID_LOG_INDEX); LOG.info("{}: InstallSnapshot notification result: {}, current snapshot index: {}", getMemberId(), InstallSnapshotResult.ALREADY_INSTALLED, snapshotIndex); - return ServerProtoUtils.toInstallSnapshotReplyProto(leaderId, getMemberId(), currentTerm, + return toInstallSnapshotReplyProto(leaderId, getMemberId(), currentTerm, InstallSnapshotResult.ALREADY_INSTALLED, snapshotIndex); } @@ -307,7 +306,7 @@ private InstallSnapshotReplyProto notifyStateMachineToInstallSnapshot( inProgressInstallSnapshotIndex.set(INVALID_LOG_INDEX); server.getStateMachine().event().notifySnapshotInstalled( InstallSnapshotResult.SNAPSHOT_UNAVAILABLE, INVALID_LOG_INDEX, server.getPeer()); - return ServerProtoUtils.toInstallSnapshotReplyProto(leaderId, getMemberId(), + return toInstallSnapshotReplyProto(leaderId, getMemberId(), currentTerm, InstallSnapshotResult.SNAPSHOT_UNAVAILABLE); } @@ -325,7 +324,7 @@ private InstallSnapshotReplyProto notifyStateMachineToInstallSnapshot( server.getStateMachine().event().notifySnapshotInstalled( InstallSnapshotResult.SNAPSHOT_INSTALLED, latestInstalledIndex, server.getPeer()); installedIndex.set(latestInstalledIndex); - return ServerProtoUtils.toInstallSnapshotReplyProto(leaderId, getMemberId(), + return toInstallSnapshotReplyProto(leaderId, getMemberId(), currentTerm, InstallSnapshotResult.SNAPSHOT_INSTALLED, latestInstalledSnapshotTermIndex.getIndex()); } @@ -334,7 +333,7 @@ private InstallSnapshotReplyProto notifyStateMachineToInstallSnapshot( LOG.debug("{}: InstallSnapshot notification result: {}", getMemberId(), InstallSnapshotResult.IN_PROGRESS); } - return ServerProtoUtils.toInstallSnapshotReplyProto(leaderId, getMemberId(), + return toInstallSnapshotReplyProto(leaderId, getMemberId(), currentTerm, InstallSnapshotResult.IN_PROGRESS); } } @@ -342,7 +341,7 @@ private InstallSnapshotReplyProto notifyStateMachineToInstallSnapshot( private RoleInfoProto getRoleInfoProto(RaftPeer leader) { final RoleInfo role = server.getRole(); final Optional fs = role.getFollowerState(); - final ServerRpcProto leaderInfo = ServerProtoUtils.toServerRpcProto(leader, + final ServerRpcProto leaderInfo = toServerRpcProto(leader, fs.map(FollowerState::getLastRpcTime).map(Timestamp::elapsedTimeMs).orElse(0L)); final FollowerInfoProto.Builder followerInfo = FollowerInfoProto.newBuilder() .setLeaderInfo(leaderInfo) diff --git a/ratis-server/src/main/java/org/apache/ratis/server/util/ServerStringUtils.java b/ratis-server/src/main/java/org/apache/ratis/server/util/ServerStringUtils.java index 25223c0f4d..284664d012 100644 --- a/ratis-server/src/main/java/org/apache/ratis/server/util/ServerStringUtils.java +++ b/ratis-server/src/main/java/org/apache/ratis/server/util/ServerStringUtils.java @@ -21,16 +21,19 @@ import org.apache.ratis.proto.RaftProtos.AppendEntriesRequestProto; import org.apache.ratis.proto.RaftProtos.InstallSnapshotReplyProto; import org.apache.ratis.proto.RaftProtos.InstallSnapshotRequestProto; +import org.apache.ratis.proto.RaftProtos.LogEntryProto; import org.apache.ratis.proto.RaftProtos.RequestVoteReplyProto; import org.apache.ratis.server.protocol.TermIndex; import org.apache.ratis.server.raftlog.LogProtoUtils; import org.apache.ratis.util.ProtoUtils; +import java.util.List; + /** * This class provides convenient utilities for converting Protocol Buffers messages to strings. * The output strings are for information purpose only. * They are concise and compact compared to the Protocol Buffers implementations of {@link Object#toString()}. - * + *

* The output messages or the output formats may be changed without notice. * Callers of this class should not try to parse the output strings for any purposes. * Instead, they should use the public APIs provided by Protocol Buffers. @@ -42,12 +45,13 @@ public static String toAppendEntriesRequestString(AppendEntriesRequestProto requ if (request == null) { return null; } + final List entries = request.getEntriesList(); return ProtoUtils.toString(request.getServerRequest()) + "-t" + request.getLeaderTerm() + ",previous=" + TermIndex.valueOf(request.getPreviousLog()) + ",leaderCommit=" + request.getLeaderCommit() + ",initializing? " + request.getInitializing() - + ",entries: " + LogProtoUtils.toLogEntriesShortString(request.getEntriesList()); + + "," + (entries.isEmpty()? "HEARTBEAT" : "entries: " + LogProtoUtils.toLogEntriesShortString(entries)); } public static String toAppendEntriesReplyString(AppendEntriesReplyProto reply) { From 548d7dc3f5b560091c74cc752f1c6973d7d29b1a Mon Sep 17 00:00:00 2001 From: Tsz-Wo Nicholas Sze Date: Wed, 7 Feb 2024 09:20:06 -0800 Subject: [PATCH 004/397] RATIS-2025. Move out assert and proto methods from RaftServerImpl. (#1041) --- .../ratis/server/impl/RaftServerImpl.java | 233 +++++++----------- .../apache/ratis/server/impl/RoleInfo.java | 57 +++++ .../ratis/server/impl/ServerImplUtils.java | 55 ++++- .../ratis/server/impl/ServerProtoUtils.java | 4 + .../impl/SnapshotInstallationHandler.java | 2 +- 5 files changed, 205 insertions(+), 146 deletions(-) diff --git a/ratis-server/src/main/java/org/apache/ratis/server/impl/RaftServerImpl.java b/ratis-server/src/main/java/org/apache/ratis/server/impl/RaftServerImpl.java index 1b59d21ad5..08ccabac6d 100644 --- a/ratis-server/src/main/java/org/apache/ratis/server/impl/RaftServerImpl.java +++ b/ratis-server/src/main/java/org/apache/ratis/server/impl/RaftServerImpl.java @@ -17,36 +17,30 @@ */ package org.apache.ratis.server.impl; -import java.io.File; -import java.io.IOException; -import java.nio.file.NoSuchFileException; -import java.util.ArrayList; -import java.util.Collection; -import java.util.Collections; -import java.util.Comparator; -import java.util.List; -import java.util.Map; -import java.util.Objects; -import java.util.Optional; -import java.util.Set; -import java.util.concurrent.CompletableFuture; -import java.util.concurrent.CompletionException; -import java.util.concurrent.ExecutionException; -import java.util.concurrent.ExecutorService; -import java.util.concurrent.ThreadLocalRandom; -import java.util.concurrent.TimeUnit; -import java.util.concurrent.atomic.AtomicBoolean; -import java.util.function.Function; -import java.util.function.Supplier; -import java.util.stream.Collectors; -import java.util.stream.Stream; - import org.apache.ratis.client.impl.ClientProtoUtils; import org.apache.ratis.conf.RaftProperties; import org.apache.ratis.metrics.Timekeeper; -import org.apache.ratis.proto.RaftProtos.*; +import org.apache.ratis.proto.RaftProtos.AppendEntriesReplyProto; import org.apache.ratis.proto.RaftProtos.AppendEntriesReplyProto.AppendResult; +import org.apache.ratis.proto.RaftProtos.AppendEntriesRequestProto; +import org.apache.ratis.proto.RaftProtos.CommitInfoProto; +import org.apache.ratis.proto.RaftProtos.InstallSnapshotReplyProto; +import org.apache.ratis.proto.RaftProtos.InstallSnapshotRequestProto; +import org.apache.ratis.proto.RaftProtos.InstallSnapshotResult; +import org.apache.ratis.proto.RaftProtos.LogEntryProto; +import org.apache.ratis.proto.RaftProtos.RaftClientRequestProto; import org.apache.ratis.proto.RaftProtos.RaftClientRequestProto.TypeCase; +import org.apache.ratis.proto.RaftProtos.RaftConfigurationProto; +import org.apache.ratis.proto.RaftProtos.RaftPeerRole; +import org.apache.ratis.proto.RaftProtos.RaftRpcRequestProto; +import org.apache.ratis.proto.RaftProtos.ReadIndexReplyProto; +import org.apache.ratis.proto.RaftProtos.ReadIndexRequestProto; +import org.apache.ratis.proto.RaftProtos.ReplicationLevel; +import org.apache.ratis.proto.RaftProtos.RequestVoteReplyProto; +import org.apache.ratis.proto.RaftProtos.RequestVoteRequestProto; +import org.apache.ratis.proto.RaftProtos.RoleInfoProto; +import org.apache.ratis.proto.RaftProtos.StartLeaderElectionReplyProto; +import org.apache.ratis.proto.RaftProtos.StartLeaderElectionRequestProto; import org.apache.ratis.protocol.ClientInvocationId; import org.apache.ratis.protocol.GroupInfoReply; import org.apache.ratis.protocol.GroupInfoRequest; @@ -64,7 +58,20 @@ import org.apache.ratis.protocol.SetConfigurationRequest; import org.apache.ratis.protocol.SnapshotManagementRequest; import org.apache.ratis.protocol.TransferLeadershipRequest; -import org.apache.ratis.protocol.exceptions.*; +import org.apache.ratis.protocol.exceptions.GroupMismatchException; +import org.apache.ratis.protocol.exceptions.LeaderNotReadyException; +import org.apache.ratis.protocol.exceptions.LeaderSteppingDownException; +import org.apache.ratis.protocol.exceptions.NotLeaderException; +import org.apache.ratis.protocol.exceptions.RaftException; +import org.apache.ratis.protocol.exceptions.ReadException; +import org.apache.ratis.protocol.exceptions.ReadIndexException; +import org.apache.ratis.protocol.exceptions.ReconfigurationInProgressException; +import org.apache.ratis.protocol.exceptions.ResourceUnavailableException; +import org.apache.ratis.protocol.exceptions.ServerNotReadyException; +import org.apache.ratis.protocol.exceptions.SetConfigurationException; +import org.apache.ratis.protocol.exceptions.StaleReadException; +import org.apache.ratis.protocol.exceptions.StateMachineException; +import org.apache.ratis.protocol.exceptions.TransferLeadershipException; import org.apache.ratis.server.DataStreamMap; import org.apache.ratis.server.DivisionInfo; import org.apache.ratis.server.DivisionProperties; @@ -74,7 +81,6 @@ import org.apache.ratis.server.impl.LeaderElection.Phase; import org.apache.ratis.server.impl.RetryCacheImpl.CacheEntry; import org.apache.ratis.server.leader.LeaderState; -import org.apache.ratis.server.leader.LogAppender; import org.apache.ratis.server.metrics.LeaderElectionMetrics; import org.apache.ratis.server.metrics.RaftServerMetricsImpl; import org.apache.ratis.server.protocol.RaftServerAsynchronousProtocol; @@ -85,7 +91,6 @@ import org.apache.ratis.server.raftlog.RaftLogIOException; import org.apache.ratis.server.storage.RaftStorage; import org.apache.ratis.server.storage.RaftStorageDirectory; -import org.apache.ratis.server.util.ServerStringUtils; import org.apache.ratis.statemachine.SnapshotInfo; import org.apache.ratis.statemachine.StateMachine; import org.apache.ratis.statemachine.TransactionContext; @@ -105,13 +110,43 @@ import org.apache.ratis.util.ProtoUtils; import org.apache.ratis.util.ReferenceCountedObject; import org.apache.ratis.util.TimeDuration; -import org.apache.ratis.util.Timestamp; import org.apache.ratis.util.function.CheckedSupplier; +import java.io.File; +import java.io.IOException; +import java.nio.file.NoSuchFileException; +import java.util.ArrayList; +import java.util.Collection; +import java.util.Collections; +import java.util.Comparator; +import java.util.List; +import java.util.Map; +import java.util.Objects; +import java.util.Optional; +import java.util.Set; +import java.util.concurrent.CompletableFuture; +import java.util.concurrent.CompletionException; +import java.util.concurrent.ExecutionException; +import java.util.concurrent.ExecutorService; +import java.util.concurrent.ThreadLocalRandom; +import java.util.concurrent.TimeUnit; +import java.util.concurrent.atomic.AtomicBoolean; +import java.util.function.Function; +import java.util.function.Supplier; +import java.util.stream.Collectors; +import java.util.stream.Stream; + +import static org.apache.ratis.server.impl.ServerImplUtils.assertEntries; +import static org.apache.ratis.server.impl.ServerImplUtils.assertGroup; import static org.apache.ratis.server.impl.ServerImplUtils.effectiveCommitIndex; import static org.apache.ratis.server.impl.ServerProtoUtils.toAppendEntriesReplyProto; +import static org.apache.ratis.server.impl.ServerProtoUtils.toReadIndexReplyProto; +import static org.apache.ratis.server.impl.ServerProtoUtils.toReadIndexRequestProto; +import static org.apache.ratis.server.impl.ServerProtoUtils.toRequestVoteReplyProto; +import static org.apache.ratis.server.impl.ServerProtoUtils.toStartLeaderElectionReplyProto; import static org.apache.ratis.server.util.ServerStringUtils.toAppendEntriesReplyString; import static org.apache.ratis.server.util.ServerStringUtils.toAppendEntriesRequestString; +import static org.apache.ratis.server.util.ServerStringUtils.toRequestVoteReplyString; class RaftServerImpl implements RaftServer.Division, RaftServerProtocol, RaftServerAsynchronousProtocol, @@ -611,46 +646,7 @@ GroupInfoReply getGroupInfo(GroupInfoRequest request) { } RoleInfoProto getRoleInfoProto() { - RaftPeerRole currentRole = role.getCurrentRole(); - RoleInfoProto.Builder roleInfo = RoleInfoProto.newBuilder() - .setSelf(getPeer().getRaftPeerProto()) - .setRole(currentRole) - .setRoleElapsedTimeMs(role.getRoleElapsedTimeMs()); - switch (currentRole) { - case CANDIDATE: - CandidateInfoProto.Builder candidate = CandidateInfoProto.newBuilder() - .setLastLeaderElapsedTimeMs(state.getLastLeaderElapsedTimeMs()); - roleInfo.setCandidateInfo(candidate); - break; - - case LISTENER: - case FOLLOWER: - final Optional fs = role.getFollowerState(); - final ServerRpcProto leaderInfo = ServerProtoUtils.toServerRpcProto( - getRaftConf().getPeer(state.getLeaderId()), - fs.map(FollowerState::getLastRpcTime).map(Timestamp::elapsedTimeMs).orElse(0L)); - // FollowerState can be null while adding a new peer as it is not - // a voting member yet - roleInfo.setFollowerInfo(FollowerInfoProto.newBuilder() - .setLeaderInfo(leaderInfo) - .setOutstandingOp(fs.map(FollowerState::getOutstandingOp).orElse(0))); - break; - - case LEADER: - role.getLeaderState().ifPresent(ls -> { - final LeaderInfoProto.Builder leader = LeaderInfoProto.newBuilder(); - ls.getLogAppenders().map(LogAppender::getFollower).forEach(f -> - leader.addFollowerInfo(ServerProtoUtils.toServerRpcProto( - f.getPeer(), f.getLastRpcResponseTime().elapsedTimeMs()))); - leader.setTerm(ls.getCurrentTerm()); - roleInfo.setLeaderInfo(leader); - }); - break; - - default: - throw new IllegalStateException("incorrect role of server " + currentRole); - } - return roleInfo.build(); + return role.buildRoleInfoProto(this); } synchronized void changeToCandidate(boolean forceStartLeaderElection) { @@ -711,7 +707,7 @@ private CompletableFuture checkLeaderState(RaftClientRequest re */ private CompletableFuture checkLeaderState(RaftClientRequest request, CacheEntry entry) { try { - assertGroup(request.getRequestorId(), request.getRaftGroupId()); + assertGroup(getMemberId(), request); } catch (GroupMismatchException e) { return RetryCacheImpl.failWithException(e, entry); } @@ -760,15 +756,6 @@ void assertLifeCycleState(Set expected) throws ServerNotReadyEx getMemberId() + " is not in " + expected + ": current state is " + c), expected); } - void assertGroup(Object requestorId, RaftGroupId requestorGroupId) throws GroupMismatchException { - final RaftGroupId groupId = getMemberId().getGroupId(); - if (!groupId.equals(requestorGroupId)) { - throw new GroupMismatchException(getMemberId() - + ": The group (" + requestorGroupId + ") of " + requestorId - + " does not match the group (" + groupId + ") of the server " + getId()); - } - } - /** * Append a transaction to the log for processing a client request. * Note that the given request could be different from {@link TransactionContext#getClientRequest()} @@ -998,8 +985,7 @@ private CompletableFuture sendReadIndexAsync(RaftClientRequ if (leaderId == null) { return JavaUtils.completeExceptionally(new ReadIndexException(getMemberId() + ": Leader is unknown.")); } - final ReadIndexRequestProto request = - ServerProtoUtils.toReadIndexRequestProto(clientRequest, getMemberId(), leaderId); + final ReadIndexRequestProto request = toReadIndexRequestProto(clientRequest, getMemberId(), leaderId); try { return getServerRpc().async().readIndexAsync(request); } catch (IOException e) { @@ -1169,7 +1155,7 @@ CompletableFuture transferLeadershipAsync(TransferLeadershipReq LOG.info("{}: receive transferLeadership {}", getMemberId(), request); assertLifeCycleState(LifeCycle.States.RUNNING); - assertGroup(request.getRequestorId(), request.getRaftGroupId()); + assertGroup(getMemberId(), request); synchronized (this) { CompletableFuture reply = checkLeaderState(request); @@ -1210,7 +1196,7 @@ CompletableFuture transferLeadershipAsync(TransferLeadershipReq CompletableFuture takeSnapshotAsync(SnapshotManagementRequest request) throws IOException { LOG.info("{}: takeSnapshotAsync {}", getMemberId(), request); assertLifeCycleState(LifeCycle.States.RUNNING); - assertGroup(request.getRequestorId(), request.getRaftGroupId()); + assertGroup(getMemberId(), request); //TODO(liuyaolong): get the gap value from shell command long minGapValue = RaftServerConfigKeys.Snapshot.creationGap(proxy.getProperties()); @@ -1242,7 +1228,7 @@ CompletableFuture leaderElectionManagementAsync(LeaderElectionM throws IOException { LOG.info("{} receive leaderElectionManagement request {}", getMemberId(), request); assertLifeCycleState(LifeCycle.States.RUNNING); - assertGroup(request.getRequestorId(), request.getRaftGroupId()); + assertGroup(getMemberId(), request); final LeaderElectionManagementRequest.Pause pause = request.getPause(); if (pause != null) { @@ -1261,7 +1247,7 @@ CompletableFuture leaderElectionManagementAsync(LeaderElectionM CompletableFuture stepDownLeaderAsync(TransferLeadershipRequest request) throws IOException { LOG.info("{} receive stepDown leader request {}", getMemberId(), request); assertLifeCycleState(LifeCycle.States.RUNNING); - assertGroup(request.getRequestorId(), request.getRaftGroupId()); + assertGroup(getMemberId(), request); return role.getLeaderState().map(leader -> leader.submitStepDownRequestAsync(request)) .orElseGet(() -> CompletableFuture.completedFuture( @@ -1278,7 +1264,7 @@ public RaftClientReply setConfiguration(SetConfigurationRequest request) throws public CompletableFuture setConfigurationAsync(SetConfigurationRequest request) throws IOException { LOG.info("{}: receive setConfiguration {}", getMemberId(), request); assertLifeCycleState(LifeCycle.States.RUNNING); - assertGroup(request.getRequestorId(), request.getRaftGroupId()); + assertGroup(getMemberId(), request); CompletableFuture reply = checkLeaderState(request); if (reply != null) { @@ -1357,15 +1343,13 @@ static List add(RaftPeerRole role, RaftConfigurationImpl conf, SetConf } /** - * check if the remote peer is not included in the current conf - * and should shutdown. should shutdown if all the following stands: - * 1. this is a leader + * The remote peer should shut down if all the following are true. + * 1. this is the current leader * 2. current conf is stable and has been committed - * 3. candidate id is not included in conf - * 4. candidate's last entry's index < conf's index + * 3. candidate is not in the current conf + * 4. candidate last entry index < conf index (the candidate was removed) */ - private boolean shouldSendShutdown(RaftPeerId candidateId, - TermIndex candidateLastEntry) { + private boolean shouldSendShutdown(RaftPeerId candidateId, TermIndex candidateLastEntry) { return getInfo().isLeader() && getRaftConf().isStable() && getState().isConfCommitted() @@ -1392,7 +1376,7 @@ private RequestVoteReplyProto requestVote(Phase phase, LOG.info("{}: receive requestVote({}, {}, {}, {}, {})", getMemberId(), phase, candidateId, candidateGroupId, candidateTerm, candidateLastEntry); assertLifeCycleState(LifeCycle.States.RUNNING); - assertGroup(candidateId, candidateGroupId); + assertGroup(getMemberId(), candidateId, candidateGroupId); boolean shouldShutdown = false; final RequestVoteReplyProto reply; @@ -1419,49 +1403,16 @@ private RequestVoteReplyProto requestVote(Phase phase, } else if(shouldSendShutdown(candidateId, candidateLastEntry)) { shouldShutdown = true; } - reply = ServerProtoUtils.toRequestVoteReplyProto(candidateId, getMemberId(), + reply = toRequestVoteReplyProto(candidateId, getMemberId(), voteGranted, state.getCurrentTerm(), shouldShutdown); if (LOG.isInfoEnabled()) { LOG.info("{} replies to {} vote request: {}. Peer's state: {}", - getMemberId(), phase, ServerStringUtils.toRequestVoteReplyString(reply), state); + getMemberId(), phase, toRequestVoteReplyString(reply), state); } } return reply; } - private void validateEntries(long expectedTerm, TermIndex previous, - List entries) { - if (entries != null && !entries.isEmpty()) { - final long index0 = entries.get(0).getIndex(); - // Check if next entry's index is 1 greater than the snapshotIndex. If yes, then - // we do not have to check for the existence of previous. - if (index0 != state.getSnapshotIndex() + 1) { - if (previous == null || previous.getTerm() == 0) { - Preconditions.assertTrue(index0 == 0, - "Unexpected Index: previous is null but entries[%s].getIndex()=%s", - 0, index0); - } else { - Preconditions.assertTrue(previous.getIndex() == index0 - 1, - "Unexpected Index: previous is %s but entries[%s].getIndex()=%s", - previous, 0, index0); - } - } - - for (int i = 0; i < entries.size(); i++) { - LogEntryProto entry = entries.get(i); - final long t = entry.getTerm(); - Preconditions.assertTrue(expectedTerm >= t, - "Unexpected Term: entries[%s].getTerm()=%s but expectedTerm=%s", - i, t, expectedTerm); - - final long indexi = entry.getIndex(); - Preconditions.assertTrue(indexi == index0 + i, - "Unexpected Index: entries[%s].getIndex()=%s but entries[0].getIndex()=%s", - i, indexi, index0); - } - } - } - @Override public AppendEntriesReplyProto appendEntries(AppendEntriesRequestProto r) throws IOException { @@ -1487,8 +1438,8 @@ public CompletableFuture appendEntriesAsync(AppendEntri if (!startComplete.get()) { throw new ServerNotReadyException(getMemberId() + ": The server role is not yet initialized."); } - assertGroup(leaderId, leaderGroupId); - validateEntries(r.getLeaderTerm(), previous, r.getEntriesList()); + assertGroup(getMemberId(), leaderId, leaderGroupId); + assertEntries(r, previous, state); return appendEntriesAsync(leaderId, request.getCallId(), previous, r); } catch(Exception t) { @@ -1505,14 +1456,12 @@ public CompletableFuture readIndexAsync(ReadIndexRequestPro final LeaderStateImpl leader = role.getLeaderState().orElse(null); if (leader == null) { - return CompletableFuture.completedFuture( - ServerProtoUtils.toReadIndexReplyProto(peerId, getMemberId(), false, RaftLog.INVALID_LOG_INDEX)); + return CompletableFuture.completedFuture(toReadIndexReplyProto(peerId, getMemberId())); } return getReadIndex(ClientProtoUtils.toRaftClientRequest(request.getClientRequest()), leader) - .thenApply(index -> ServerProtoUtils.toReadIndexReplyProto(peerId, getMemberId(), true, index)) - .exceptionally(throwable -> - ServerProtoUtils.toReadIndexReplyProto(peerId, getMemberId(), false, RaftLog.INVALID_LOG_INDEX)); + .thenApply(index -> toReadIndexReplyProto(peerId, getMemberId(), true, index)) + .exceptionally(throwable -> toReadIndexReplyProto(peerId, getMemberId())); } static void logAppendEntries(boolean isHeartbeat, Supplier message) { @@ -1715,37 +1664,37 @@ public StartLeaderElectionReplyProto startLeaderElection(StartLeaderElectionRequ if (!request.hasLeaderLastEntry()) { // It should have a leaderLastEntry since there is a placeHolder entry. LOG.warn("{}: leaderLastEntry is missing in {}", getMemberId(), request); - return ServerProtoUtils.toStartLeaderElectionReplyProto(leaderId, getMemberId(), false); + return toStartLeaderElectionReplyProto(leaderId, getMemberId(), false); } final TermIndex leaderLastEntry = TermIndex.valueOf(request.getLeaderLastEntry()); LOG.debug("{}: receive startLeaderElection from {} with lastEntry {}", getMemberId(), leaderId, leaderLastEntry); assertLifeCycleState(LifeCycle.States.RUNNING); - assertGroup(leaderId, leaderGroupId); + assertGroup(getMemberId(), leaderId, leaderGroupId); synchronized (this) { // Check life cycle state again to avoid the PAUSING/PAUSED state. assertLifeCycleState(LifeCycle.States.STARTING_OR_RUNNING); final boolean recognized = state.recognizeLeader("startLeaderElection", leaderId, leaderLastEntry.getTerm()); if (!recognized) { - return ServerProtoUtils.toStartLeaderElectionReplyProto(leaderId, getMemberId(), false); + return toStartLeaderElectionReplyProto(leaderId, getMemberId(), false); } if (!getInfo().isFollower()) { LOG.warn("{} refused StartLeaderElectionRequest from {}, because role is:{}", getMemberId(), leaderId, role.getCurrentRole()); - return ServerProtoUtils.toStartLeaderElectionReplyProto(leaderId, getMemberId(), false); + return toStartLeaderElectionReplyProto(leaderId, getMemberId(), false); } if (ServerState.compareLog(state.getLastEntry(), leaderLastEntry) < 0) { LOG.warn("{} refused StartLeaderElectionRequest from {}, because lastEntry:{} less than leaderEntry:{}", getMemberId(), leaderId, leaderLastEntry, state.getLastEntry()); - return ServerProtoUtils.toStartLeaderElectionReplyProto(leaderId, getMemberId(), false); + return toStartLeaderElectionReplyProto(leaderId, getMemberId(), false); } changeToCandidate(true); - return ServerProtoUtils.toStartLeaderElectionReplyProto(leaderId, getMemberId(), true); + return toStartLeaderElectionReplyProto(leaderId, getMemberId(), true); } } diff --git a/ratis-server/src/main/java/org/apache/ratis/server/impl/RoleInfo.java b/ratis-server/src/main/java/org/apache/ratis/server/impl/RoleInfo.java index fe2bc963b1..5eb01a9d6b 100644 --- a/ratis-server/src/main/java/org/apache/ratis/server/impl/RoleInfo.java +++ b/ratis-server/src/main/java/org/apache/ratis/server/impl/RoleInfo.java @@ -18,8 +18,14 @@ package org.apache.ratis.server.impl; +import org.apache.ratis.proto.RaftProtos.CandidateInfoProto; +import org.apache.ratis.proto.RaftProtos.FollowerInfoProto; +import org.apache.ratis.proto.RaftProtos.LeaderInfoProto; import org.apache.ratis.proto.RaftProtos.RaftPeerRole; +import org.apache.ratis.proto.RaftProtos.RoleInfoProto; +import org.apache.ratis.protocol.RaftPeer; import org.apache.ratis.protocol.RaftPeerId; +import org.apache.ratis.server.leader.LogAppender; import org.apache.ratis.util.JavaUtils; import org.apache.ratis.util.Preconditions; import org.apache.ratis.util.Timestamp; @@ -32,6 +38,8 @@ import java.util.concurrent.atomic.AtomicBoolean; import java.util.concurrent.atomic.AtomicReference; +import static org.apache.ratis.server.impl.ServerProtoUtils.toServerRpcProto; + /** * Maintain the Role of a Raft Peer. */ @@ -141,6 +149,55 @@ private T updateAndGet(AtomicReference ref, T current) { return updated; } + RoleInfoProto buildRoleInfoProto(RaftServerImpl server) { + final RaftPeerRole currentRole = getCurrentRole(); + final RoleInfoProto.Builder proto = RoleInfoProto.newBuilder() + .setSelf(server.getPeer().getRaftPeerProto()) + .setRole(currentRole) + .setRoleElapsedTimeMs(getRoleElapsedTimeMs()); + + switch (currentRole) { + case LEADER: + getLeaderState().ifPresent(leader -> { + final LeaderInfoProto.Builder b = LeaderInfoProto.newBuilder() + .setTerm(leader.getCurrentTerm()); + leader.getLogAppenders() + .map(LogAppender::getFollower) + .map(f -> toServerRpcProto(f.getPeer(), f.getLastRpcResponseTime().elapsedTimeMs())) + .forEach(b::addFollowerInfo); + proto.setLeaderInfo(b); + }); + return proto.build(); + + case CANDIDATE: + return proto.setCandidateInfo(CandidateInfoProto.newBuilder() + .setLastLeaderElapsedTimeMs(server.getState().getLastLeaderElapsedTimeMs())) + .build(); + + case LISTENER: + case FOLLOWER: + // FollowerState can be null while adding a new peer as it is not a voting member yet + final FollowerState follower = getFollowerState().orElse(null); + final long rpcElapsed; + final int outstandingOp; + if (follower != null) { + rpcElapsed = follower.getLastRpcTime().elapsedTimeMs(); + outstandingOp = follower.getOutstandingOp(); + } else { + rpcElapsed = 0; + outstandingOp = 0; + } + final RaftPeer leader = server.getRaftConf().getPeer(server.getState().getLeaderId()); + return proto.setFollowerInfo(FollowerInfoProto.newBuilder() + .setLeaderInfo(toServerRpcProto(leader, rpcElapsed)) + .setOutstandingOp(outstandingOp)) + .build(); + + default: + throw new IllegalStateException("Unexpected role " + currentRole); + } + } + @Override public String toString() { return String.format("%9s", role); diff --git a/ratis-server/src/main/java/org/apache/ratis/server/impl/ServerImplUtils.java b/ratis-server/src/main/java/org/apache/ratis/server/impl/ServerImplUtils.java index e4fe8f232f..e26c6e0ab1 100644 --- a/ratis-server/src/main/java/org/apache/ratis/server/impl/ServerImplUtils.java +++ b/ratis-server/src/main/java/org/apache/ratis/server/impl/ServerImplUtils.java @@ -19,9 +19,15 @@ import org.apache.ratis.conf.Parameters; import org.apache.ratis.conf.RaftProperties; +import org.apache.ratis.proto.RaftProtos.AppendEntriesRequestProto; +import org.apache.ratis.proto.RaftProtos.LogEntryProto; +import org.apache.ratis.protocol.RaftClientRequest; import org.apache.ratis.protocol.RaftGroup; +import org.apache.ratis.protocol.RaftGroupId; +import org.apache.ratis.protocol.RaftGroupMemberId; import org.apache.ratis.protocol.RaftPeer; import org.apache.ratis.protocol.RaftPeerId; +import org.apache.ratis.protocol.exceptions.GroupMismatchException; import org.apache.ratis.server.RaftConfiguration; import org.apache.ratis.server.RaftServer; import org.apache.ratis.server.protocol.TermIndex; @@ -35,7 +41,6 @@ import java.io.IOException; import java.util.List; -import java.util.Optional; import java.util.concurrent.TimeUnit; /** Server utilities for internal use. */ @@ -88,7 +93,51 @@ public static RaftConfiguration newRaftConfiguration(List conf, List entries = proto.getEntriesList(); + if (entries != null && !entries.isEmpty()) { + final long index0 = entries.get(0).getIndex(); + // Check if next entry's index is 1 greater than the snapshotIndex. If yes, then + // we do not have to check for the existence of previous. + if (index0 != state.getSnapshotIndex() + 1) { + final long expected = previous == null || previous.getTerm() == 0 ? 0 : previous.getIndex() + 1; + Preconditions.assertTrue(index0 == expected, + "Unexpected Index: previous is %s but entries[%s].getIndex() == %s != %s", + previous, 0, index0, expected); + } + + final long leaderTerm = proto.getLeaderTerm(); + for (int i = 0; i < entries.size(); i++) { + final LogEntryProto entry = entries.get(i); + final long entryTerm = entry.getTerm(); + Preconditions.assertTrue(entryTerm <= leaderTerm , + "Unexpected Term: entries[%s].getTerm() == %s > leaderTerm == %s", + i, entryTerm, leaderTerm); + + final long indexI = entry.getIndex(); + final long expected = index0 + i; + Preconditions.assertTrue(indexI == expected, + "Unexpected Index: entries[0].getIndex() == %s but entries[%s].getIndex() == %s != %s", + index0, i, indexI, expected); + } + } } } diff --git a/ratis-server/src/main/java/org/apache/ratis/server/impl/ServerProtoUtils.java b/ratis-server/src/main/java/org/apache/ratis/server/impl/ServerProtoUtils.java index f2be8c61c6..e35cb23867 100644 --- a/ratis-server/src/main/java/org/apache/ratis/server/impl/ServerProtoUtils.java +++ b/ratis-server/src/main/java/org/apache/ratis/server/impl/ServerProtoUtils.java @@ -126,6 +126,10 @@ static ReadIndexReplyProto toReadIndexReplyProto( .build(); } + static ReadIndexReplyProto toReadIndexReplyProto(RaftPeerId requestorId, RaftGroupMemberId replyId) { + return toReadIndexReplyProto(requestorId, replyId, false, RaftLog.INVALID_LOG_INDEX); + } + @SuppressWarnings("parameternumber") static AppendEntriesReplyProto toAppendEntriesReplyProto( RaftPeerId requestorId, RaftGroupMemberId replyId, long term, diff --git a/ratis-server/src/main/java/org/apache/ratis/server/impl/SnapshotInstallationHandler.java b/ratis-server/src/main/java/org/apache/ratis/server/impl/SnapshotInstallationHandler.java index 7aae944a43..3e5ac2b671 100644 --- a/ratis-server/src/main/java/org/apache/ratis/server/impl/SnapshotInstallationHandler.java +++ b/ratis-server/src/main/java/org/apache/ratis/server/impl/SnapshotInstallationHandler.java @@ -113,7 +113,7 @@ private InstallSnapshotReplyProto installSnapshotImpl(InstallSnapshotRequestProt CodeInjectionForTesting.execute(RaftServerImpl.INSTALL_SNAPSHOT, server.getId(), leaderId, request); server.assertLifeCycleState(LifeCycle.States.STARTING_OR_RUNNING); - server.assertGroup(leaderId, leaderGroupId); + ServerImplUtils.assertGroup(getMemberId(), leaderId, leaderGroupId); InstallSnapshotReplyProto reply = null; // Check if install snapshot from Leader is enabled From 9a62fa7a17cefe912c6bbccc81f008e697d2b3c5 Mon Sep 17 00:00:00 2001 From: Nandakumar Vadivelu Date: Tue, 20 Feb 2024 22:41:53 +0530 Subject: [PATCH 005/397] RATIS-2021. Migrate ratis-test tests to Junit 5 - Part 2. (#1043) --- .../java/org/apache/ratis/RaftBasicTests.java | 41 ++--- .../ratis/TestRaftServerNoLeaderTimeout.java | 20 +-- .../TestRaftServerSlownessDetection.java | 27 ++- .../apache/ratis/netty/TestRaftWithNetty.java | 2 +- .../ratis/netty/TestTlsConfWithNetty.java | 6 +- .../apache/ratis/protocol/TestRaftGroup.java | 16 +- .../org/apache/ratis/protocol/TestRaftId.java | 28 ++- .../ratis/protocol/TestRoutingTable.java | 14 +- .../retry/TestExceptionDependentRetry.java | 33 ++-- .../retry/TestExponentialBackoffRetry.java | 14 +- .../retry/TestMultipleLinearRandomRetry.java | 18 +- .../apache/ratis/retry/TestRetryPolicy.java | 40 ++--- .../ratis/security/SecurityTestUtils.java | 12 +- .../ratis/server/ServerBuilderTest.java | 8 +- .../ratis/server/ServerRestartTests.java | 38 ++-- .../server/TestRaftServerConfigKeys.java | 16 +- .../server/impl/TestPeerConfiguration.java | 6 +- .../server/impl/TestRaftConfiguration.java | 28 +-- .../ratis/server/impl/TestRaftServerJmx.java | 19 +- .../server/raftlog/TestRaftLogIndex.java | 12 +- .../server/raftlog/TestRaftLogMetrics.java | 44 ++--- .../raftlog/memory/MemoryRaftLogTest.java | 4 +- .../segmented/TestBufferedWriteChannel.java | 15 +- .../raftlog/segmented/TestCacheEviction.java | 58 +++--- .../raftlog/segmented/TestLogSegment.java | 80 ++++----- .../segmented/TestRaftLogReadWrite.java | 36 ++-- .../segmented/TestSegmentedRaftLog.java | 167 ++++++++++-------- .../segmented/TestSegmentedRaftLogCache.java | 106 +++++------ .../ratis/server/storage/TestRaftStorage.java | 86 ++++----- .../server/storage/TestStorageImplUtils.java | 22 +-- .../sh/ElectionCommandIntegrationTest.java | 44 ++--- .../cli/sh/GroupCommandIntegrationTest.java | 12 +- .../cli/sh/PeerCommandIntegrationTest.java | 16 +- .../sh/SnapshotCommandIntegrationTest.java | 18 +- .../ratis/shell/cli/sh/TestRatisShell.java | 12 +- .../ratis/statemachine/TestStateMachine.java | 14 +- .../ratis/util/TestDataBlockingQueue.java | 33 ++-- .../org/apache/ratis/util/TestDataQueue.java | 73 ++++---- .../org/apache/ratis/util/TestExitUtils.java | 31 ++-- .../org/apache/ratis/util/TestLifeCycle.java | 19 +- .../org/apache/ratis/util/TestMinMax.java | 16 +- .../apache/ratis/util/TestPeerProxyMap.java | 15 +- .../apache/ratis/util/TestPreconditions.java | 9 +- .../util/TestReferenceCountedObject.java | 45 ++--- .../ratis/util/TestResourceSemaphore.java | 14 +- .../apache/ratis/util/TestTimeDuration.java | 60 ++++--- .../ratis/util/TestTimeoutScheduler.java | 134 +++++++------- .../util/TestTraditionalBinaryPrefix.java | 20 ++- 48 files changed, 832 insertions(+), 769 deletions(-) diff --git a/ratis-server/src/test/java/org/apache/ratis/RaftBasicTests.java b/ratis-server/src/test/java/org/apache/ratis/RaftBasicTests.java index 4ff9681f0a..13ee08c83b 100644 --- a/ratis-server/src/test/java/org/apache/ratis/RaftBasicTests.java +++ b/ratis-server/src/test/java/org/apache/ratis/RaftBasicTests.java @@ -42,8 +42,9 @@ import org.apache.ratis.util.Slf4jUtils; import org.apache.ratis.util.TimeDuration; import org.apache.ratis.util.Timestamp; -import org.junit.Assert; -import org.junit.Test; +import org.junit.jupiter.api.Assertions; +import org.junit.jupiter.api.Assumptions; +import org.junit.jupiter.api.Test; import org.slf4j.Logger; import org.slf4j.event.Level; @@ -152,12 +153,12 @@ static void runTestBasicAppendEntries( }); } else { final RaftClientReply reply = client.io().send(message); - Assert.assertTrue(reply.isSuccess()); + Assertions.assertTrue(reply.isSuccess()); } } if (async) { f.join(); - Assert.assertEquals(messages.length, asyncReplyCount.get()); + Assertions.assertEquals(messages.length, asyncReplyCount.get()); } } Thread.sleep(cluster.getTimeoutMax().toIntExact(TimeUnit.MILLISECONDS) + 100); @@ -194,7 +195,7 @@ void runTestOldLeaderCommit(CLUSTER cluster) throws Exception { Thread.sleep(cluster.getTimeoutMax().toLong(TimeUnit.MILLISECONDS) + 100); for (RaftServer.Division followerToSendLog : followersToSendLog) { RaftLog followerLog = followerToSendLog.getRaftLog(); - Assert.assertTrue(RaftTestUtil.logEntriesContains(followerLog, messages)); + Assertions.assertTrue(RaftTestUtil.logEntriesContains(followerLog, messages)); } LOG.info(String.format("killing old leader: %s", leaderId.toString())); @@ -212,7 +213,7 @@ void runTestOldLeaderCommit(CLUSTER cluster) throws Exception { Set followersToSendLogIds = followersToSendLog.stream().map(f -> f.getId()).collect(Collectors.toSet()); - Assert.assertTrue(followersToSendLogIds.contains(newLeaderId)); + Assertions.assertTrue(followersToSendLogIds.contains(newLeaderId)); cluster.getServerAliveStream() .map(RaftServer.Division::getRaftLog) @@ -234,8 +235,8 @@ void runTestOldLeaderNotCommit(CLUSTER cluster) throws Exception { cluster.killServer(followers.get(i).getId()); } } catch (IndexOutOfBoundsException e) { - throw new org.junit.AssumptionViolatedException("The assumption is follower.size() = NUM_SERVERS - 1, " - + "actual NUM_SERVERS is " + NUM_SERVERS + ", and actual follower.size() is " + followers.size(), e); + Assumptions.abort("The assumption is follower.size() = NUM_SERVERS - 1, " + + "actual NUM_SERVERS is " + NUM_SERVERS + ", and actual follower.size() is " + followers.size()); } SimpleMessage[] messages = SimpleMessage.create(1); @@ -293,7 +294,7 @@ public void run() { if (!useAsync) { final RaftClientReply reply = client.io().send(messages[step.getAndIncrement()]); - Assert.assertTrue(reply.isSuccess()); + Assertions.assertTrue(reply.isSuccess()); } else { final CompletableFuture replyFuture = client.async().send(messages[i]); @@ -305,13 +306,13 @@ public void run() { if (step.incrementAndGet() == messages.length) { f.complete(null); } - Assert.assertTrue(r.isSuccess()); + Assertions.assertTrue(r.isSuccess()); }); } } if (useAsync) { f.join(); - Assert.assertTrue(step.get() == messages.length); + Assertions.assertEquals(step.get(), messages.length); } } catch(Exception t) { if (exceptionInClientThread.compareAndSet(null, t)) { @@ -393,7 +394,7 @@ public void run() { } final int n = clients.stream().mapToInt(c -> c.step.get()).sum(); - Assert.assertTrue(n >= lastStep.get()); + Assertions.assertTrue(n >= lastStep.get()); if (n - lastStep.get() < 50 * numClients) { // Change leader at least 50 steps. Thread.sleep(10); @@ -447,7 +448,7 @@ public static void testRequestTimeout(boolean async, MiniRaftCluster cluster, Lo // The duration for which the client waits should be more than the retryCacheExpiryDuration. final TimeDuration duration = startTime.elapsedTime(); TimeDuration retryCacheExpiryDuration = RaftServerConfigKeys.RetryCache.expiryTime(cluster.getProperties()); - Assert.assertTrue(duration.compareTo(retryCacheExpiryDuration) >= 0); + Assertions.assertTrue(duration.compareTo(retryCacheExpiryDuration) >= 0); } } @@ -479,10 +480,10 @@ static void runTestStateMachineMetrics(boolean async, MiniRaftCluster cluster) t long smAppliedIndexAfter = (Long) smAppliedIndexGauge.getValue(); checkFollowerCommitLagsLeader(cluster); - Assert.assertTrue("StateMachine Applied Index not incremented", - appliedIndexAfter > appliedIndexBefore); - Assert.assertTrue("StateMachine Apply completed Index not incremented", - smAppliedIndexAfter > smAppliedIndexBefore); + Assertions.assertTrue(appliedIndexAfter > appliedIndexBefore, + "StateMachine Applied Index not incremented"); + Assertions.assertTrue(smAppliedIndexAfter > smAppliedIndexBefore, + "StateMachine Apply completed Index not incremented"); } } @@ -495,12 +496,12 @@ private static void checkFollowerCommitLagsLeader(MiniRaftCluster cluster) { for (RaftServer.Division f : followers) { final RaftGroupMemberId follower = f.getMemberId(); Gauge followerCommitGauge = ServerMetricsTestUtils.getPeerCommitIndexGauge(leader, follower.getPeerId()); - Assert.assertTrue((Long)leaderCommitGauge.getValue() >= + Assertions.assertTrue((Long)leaderCommitGauge.getValue() >= (Long)followerCommitGauge.getValue()); Gauge followerMetric = ServerMetricsTestUtils.getPeerCommitIndexGauge(follower, follower.getPeerId()); System.out.println(followerCommitGauge.getValue()); System.out.println(followerMetric.getValue()); - Assert.assertTrue((Long)followerCommitGauge.getValue() <= (Long)followerMetric.getValue()); + Assertions.assertTrue((Long)followerCommitGauge.getValue() <= (Long)followerMetric.getValue()); } } @@ -511,7 +512,7 @@ private static Gauge getStatemachineGaugeWithName(RaftServer.Division server, St RATIS_STATEMACHINE_METRICS, RATIS_STATEMACHINE_METRICS_DESC); Optional metricRegistry = MetricRegistries.global().get(info); - Assert.assertTrue(metricRegistry.isPresent()); + Assertions.assertTrue(metricRegistry.isPresent()); return ServerMetricsTestUtils.getGaugeWithName(gaugeName, metricRegistry::get); } diff --git a/ratis-test/src/test/java/org/apache/ratis/TestRaftServerNoLeaderTimeout.java b/ratis-test/src/test/java/org/apache/ratis/TestRaftServerNoLeaderTimeout.java index 7f4da0e0fb..c10dc105a5 100644 --- a/ratis-test/src/test/java/org/apache/ratis/TestRaftServerNoLeaderTimeout.java +++ b/ratis-test/src/test/java/org/apache/ratis/TestRaftServerNoLeaderTimeout.java @@ -28,10 +28,10 @@ import org.apache.ratis.statemachine.StateMachine; import org.apache.ratis.util.Slf4jUtils; import org.apache.ratis.util.TimeDuration; -import org.junit.After; -import org.junit.Assert; -import org.junit.Before; -import org.junit.Test; +import org.junit.jupiter.api.AfterEach; +import org.junit.jupiter.api.Assertions; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.Test; import org.slf4j.event.Level; import java.io.IOException; @@ -60,13 +60,13 @@ private static RaftProperties getProperties() { return properties; } - @Before + @BeforeEach public void setup() throws IOException { - Assert.assertNull(cluster.getLeader()); + Assertions.assertNull(cluster.getLeader()); cluster.start(); } - @After + @AfterEach public void tearDown() { if (cluster != null) { cluster.shutdown(); @@ -91,10 +91,10 @@ public void testLeaderElectionDetection() throws Exception { RaftProtos.RoleInfoProto roleInfoProto = SimpleStateMachine4Testing.get(healthyFollower).getLeaderElectionTimeoutInfo(); - Assert.assertNotNull(roleInfoProto); + Assertions.assertNotNull(roleInfoProto); - Assert.assertEquals(roleInfoProto.getRole(), RaftProtos.RaftPeerRole.CANDIDATE); + Assertions.assertEquals(roleInfoProto.getRole(), RaftProtos.RaftPeerRole.CANDIDATE); final long noLeaderTimeoutMs = noLeaderTimeout.toLong(TimeUnit.MILLISECONDS); - Assert.assertTrue(roleInfoProto.getCandidateInfo().getLastLeaderElapsedTimeMs() > noLeaderTimeoutMs); + Assertions.assertTrue(roleInfoProto.getCandidateInfo().getLastLeaderElapsedTimeMs() > noLeaderTimeoutMs); } } diff --git a/ratis-test/src/test/java/org/apache/ratis/TestRaftServerSlownessDetection.java b/ratis-test/src/test/java/org/apache/ratis/TestRaftServerSlownessDetection.java index 0ea0fe9cef..b150a1aa05 100644 --- a/ratis-test/src/test/java/org/apache/ratis/TestRaftServerSlownessDetection.java +++ b/ratis-test/src/test/java/org/apache/ratis/TestRaftServerSlownessDetection.java @@ -30,12 +30,6 @@ import org.apache.ratis.statemachine.StateMachine; import org.apache.ratis.util.Slf4jUtils; import org.apache.ratis.util.TimeDuration; -import org.junit.After; -import org.junit.Assert; -import org.junit.Before; -import org.junit.Ignore; -import org.junit.Test; - import java.io.IOException; import java.util.List; @@ -43,13 +37,18 @@ import java.util.concurrent.TimeUnit; import org.apache.ratis.thirdparty.com.codahale.metrics.Gauge; +import org.junit.jupiter.api.AfterEach; +import org.junit.jupiter.api.Assertions; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.Disabled; +import org.junit.jupiter.api.Test; import org.slf4j.event.Level; /** * Test Raft Server Slowness detection and notification to Leader's statemachine. */ //TODO: fix StateMachine.notifySlowness(..); see RATIS-370 -@Ignore +@Disabled public class TestRaftServerSlownessDetection extends BaseTest { static { Slf4jUtils.setLogLevel(RaftServer.Division.LOG, Level.DEBUG); @@ -70,13 +69,13 @@ public RaftProperties getProperties() { return properties; } - @Before + @BeforeEach public void setup() throws IOException { - Assert.assertNull(cluster.getLeader()); + Assertions.assertNull(cluster.getLeader()); cluster.start(); } - @After + @AfterEach public void tearDown() { if (cluster != null) { cluster.shutdown(); @@ -107,23 +106,23 @@ public void testSlownessDetection() throws Exception { Thread.sleep( slownessTimeout * 2); long followerHeartBeatElapsedMetricNew = (long) metric.getValue(); - Assert.assertTrue(followerHeartBeatElapsedMetricNew > followerHeartBeatElapsedMetric); + Assertions.assertTrue(followerHeartBeatElapsedMetricNew > followerHeartBeatElapsedMetric); // Followers should not get any failed not notification for (RaftServer.Division followerServer : cluster.getFollowers()) { - Assert.assertNull(SimpleStateMachine4Testing.get(followerServer).getSlownessInfo()); + Assertions.assertNull(SimpleStateMachine4Testing.get(followerServer).getSlownessInfo()); } // the leader should get notification that the follower has failed now RaftProtos.RoleInfoProto roleInfoProto = SimpleStateMachine4Testing.get(cluster.getLeader()).getSlownessInfo(); - Assert.assertNotNull(roleInfoProto); + Assertions.assertNotNull(roleInfoProto); List followers = roleInfoProto.getLeaderInfo().getFollowerInfoList(); //Assert that the node shutdown is lagging behind for (RaftProtos.ServerRpcProto serverProto : followers) { if (RaftPeerId.valueOf(serverProto.getId().getId()).equals(failedFollower.getId())) { - Assert.assertTrue(serverProto.getLastRpcElapsedTimeMs() > slownessTimeout); + Assertions.assertTrue(serverProto.getLastRpcElapsedTimeMs() > slownessTimeout); } } } diff --git a/ratis-test/src/test/java/org/apache/ratis/netty/TestRaftWithNetty.java b/ratis-test/src/test/java/org/apache/ratis/netty/TestRaftWithNetty.java index 28815d76ce..55b9c37286 100644 --- a/ratis-test/src/test/java/org/apache/ratis/netty/TestRaftWithNetty.java +++ b/ratis-test/src/test/java/org/apache/ratis/netty/TestRaftWithNetty.java @@ -19,7 +19,7 @@ import org.apache.ratis.RaftBasicTests; import org.apache.ratis.server.impl.BlockRequestHandlingInjection; -import org.junit.Test; +import org.junit.jupiter.api.Test; public class TestRaftWithNetty extends RaftBasicTests diff --git a/ratis-test/src/test/java/org/apache/ratis/netty/TestTlsConfWithNetty.java b/ratis-test/src/test/java/org/apache/ratis/netty/TestTlsConfWithNetty.java index db967e391f..77910b2489 100644 --- a/ratis-test/src/test/java/org/apache/ratis/netty/TestTlsConfWithNetty.java +++ b/ratis-test/src/test/java/org/apache/ratis/netty/TestTlsConfWithNetty.java @@ -38,8 +38,8 @@ import org.apache.ratis.thirdparty.io.netty.handler.logging.LoggingHandler; import org.apache.ratis.thirdparty.io.netty.handler.ssl.SslContext; import org.apache.ratis.util.JavaUtils; -import org.junit.Assert; -import org.junit.Test; +import org.junit.jupiter.api.Assertions; +import org.junit.jupiter.api.Test; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -112,7 +112,7 @@ static void runTest(int port, TlsConf serverSslConf, TlsConf clientSslConf) thro final CompletableFuture future = replyFutures.get(i); final String reply = future.get(3, TimeUnit.SECONDS); LOG.info(reply); - Assert.assertEquals(NettyTestServer.toReply(words[i]), reply); + Assertions.assertEquals(NettyTestServer.toReply(words[i]), reply); } } } diff --git a/ratis-test/src/test/java/org/apache/ratis/protocol/TestRaftGroup.java b/ratis-test/src/test/java/org/apache/ratis/protocol/TestRaftGroup.java index 5267b22385..f4a88e40fe 100644 --- a/ratis-test/src/test/java/org/apache/ratis/protocol/TestRaftGroup.java +++ b/ratis-test/src/test/java/org/apache/ratis/protocol/TestRaftGroup.java @@ -18,27 +18,25 @@ package org.apache.ratis.protocol; import org.apache.ratis.BaseTest; -import org.apache.ratis.thirdparty.com.google.protobuf.ByteString; -import org.junit.Assert; -import org.junit.Test; +import org.junit.jupiter.api.Assertions; +import org.junit.jupiter.api.Test; +import org.junit.jupiter.api.Timeout; import java.util.LinkedList; import java.util.List; import java.util.UUID; +@Timeout(value = 1) public class TestRaftGroup extends BaseTest { - @Override - public int getGlobalTimeoutSeconds() { - return 1; - } - @Test(expected = IllegalStateException.class) + @Test public void testDuplicatePeerId() throws Exception { UUID groupId = UUID.fromString("02511d47-d67c-49a3-9011-abb3109a44c1"); List peers = new LinkedList<>(); peers.add(RaftPeer.newBuilder().setId("n0").build()); peers.add(RaftPeer.newBuilder().setId("n0").build()); - RaftGroup.valueOf(RaftGroupId.valueOf(groupId), peers); + Assertions.assertThrows(IllegalStateException.class, + () -> RaftGroup.valueOf(RaftGroupId.valueOf(groupId), peers)); } } diff --git a/ratis-test/src/test/java/org/apache/ratis/protocol/TestRaftId.java b/ratis-test/src/test/java/org/apache/ratis/protocol/TestRaftId.java index 6610b3d049..b0e31ce72e 100644 --- a/ratis-test/src/test/java/org/apache/ratis/protocol/TestRaftId.java +++ b/ratis-test/src/test/java/org/apache/ratis/protocol/TestRaftId.java @@ -19,16 +19,14 @@ import org.apache.ratis.BaseTest; import org.apache.ratis.thirdparty.com.google.protobuf.ByteString; -import org.junit.Assert; -import org.junit.Test; +import org.junit.jupiter.api.Assertions; +import org.junit.jupiter.api.Test; +import org.junit.jupiter.api.Timeout; import java.util.UUID; +@Timeout(value = 1) public class TestRaftId extends BaseTest { - @Override - public int getGlobalTimeoutSeconds() { - return 1; - } @Test public void testRaftId() { @@ -39,35 +37,35 @@ public void testRaftId() { static void assertRaftId(UUID original, ByteString expected) { final ByteString bytes = RaftId.toByteString(original); if (expected != null) { - Assert.assertEquals(expected, bytes); + Assertions.assertEquals(expected, bytes); } final UUID computed = RaftId.toUuid(bytes); - Assert.assertEquals(original, computed); - Assert.assertEquals(bytes, RaftId.toByteString(computed)); + Assertions.assertEquals(original, computed); + Assertions.assertEquals(bytes, RaftId.toByteString(computed)); } @Test public void testClientId() { final ClientId id = ClientId.randomId(); final ByteString bytes = id.toByteString(); - Assert.assertEquals(bytes, id.toByteString()); - Assert.assertEquals(id, ClientId.valueOf(bytes)); + Assertions.assertEquals(bytes, id.toByteString()); + Assertions.assertEquals(id, ClientId.valueOf(bytes)); } @Test public void testRaftGroupId() { final RaftGroupId id = RaftGroupId.randomId(); final ByteString bytes = id.toByteString(); - Assert.assertEquals(bytes, id.toByteString()); - Assert.assertEquals(id, RaftGroupId.valueOf(bytes)); + Assertions.assertEquals(bytes, id.toByteString()); + Assertions.assertEquals(id, RaftGroupId.valueOf(bytes)); } @Test public void testRaftPeerId() { final RaftPeerId id = RaftPeerId.valueOf("abc"); final ByteString bytes = id.toByteString(); - Assert.assertEquals(bytes, id.toByteString()); - Assert.assertEquals(id, RaftPeerId.valueOf(bytes)); + Assertions.assertEquals(bytes, id.toByteString()); + Assertions.assertEquals(id, RaftPeerId.valueOf(bytes)); } } diff --git a/ratis-test/src/test/java/org/apache/ratis/protocol/TestRoutingTable.java b/ratis-test/src/test/java/org/apache/ratis/protocol/TestRoutingTable.java index 58bdf07fac..f3e08ece7a 100644 --- a/ratis-test/src/test/java/org/apache/ratis/protocol/TestRoutingTable.java +++ b/ratis-test/src/test/java/org/apache/ratis/protocol/TestRoutingTable.java @@ -18,14 +18,12 @@ package org.apache.ratis.protocol; import org.apache.ratis.BaseTest; -import org.junit.Assert; -import org.junit.Test; +import org.junit.jupiter.api.Assertions; +import org.junit.jupiter.api.Test; +import org.junit.jupiter.api.Timeout; +@Timeout(value = 1) public class TestRoutingTable extends BaseTest { - @Override - public int getGlobalTimeoutSeconds() { - return 1; - } private final RaftPeerId[] peers = new RaftPeerId[10]; @@ -69,7 +67,7 @@ RoutingTable newRoutingTable(int... peerIndices) { } void testFailureCase(String name, int... peerIndices) { - Assert.assertEquals(0, peerIndices.length % 2); + Assertions.assertEquals(0, peerIndices.length % 2); testFailureCase(name + ": " + toString(peerIndices), () -> newRoutingTable(peerIndices), @@ -77,7 +75,7 @@ void testFailureCase(String name, int... peerIndices) { } String toString(int... peerIndices) { - Assert.assertEquals(0, peerIndices.length % 2); + Assertions.assertEquals(0, peerIndices.length % 2); if (peerIndices.length == 0) { return ""; } diff --git a/ratis-test/src/test/java/org/apache/ratis/retry/TestExceptionDependentRetry.java b/ratis-test/src/test/java/org/apache/ratis/retry/TestExceptionDependentRetry.java index 36e6dfbccf..560c0f9fdd 100644 --- a/ratis-test/src/test/java/org/apache/ratis/retry/TestExceptionDependentRetry.java +++ b/ratis-test/src/test/java/org/apache/ratis/retry/TestExceptionDependentRetry.java @@ -21,7 +21,6 @@ import org.apache.ratis.BaseTest; import org.apache.ratis.server.impl.MiniRaftCluster; import org.apache.ratis.RaftTestUtil; -import org.apache.ratis.client.retry.ClientRetryEvent; import org.apache.ratis.client.RaftClient; import org.apache.ratis.client.RaftClientConfigKeys; import org.apache.ratis.conf.RaftProperties; @@ -33,16 +32,14 @@ import org.apache.ratis.statemachine.impl.SimpleStateMachine4Testing; import org.apache.ratis.statemachine.StateMachine; import org.apache.ratis.util.TimeDuration; -import org.junit.Assert; -import org.junit.Test; +import org.junit.jupiter.api.Assertions; +import org.junit.jupiter.api.Test; import java.io.IOException; import java.util.concurrent.ExecutionException; import java.util.concurrent.TimeUnit; import java.util.concurrent.TimeoutException; -import static org.junit.Assert.fail; - /** * Class to test {@link ExceptionDependentRetry}. */ @@ -99,9 +96,9 @@ public void testExceptionDependentRetryFailureWithExceptionDuplicate() { builder.setExceptionToPolicy(IOException.class, RetryPolicies.retryUpToMaximumCountWithFixedSleep(1, TimeDuration.valueOf(1, TimeUnit.SECONDS))); - fail("testExceptionDependentRetryFailure failed"); + Assertions.fail("testExceptionDependentRetryFailure failed"); } catch (Exception ex) { - Assert.assertEquals(IllegalStateException.class, ex.getClass()); + Assertions.assertEquals(IllegalStateException.class, ex.getClass()); } } @@ -115,9 +112,9 @@ public void testExceptionDependentRetryFailureWithExceptionMappedToNull() { RetryPolicies.retryUpToMaximumCountWithFixedSleep(1, TimeDuration.valueOf(1, TimeUnit.SECONDS))); builder.setExceptionToPolicy(IOException.class, null); - fail("testExceptionDependentRetryFailure failed"); + Assertions.fail("testExceptionDependentRetryFailure failed"); } catch (Exception ex) { - Assert.assertEquals(IllegalStateException.class, ex.getClass()); + Assertions.assertEquals(IllegalStateException.class, ex.getClass()); } } @@ -131,9 +128,9 @@ public void testExceptionDependentRetryFailureWithNoDefault() { RetryPolicies.retryUpToMaximumCountWithFixedSleep(1, TimeDuration.valueOf(1, TimeUnit.SECONDS))); builder.build(); - fail("testExceptionDependentRetryFailureWithNoDefault failed"); + Assertions.fail("testExceptionDependentRetryFailureWithNoDefault failed"); } catch (Exception ex) { - Assert.assertEquals(IllegalStateException.class, ex.getClass()); + Assertions.assertEquals(IllegalStateException.class, ex.getClass()); } try { @@ -143,9 +140,9 @@ public void testExceptionDependentRetryFailureWithNoDefault() { RetryPolicies.retryUpToMaximumCountWithFixedSleep(1, TimeDuration.valueOf(1, TimeUnit.SECONDS))); builder.setDefaultPolicy(null); - fail("testExceptionDependentRetryFailureWithNoDefault failed"); + Assertions.fail("testExceptionDependentRetryFailureWithNoDefault failed"); } catch (Exception ex) { - Assert.assertEquals(IllegalStateException.class, ex.getClass()); + Assertions.assertEquals(IllegalStateException.class, ex.getClass()); } } @@ -157,11 +154,11 @@ private void testException(int retries, int maxAttempts, .handleAttemptFailure(TestRetryPolicy.newClientRetryEvent(i, null, exception)); final boolean expected = i < retries && i < maxAttempts; - Assert.assertEquals(expected, action.shouldRetry()); + Assertions.assertEquals(expected, action.shouldRetry()); if (expected) { - Assert.assertEquals(sleepTime, action.getSleepTime().getDuration()); + Assertions.assertEquals(sleepTime, action.getSleepTime().getDuration()); } else { - Assert.assertEquals(0L, action.getSleepTime().getDuration()); + Assertions.assertEquals(0L, action.getSleepTime().getDuration()); } } } @@ -195,11 +192,11 @@ void runTestExceptionRetryAttempts(MiniRaftClusterWithGrpc cluster) throws Excep SimpleStateMachine4Testing.get(leader).blockWriteStateMachineData(); client.async().send(new RaftTestUtil.SimpleMessage("2")).get(); - Assert.fail("Test should have failed."); + Assertions.fail("Test should have failed."); } catch (ExecutionException e) { RaftRetryFailureException rrfe = (RaftRetryFailureException) e.getCause(); final int expectedCount = 1 + retryCount; // new request attempt + retry attempts - Assert.assertEquals(expectedCount, rrfe.getAttemptCount()); + Assertions.assertEquals(expectedCount, rrfe.getAttemptCount()); } finally { SimpleStateMachine4Testing.get(leader).unblockWriteStateMachineData(); cluster.shutdown(); diff --git a/ratis-test/src/test/java/org/apache/ratis/retry/TestExponentialBackoffRetry.java b/ratis-test/src/test/java/org/apache/ratis/retry/TestExponentialBackoffRetry.java index 74f1807640..c4d1d3a9c0 100644 --- a/ratis-test/src/test/java/org/apache/ratis/retry/TestExponentialBackoffRetry.java +++ b/ratis-test/src/test/java/org/apache/ratis/retry/TestExponentialBackoffRetry.java @@ -19,8 +19,8 @@ import org.apache.ratis.BaseTest; import org.apache.ratis.util.TimeDuration; -import org.junit.Assert; -import org.junit.Test; +import org.junit.jupiter.api.Assertions; +import org.junit.jupiter.api.Test; import java.util.concurrent.TimeUnit; @@ -36,12 +36,12 @@ public void testExponentialBackoffRetry() { // Test maxAttempts ExponentialBackoffRetry retryPolicy = createPolicy(baseSleep, null, 1); - Assert.assertFalse(retryPolicy.handleAttemptFailure(() -> 1).shouldRetry()); + Assertions.assertFalse(retryPolicy.handleAttemptFailure(() -> 1).shouldRetry()); try { // baseSleep should not be null createPolicy(null, null, 1); - Assert.fail("Policy creation should have failed"); + Assertions.fail("Policy creation should have failed"); } catch (Exception e) { } @@ -67,9 +67,9 @@ private void assertSleep(ExponentialBackoffRetry retryPolicy, // sleep time with randomness added long randomizedDuration = action.getSleepTime().toLong(TimeUnit.MILLISECONDS); - Assert.assertTrue(action.shouldRetry()); - Assert.assertTrue(randomizedDuration >= d * 0.5); - Assert.assertTrue(randomizedDuration <= d * 1.5); + Assertions.assertTrue(action.shouldRetry()); + Assertions.assertTrue(randomizedDuration >= d * 0.5); + Assertions.assertTrue(randomizedDuration <= d * 1.5); } } diff --git a/ratis-test/src/test/java/org/apache/ratis/retry/TestMultipleLinearRandomRetry.java b/ratis-test/src/test/java/org/apache/ratis/retry/TestMultipleLinearRandomRetry.java index 09ae55f33d..eb4b7967cd 100644 --- a/ratis-test/src/test/java/org/apache/ratis/retry/TestMultipleLinearRandomRetry.java +++ b/ratis-test/src/test/java/org/apache/ratis/retry/TestMultipleLinearRandomRetry.java @@ -19,8 +19,8 @@ import org.apache.ratis.BaseTest; import org.apache.ratis.util.TimeDuration; -import org.junit.Assert; -import org.junit.Test; +import org.junit.jupiter.api.Assertions; +import org.junit.jupiter.api.Test; public class TestMultipleLinearRandomRetry extends BaseTest { @Override @@ -49,12 +49,12 @@ public void testParseCommaSeparated() { private static void assertIllegalInput(String input) { final MultipleLinearRandomRetry computed = MultipleLinearRandomRetry.parseCommaSeparated(input); - Assert.assertNull(computed); + Assertions.assertNull(computed); } private static MultipleLinearRandomRetry assertLegalInput(String expected, String input) { final MultipleLinearRandomRetry computed = MultipleLinearRandomRetry.parseCommaSeparated(input); - Assert.assertNotNull(computed); - Assert.assertTrue(computed.toString().endsWith(expected)); + Assertions.assertNotNull(computed); + Assertions.assertTrue(computed.toString().endsWith(expected)); return computed; } @@ -69,18 +69,18 @@ public void testMultipleLinearRandomRetry() { for (int j = 1; j <= counts[i]; j++) { final int attempt = ++k; final RetryPolicy.Action action = r.handleAttemptFailure(() -> attempt); - Assert.assertTrue(action.shouldRetry()); + Assertions.assertTrue(action.shouldRetry()); final TimeDuration randomized = action.getSleepTime(); final TimeDuration expected = times[i].to(randomized.getUnit()); final long d = expected.getDuration(); LOG.info("times[{},{}] = {}, randomized={}", i, j, times[i], randomized); - Assert.assertTrue(randomized.getDuration() >= d*0.5); - Assert.assertTrue(randomized.getDuration() < (d*1.5 + precision)); + Assertions.assertTrue(randomized.getDuration() >= d*0.5); + Assertions.assertTrue(randomized.getDuration() < (d*1.5 + precision)); } } final int attempt = ++k; final RetryPolicy.Action action = r.handleAttemptFailure(() -> attempt); - Assert.assertFalse(action.shouldRetry()); + Assertions.assertFalse(action.shouldRetry()); } } \ No newline at end of file diff --git a/ratis-test/src/test/java/org/apache/ratis/retry/TestRetryPolicy.java b/ratis-test/src/test/java/org/apache/ratis/retry/TestRetryPolicy.java index 1b9536b4b6..cee8ee3385 100644 --- a/ratis-test/src/test/java/org/apache/ratis/retry/TestRetryPolicy.java +++ b/ratis-test/src/test/java/org/apache/ratis/retry/TestRetryPolicy.java @@ -34,8 +34,8 @@ import org.apache.ratis.protocol.exceptions.ResourceUnavailableException; import org.apache.ratis.util.TimeDuration; import org.apache.ratis.util.Timestamp; -import org.junit.Assert; -import org.junit.Test; +import org.junit.jupiter.api.Assertions; +import org.junit.jupiter.api.Test; import java.io.IOException; import java.util.ArrayList; @@ -62,11 +62,11 @@ public void testRetryMultipleTimesWithFixedSleep() { final RetryPolicy.Action action = policy.handleAttemptFailure(event); final boolean expected = i < n; - Assert.assertEquals(expected, action.shouldRetry()); + Assertions.assertEquals(expected, action.shouldRetry()); if (expected) { - Assert.assertEquals(sleepTime, action.getSleepTime()); + Assertions.assertEquals(sleepTime, action.getSleepTime()); } else { - Assert.assertEquals(0L, action.getSleepTime().getDuration()); + Assertions.assertEquals(0L, action.getSleepTime().getDuration()); } } } @@ -97,33 +97,33 @@ public void testRequestTypeDependentRetry() { final RetryPolicy.Action action = policy.handleAttemptFailure(event); final boolean expected = i < n; - Assert.assertEquals(expected, action.shouldRetry()); + Assertions.assertEquals(expected, action.shouldRetry()); if (expected) { - Assert.assertEquals(writeSleep, action.getSleepTime()); + Assertions.assertEquals(writeSleep, action.getSleepTime()); } else { - Assert.assertEquals(0L, action.getSleepTime().getDuration()); + Assertions.assertEquals(0L, action.getSleepTime().getDuration()); } } { //read and stale read are using default final ClientRetryEvent event = newClientRetryEvent(i, readRequest, null); final RetryPolicy.Action action = policy.handleAttemptFailure(event); - Assert.assertTrue(action.shouldRetry()); - Assert.assertEquals(0L, action.getSleepTime().getDuration()); + Assertions.assertTrue(action.shouldRetry()); + Assertions.assertEquals(0L, action.getSleepTime().getDuration()); } { final ClientRetryEvent event = newClientRetryEvent(i, staleReadRequest, null); final RetryPolicy.Action action = policy.handleAttemptFailure(event); - Assert.assertTrue(action.shouldRetry()); - Assert.assertEquals(0L, action.getSleepTime().getDuration()); + Assertions.assertTrue(action.shouldRetry()); + Assertions.assertEquals(0L, action.getSleepTime().getDuration()); } { //watch has no retry final ClientRetryEvent event = newClientRetryEvent(i, watchRequest, null); final RetryPolicy.Action action = policy.handleAttemptFailure(event); - Assert.assertFalse(action.shouldRetry()); - Assert.assertEquals(0L, action.getSleepTime().getDuration()); + Assertions.assertFalse(action.shouldRetry()); + Assertions.assertEquals(0L, action.getSleepTime().getDuration()); } } @@ -155,15 +155,15 @@ public RaftClientRequest newRequestImpl() { for (RaftClientRequest request : requests) { final ClientRetryEvent event = pending.newClientRetryEvent(request, new Exception()); final RetryPolicy.Action action = policy.handleAttemptFailure(event); - Assert.assertTrue(action.shouldRetry()); - Assert.assertEquals(0L, action.getSleepTime().getDuration()); + Assertions.assertTrue(action.shouldRetry()); + Assertions.assertEquals(0L, action.getSleepTime().getDuration()); } timeout.sleep(); for (RaftClientRequest request : requests) { final ClientRetryEvent event = pending.newClientRetryEvent(request, new Exception()); final RetryPolicy.Action action = policy.handleAttemptFailure(event); - Assert.assertFalse(action.shouldRetry()); + Assertions.assertFalse(action.shouldRetry()); } } @@ -227,11 +227,11 @@ private void checkEvent(int exceptionAttemptCount, RetryPolicy retryPolicy, Raft final RetryPolicy.Action action = retryPolicy.handleAttemptFailure(event); final boolean expected = exceptionAttemptCount < exceptionPolicyPair.retries; - Assert.assertEquals(expected, action.shouldRetry()); + Assertions.assertEquals(expected, action.shouldRetry()); if (expected) { - Assert.assertEquals(exceptionPolicyPair.sleepTime, action.getSleepTime().getDuration()); + Assertions.assertEquals(exceptionPolicyPair.sleepTime, action.getSleepTime().getDuration()); } else { - Assert.assertEquals(0L, action.getSleepTime().getDuration()); + Assertions.assertEquals(0L, action.getSleepTime().getDuration()); } } diff --git a/ratis-test/src/test/java/org/apache/ratis/security/SecurityTestUtils.java b/ratis-test/src/test/java/org/apache/ratis/security/SecurityTestUtils.java index d6222b2271..c390f10f4a 100644 --- a/ratis-test/src/test/java/org/apache/ratis/security/SecurityTestUtils.java +++ b/ratis-test/src/test/java/org/apache/ratis/security/SecurityTestUtils.java @@ -23,7 +23,7 @@ import org.apache.ratis.util.FileUtils; import org.bouncycastle.util.io.pem.PemObject; import org.bouncycastle.util.io.pem.PemReader; -import org.junit.Assert; +import org.junit.jupiter.api.Assertions; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -100,7 +100,7 @@ static PrivateKey getPrivateKey(String keyPath) { KeyFactory keyFactory = KeyFactory.getInstance("RSA"); return keyFactory.generatePrivate(privKeySpec); } catch (Exception e) { - Assert.fail("Failed to get private key from " + keyPath + ". Error: " + + Assertions.fail("Failed to get private key from " + keyPath + ". Error: " + e.getMessage()); } return null; @@ -116,7 +116,7 @@ static X509Certificate[] getCertificate(String certPath) { } return certificate; } catch (Exception e) { - Assert.fail("Failed to get certificate from " + certPath + ". Error: " + + Assertions.fail("Failed to get certificate from " + certPath + ". Error: " + e.getMessage()); } return null; @@ -133,7 +133,7 @@ static KeyStore getServerKeyStore() { keyStore.setKeyEntry("ratis-server-key", privateKey, new char[0], certificate); return keyStore; } catch (Exception e) { - Assert.fail("Failed to get sever key store " + e.getMessage()); + Assertions.fail("Failed to get sever key store " + e.getMessage()); } return null; } @@ -149,7 +149,7 @@ static KeyStore getClientKeyStore() { keyStore.setKeyEntry("ratis-client-key", privateKey, new char[0], certificate); return keyStore; } catch (Exception e) { - Assert.fail("Failed to get client key store " + e.getMessage()); + Assertions.fail("Failed to get client key store " + e.getMessage()); } return null; } @@ -167,7 +167,7 @@ static KeyStore getTrustStore() { } return trustStore; } catch (Exception e) { - Assert.fail("Failed to get sever key store " + e.getMessage()); + Assertions.fail("Failed to get sever key store " + e.getMessage()); } return null; } diff --git a/ratis-test/src/test/java/org/apache/ratis/server/ServerBuilderTest.java b/ratis-test/src/test/java/org/apache/ratis/server/ServerBuilderTest.java index dd76a2ec1e..9360b9b847 100644 --- a/ratis-test/src/test/java/org/apache/ratis/server/ServerBuilderTest.java +++ b/ratis-test/src/test/java/org/apache/ratis/server/ServerBuilderTest.java @@ -26,8 +26,8 @@ import org.apache.ratis.protocol.RaftPeerId; import org.apache.ratis.statemachine.impl.BaseStateMachine; import org.apache.ratis.util.Preconditions; -import org.junit.Assert; -import org.junit.Test; +import org.junit.jupiter.api.Assertions; +import org.junit.jupiter.api.Test; /** * Test {@link RaftServer.Builder}. @@ -58,7 +58,7 @@ public void testPeerIdNotInRaftGroup() { .setStateMachine(new BaseStateMachine()) .setProperties(new RaftProperties()) .build(); - Assert.fail("did not get expected exception"); + Assertions.fail("did not get expected exception"); } catch (IOException e) { Preconditions.assertInstanceOf(e.getCause(), IllegalStateException.class); } @@ -74,7 +74,7 @@ public void testNullPeerIdWithRaftGroup() { .setStateMachine(new BaseStateMachine()) .setProperties(new RaftProperties()) .build(); - Assert.fail("did not get expected exception"); + Assertions.fail("did not get expected exception"); } catch (IOException e) { Preconditions.assertInstanceOf(e.getCause(), IllegalStateException.class); } diff --git a/ratis-test/src/test/java/org/apache/ratis/server/ServerRestartTests.java b/ratis-test/src/test/java/org/apache/ratis/server/ServerRestartTests.java index 2f3edf7815..db4e92b7ce 100644 --- a/ratis-test/src/test/java/org/apache/ratis/server/ServerRestartTests.java +++ b/ratis-test/src/test/java/org/apache/ratis/server/ServerRestartTests.java @@ -46,8 +46,8 @@ import org.apache.ratis.util.SizeInBytes; import org.apache.ratis.util.StringUtils; import org.apache.ratis.util.TimeDuration; -import org.junit.Assert; -import org.junit.Test; +import org.junit.jupiter.api.Assertions; +import org.junit.jupiter.api.Test; import org.slf4j.Logger; import org.slf4j.event.Level; @@ -111,7 +111,7 @@ void runTestRestartFollower(MiniRaftCluster cluster) throws Exception { // make sure the restarted follower can catchup final RaftServer.Division followerState = cluster.getDivision(followerId); JavaUtils.attemptRepeatedly(() -> { - Assert.assertTrue(followerState.getInfo().getLastAppliedIndex() >= leaderLastIndex); + Assertions.assertTrue(followerState.getInfo().getLastAppliedIndex() >= leaderLastIndex); return null; }, 10, ONE_SECOND, "follower catchup", LOG); @@ -119,9 +119,9 @@ void runTestRestartFollower(MiniRaftCluster cluster) throws Exception { final RaftServer.Division follower = cluster.restartServer(followerId, false); final RaftLog followerLog = follower.getRaftLog(); final long followerLastIndex = followerLog.getLastEntryTermIndex().getIndex(); - Assert.assertTrue(followerLastIndex >= leaderLastIndex); + Assertions.assertTrue(followerLastIndex >= leaderLastIndex); final long leaderFinalIndex = cluster.getLeader().getRaftLog().getLastEntryTermIndex().getIndex(); - Assert.assertEquals(leaderFinalIndex, followerLastIndex); + Assertions.assertEquals(leaderFinalIndex, followerLastIndex); final File followerOpenLogFile = getOpenLogFile(follower); final File leaderOpenLogFile = getOpenLogFile(cluster.getDivision(leaderId)); @@ -148,7 +148,7 @@ void runTestRestartFollower(MiniRaftCluster cluster) throws Exception { if (i != truncatedMessageIndex) { final Message m = new SimpleMessage("m" + i); final RaftClientReply reply = client.io().sendReadOnly(m); - Assert.assertTrue(reply.isSuccess()); + Assertions.assertTrue(reply.isSuccess()); LOG.info("query {}: {} {}", m, reply, LogEntryProto.parseFrom(reply.getMessage().getContent())); } } @@ -159,7 +159,7 @@ static void writeSomething(Supplier newMessage, MiniRaftCluster cluster try(final RaftClient client = cluster.createClient()) { // write some messages for(int i = 0; i < 10; i++) { - Assert.assertTrue(client.io().send(newMessage.get()).isSuccess()); + Assertions.assertTrue(client.io().send(newMessage.get()).isSuccess()); } } } @@ -171,7 +171,7 @@ static void assertTruncatedLog(RaftPeerId id, File openLogFile, long lastIndex, } final RaftServer.Division server = cluster.restartServer(id, false); // the last index should be one less than before - Assert.assertEquals(lastIndex - 1, server.getRaftLog().getLastEntryTermIndex().getIndex()); + Assertions.assertEquals(lastIndex - 1, server.getRaftLog().getLastEntryTermIndex().getIndex()); server.getRaftServer().close(); } @@ -184,7 +184,7 @@ static List getOpenLogFiles(RaftServer.Division server) throws Exception { static File getOpenLogFile(RaftServer.Division server) throws Exception { final List openLogs = getOpenLogFiles(server); - Assert.assertEquals(1, openLogs.size()); + Assertions.assertEquals(1, openLogs.size()); return openLogs.get(0).toFile(); } @@ -210,7 +210,7 @@ void runTestRestartWithCorruptedLogHeader(MiniRaftCluster cluster) throws Except 10, HUNDRED_MILLIS, impl.getId() + "-getOpenLogFile", LOG); for(int i = 0; i < SegmentedRaftLogFormat.getHeaderLength(); i++) { assertCorruptedLogHeader(impl.getId(), openLogFile, i, cluster, LOG); - Assert.assertTrue(getOpenLogFiles(impl).isEmpty()); + Assertions.assertTrue(getOpenLogFiles(impl).isEmpty()); } } } @@ -245,7 +245,7 @@ void runTestRestartCommitIndex(MiniRaftCluster cluster) throws Exception { final SimpleMessage m = messages[i]; new Thread(() -> { try (final RaftClient client = cluster.createClient()) { - Assert.assertTrue(client.io().send(m).isSuccess()); + Assertions.assertTrue(client.io().send(m).isSuccess()); } catch (IOException e) { throw new IllegalStateException("Failed to send " + m, e); } @@ -301,11 +301,11 @@ void runTestRestartCommitIndex(MiniRaftCluster cluster) throws Exception { final RaftServer.Division server = cluster.getDivision(id); final RaftLog raftLog = server.getRaftLog(); JavaUtils.attemptRepeatedly(() -> { - Assert.assertTrue(raftLog.getLastCommittedIndex() >= loggedCommitIndex); + Assertions.assertTrue(raftLog.getLastCommittedIndex() >= loggedCommitIndex); return null; }, 10, HUNDRED_MILLIS, id + "(commitIndex >= loggedCommitIndex)", LOG); JavaUtils.attemptRepeatedly(() -> { - Assert.assertTrue(server.getInfo().getLastAppliedIndex() >= loggedCommitIndex); + Assertions.assertTrue(server.getInfo().getLastAppliedIndex() >= loggedCommitIndex); return null; }, 10, HUNDRED_MILLIS, id + "(lastAppliedIndex >= loggedCommitIndex)", LOG); LOG.info("{}: commitIndex={}, lastAppliedIndex={}", @@ -318,16 +318,16 @@ static void assertLastLogEntry(RaftServer.Division server) throws RaftLogIOExcep final RaftLog raftLog = server.getRaftLog(); final long lastIndex = raftLog.getLastEntryTermIndex().getIndex(); final LogEntryProto lastEntry = raftLog.get(lastIndex); - Assert.assertTrue(lastEntry.hasMetadataEntry()); + Assertions.assertTrue(lastEntry.hasMetadataEntry()); final long loggedCommitIndex = lastEntry.getMetadataEntry().getCommitIndex(); final LogEntryProto lastCommittedEntry = raftLog.get(loggedCommitIndex); - Assert.assertTrue(lastCommittedEntry.hasStateMachineLogEntry()); + Assertions.assertTrue(lastCommittedEntry.hasStateMachineLogEntry()); final SimpleStateMachine4Testing leaderStateMachine = SimpleStateMachine4Testing.get(server); final TermIndex lastAppliedTermIndex = leaderStateMachine.getLastAppliedTermIndex(); - Assert.assertEquals(lastCommittedEntry.getTerm(), lastAppliedTermIndex.getTerm()); - Assert.assertTrue(lastCommittedEntry.getIndex() <= lastAppliedTermIndex.getIndex()); + Assertions.assertEquals(lastCommittedEntry.getTerm(), lastAppliedTermIndex.getTerm()); + Assertions.assertTrue(lastCommittedEntry.getIndex() <= lastAppliedTermIndex.getIndex()); } @Test @@ -364,11 +364,11 @@ private void runTestRestartWithCorruptedLogEntry(CLUSTER cluster) throws Excepti final SimpleMessage lastMessage = messages[messages.length - 1]; try (final RaftClient client = cluster.createClient()) { for (SimpleMessage m : messages) { - Assert.assertTrue(client.io().send(m).isSuccess()); + Assertions.assertTrue(client.io().send(m).isSuccess()); } // assert that the last message exists - Assert.assertTrue(client.io().sendReadOnly(lastMessage).isSuccess()); + Assertions.assertTrue(client.io().sendReadOnly(lastMessage).isSuccess()); } final RaftLog log = leader.getRaftLog(); diff --git a/ratis-test/src/test/java/org/apache/ratis/server/TestRaftServerConfigKeys.java b/ratis-test/src/test/java/org/apache/ratis/server/TestRaftServerConfigKeys.java index bb386e8309..2026398e33 100644 --- a/ratis-test/src/test/java/org/apache/ratis/server/TestRaftServerConfigKeys.java +++ b/ratis-test/src/test/java/org/apache/ratis/server/TestRaftServerConfigKeys.java @@ -23,9 +23,9 @@ import org.apache.ratis.util.FileUtils; import org.apache.ratis.util.JavaUtils; import org.apache.ratis.util.SizeInBytes; -import org.junit.AfterClass; -import org.junit.Assert; -import org.junit.Test; +import org.junit.jupiter.api.AfterAll; +import org.junit.jupiter.api.Assertions; +import org.junit.jupiter.api.Test; import java.io.File; import java.io.IOException; @@ -51,7 +51,7 @@ public class TestRaftServerConfigKeys { JavaUtils.getClassSimpleName(TestRaftServerConfigKeys.class) + Integer.toHexString(ThreadLocalRandom.current().nextInt()))); - @AfterClass + @AfterAll public static void tearDown() throws IOException { FileUtils.deleteFully(rootTestDir.get()); } @@ -74,7 +74,7 @@ public void testStorageDirProperty() { final String expected = directories.stream().map(File::getAbsolutePath) .collect(Collectors.joining(",")); final String actual = properties.get(RaftServerConfigKeys.STORAGE_DIR_KEY); - Assert.assertEquals(expected, actual); + Assertions.assertEquals(expected, actual); } /** @@ -97,8 +97,8 @@ public void testStorageDir() { final List actualDirs = storageDirs.stream() .map(File::getAbsolutePath).collect(Collectors.toList()); actualDirs.removeAll(expectedDirs); - Assert.assertEquals(directories.size(), storageDirs.size()); - Assert.assertEquals(0, actualDirs.size()); + Assertions.assertEquals(directories.size(), storageDirs.size()); + Assertions.assertEquals(0, actualDirs.size()); } /** @@ -113,6 +113,6 @@ public void testStorageDir() { int pendingRequestMegabyteLimit = Math.toIntExact( RaftServerConfigKeys.Write.byteLimit(properties).getSize() / SizeInBytes.ONE_MB.getSize()); - Assert.assertEquals(4096, pendingRequestMegabyteLimit); + Assertions.assertEquals(4096, pendingRequestMegabyteLimit); } } \ No newline at end of file diff --git a/ratis-test/src/test/java/org/apache/ratis/server/impl/TestPeerConfiguration.java b/ratis-test/src/test/java/org/apache/ratis/server/impl/TestPeerConfiguration.java index e1adcfa442..df4d53cf0b 100644 --- a/ratis-test/src/test/java/org/apache/ratis/server/impl/TestPeerConfiguration.java +++ b/ratis-test/src/test/java/org/apache/ratis/server/impl/TestPeerConfiguration.java @@ -20,14 +20,14 @@ import org.apache.ratis.BaseTest; import org.apache.ratis.protocol.RaftPeer; import org.apache.ratis.protocol.RaftPeerId; -import org.junit.Test; +import org.junit.jupiter.api.Test; import java.util.Arrays; import java.util.Collection; import java.util.stream.Collectors; -import static org.junit.Assert.assertFalse; -import static org.junit.Assert.assertTrue; +import static org.junit.jupiter.api.Assertions.assertFalse; +import static org.junit.jupiter.api.Assertions.assertTrue; public class TestPeerConfiguration extends BaseTest { @Test diff --git a/ratis-test/src/test/java/org/apache/ratis/server/impl/TestRaftConfiguration.java b/ratis-test/src/test/java/org/apache/ratis/server/impl/TestRaftConfiguration.java index 14e0030e6d..77ec0ed29a 100644 --- a/ratis-test/src/test/java/org/apache/ratis/server/impl/TestRaftConfiguration.java +++ b/ratis-test/src/test/java/org/apache/ratis/server/impl/TestRaftConfiguration.java @@ -22,14 +22,14 @@ import org.apache.ratis.protocol.RaftPeer; import org.apache.ratis.protocol.RaftPeerId; import org.apache.ratis.server.RaftConfiguration; -import org.junit.Test; +import org.junit.jupiter.api.Test; import java.util.Arrays; import java.util.Collection; import java.util.stream.Collectors; -import static org.junit.Assert.assertFalse; -import static org.junit.Assert.assertTrue; +import static org.junit.jupiter.api.Assertions.assertFalse; +import static org.junit.jupiter.api.Assertions.assertTrue; public class TestRaftConfiguration extends BaseTest { @Test @@ -65,29 +65,29 @@ public void testSingleMode() { RaftConfigurationImpl config = RaftConfigurationImpl.newBuilder() .setConf(new PeerConfiguration(raftPeersWithPriority(1))) .build(); - assertTrue("Peer is in single mode.", config.isSingleMode(RaftPeerId.valueOf("1"))); + assertTrue(config.isSingleMode(RaftPeerId.valueOf("1")), "Peer is in single mode."); config = RaftConfigurationImpl.newBuilder() .setConf(new PeerConfiguration(raftPeersWithPriority(0, 1))) .setOldConf(new PeerConfiguration(raftPeersWithPriority(0))) .build(); - assertTrue("Peer is in single mode.", config.isSingleMode(RaftPeerId.valueOf("0"))); - assertFalse("Peer is a new peer.", config.isSingleMode(RaftPeerId.valueOf("1"))); + assertTrue(config.isSingleMode(RaftPeerId.valueOf("0")), "Peer is in single mode."); + assertFalse(config.isSingleMode(RaftPeerId.valueOf("1")), "Peer is a new peer"); config = RaftConfigurationImpl.newBuilder() .setConf(new PeerConfiguration(raftPeersWithPriority(0, 1))) .build(); - assertFalse("Peer is in ha mode.", config.isSingleMode(RaftPeerId.valueOf("0"))); - assertFalse("Peer is in ha mode.", config.isSingleMode(RaftPeerId.valueOf("1"))); + assertFalse(config.isSingleMode(RaftPeerId.valueOf("0")), "Peer is in ha mode."); + assertFalse(config.isSingleMode(RaftPeerId.valueOf("1")), "Peer is in ha mode."); config = RaftConfigurationImpl.newBuilder() .setConf(new PeerConfiguration(raftPeersWithPriority(0, 1))) .setOldConf(new PeerConfiguration(raftPeersWithPriority(2, 3))) .build(); - assertFalse("Peer is in ha mode.", config.isSingleMode(RaftPeerId.valueOf("0"))); - assertFalse("Peer is in ha mode.", config.isSingleMode(RaftPeerId.valueOf("1"))); - assertFalse("Peer is in ha mode.", config.isSingleMode(RaftPeerId.valueOf("3"))); - assertFalse("Peer is in ha mode.", config.isSingleMode(RaftPeerId.valueOf("4"))); + assertFalse(config.isSingleMode(RaftPeerId.valueOf("0")), "Peer is in ha mode."); + assertFalse(config.isSingleMode(RaftPeerId.valueOf("1")), "Peer is in ha mode."); + assertFalse(config.isSingleMode(RaftPeerId.valueOf("3")), "Peer is in ha mode."); + assertFalse(config.isSingleMode(RaftPeerId.valueOf("4")), "Peer is in ha mode."); } @Test @@ -96,8 +96,8 @@ public void testChangeMajority() { RaftConfigurationImpl config = RaftConfigurationImpl.newBuilder() .setConf(new PeerConfiguration(raftPeersWithPriority(1))) .build(); - assertFalse("Change from single mode to ha mode is not considered as changing majority.", - config.changeMajority(raftPeersWithPriority(1, 2))); + assertFalse(config.changeMajority(raftPeersWithPriority(1, 2)), + "Change from single mode to ha mode is not considered as changing majority."); // Case 2: {1} --> {2}. assertTrue(config.changeMajority(raftPeersWithPriority(2))); diff --git a/ratis-test/src/test/java/org/apache/ratis/server/impl/TestRaftServerJmx.java b/ratis-test/src/test/java/org/apache/ratis/server/impl/TestRaftServerJmx.java index 01d8392303..39a1f8855b 100644 --- a/ratis-test/src/test/java/org/apache/ratis/server/impl/TestRaftServerJmx.java +++ b/ratis-test/src/test/java/org/apache/ratis/server/impl/TestRaftServerJmx.java @@ -24,8 +24,9 @@ import org.apache.ratis.server.RaftServerMXBean; import org.apache.ratis.server.simulation.MiniRaftClusterWithSimulatedRpc; import org.apache.ratis.util.JmxRegister; -import org.junit.Assert; -import org.junit.Test; +import org.junit.jupiter.api.Assertions; +import org.junit.jupiter.api.Test; +import org.junit.jupiter.api.Timeout; import javax.management.JMException; import javax.management.MBeanServer; @@ -38,7 +39,8 @@ import static org.apache.ratis.RaftTestUtil.waitForLeader; public class TestRaftServerJmx extends BaseTest { - @Test(timeout = 30000) + @Test + @Timeout(value = 30000) public void testJmxBeans() throws Exception { final int NUM_SERVERS = 3; final MiniRaftClusterWithSimulatedRpc cluster @@ -48,16 +50,17 @@ public void testJmxBeans() throws Exception { MBeanServer platformMBeanServer = ManagementFactory.getPlatformMBeanServer(); Set objectInstances = platformMBeanServer.queryMBeans(new ObjectName("Ratis:*"), null); - Assert.assertEquals(NUM_SERVERS, objectInstances.size()); + Assertions.assertEquals(NUM_SERVERS, objectInstances.size()); for (ObjectInstance instance : objectInstances) { Object groupId = platformMBeanServer.getAttribute(instance.getObjectName(), "GroupId"); - Assert.assertEquals(cluster.getGroupId().toString(), groupId); + Assertions.assertEquals(cluster.getGroupId().toString(), groupId); } cluster.shutdown(); } - @Test(timeout = 30000) + @Test + @Timeout(value = 30000) public void testRegister() throws JMException { { final JmxRegister jmx = new JmxRegister(); @@ -102,11 +105,11 @@ static void runRegister(boolean expectToSucceed, String name, JmxRegister jmx) { final String id = RaftPeerId.valueOf(name).toString(); final String groupId = RaftGroupId.randomId().toString(); final boolean succeeded = RaftServerJmxAdapter.registerMBean(id, groupId, mBean, jmx); - Assert.assertEquals(expectToSucceed, succeeded); + Assertions.assertEquals(expectToSucceed, succeeded); } static void runUnregister(boolean expectToSucceed, JmxRegister jmx) throws JMException { final boolean succeeded = jmx.unregister(); - Assert.assertEquals(expectToSucceed, succeeded); + Assertions.assertEquals(expectToSucceed, succeeded); } } diff --git a/ratis-test/src/test/java/org/apache/ratis/server/raftlog/TestRaftLogIndex.java b/ratis-test/src/test/java/org/apache/ratis/server/raftlog/TestRaftLogIndex.java index ac655222ba..d3b4b5ccb0 100644 --- a/ratis-test/src/test/java/org/apache/ratis/server/raftlog/TestRaftLogIndex.java +++ b/ratis-test/src/test/java/org/apache/ratis/server/raftlog/TestRaftLogIndex.java @@ -18,8 +18,8 @@ package org.apache.ratis.server.raftlog; import org.apache.ratis.BaseTest; -import org.junit.Assert; -import org.junit.Test; +import org.junit.jupiter.api.Assertions; +import org.junit.jupiter.api.Test; import java.util.function.BiFunction; import java.util.function.Consumer; @@ -34,10 +34,10 @@ static void assertUpdate(RaftLogIndex index, BiFunction update, long oldValue, LongUnaryOperator op, boolean expectUpdate) { - Assert.assertEquals(oldValue, index.get()); + Assertions.assertEquals(oldValue, index.get()); final boolean updated = update.apply(index, op); - Assert.assertEquals(expectUpdate, updated); - Assert.assertEquals(expectUpdate? op.applyAsLong(oldValue): oldValue, index.get()); + Assertions.assertEquals(expectUpdate, updated); + Assertions.assertEquals(expectUpdate? op.applyAsLong(oldValue): oldValue, index.get()); } @@ -45,7 +45,7 @@ static void assertUpdate(RaftLogIndex index, BiFunction log = System.out::println; { // test updateIncreasingly diff --git a/ratis-test/src/test/java/org/apache/ratis/server/raftlog/TestRaftLogMetrics.java b/ratis-test/src/test/java/org/apache/ratis/server/raftlog/TestRaftLogMetrics.java index 70b185e554..65493d7a37 100644 --- a/ratis-test/src/test/java/org/apache/ratis/server/raftlog/TestRaftLogMetrics.java +++ b/ratis-test/src/test/java/org/apache/ratis/server/raftlog/TestRaftLogMetrics.java @@ -36,8 +36,8 @@ import org.apache.ratis.statemachine.impl.BaseStateMachine; import org.apache.ratis.util.JavaUtils; import org.apache.ratis.util.TimeDuration; -import org.junit.Assert; -import org.junit.Test; +import org.junit.jupiter.api.Assertions; +import org.junit.jupiter.api.Test; import javax.management.ObjectName; import java.lang.management.ManagementFactory; @@ -116,7 +116,7 @@ static void runTestRaftLogMetrics(MiniRaftCluster cluster) throws Exception { static void assertCommitCount(RaftServer.Division server, int expectedMsgs) { final RatisMetricRegistry rlm = ((RatisMetrics)server.getRaftLog().getRaftLogMetrics()).getRegistry(); long stmCount = rlm.counter(STATE_MACHINE_LOG_ENTRY_COUNT).getCount(); - Assert.assertEquals(expectedMsgs, stmCount); + Assertions.assertEquals(expectedMsgs, stmCount); } static RatisMetricRegistryImpl getRegistry(RaftGroupMemberId memberId) { @@ -127,21 +127,21 @@ static void assertFlushCount(RaftServer.Division server) throws Exception { final String flushTimeMetric = RaftStorageTestUtils.getLogFlushTimeMetric(server.getMemberId().toString()); final RatisMetricRegistryImpl ratisMetricRegistry = getRegistry(server.getMemberId()); Timer tm = (Timer) ratisMetricRegistry.get(RAFT_LOG_FLUSH_TIME); - Assert.assertNotNull(tm); + Assertions.assertNotNull(tm); final MetricsStateMachine stateMachine = MetricsStateMachine.get(server); final int expectedFlush = stateMachine.getFlushCount(); JavaUtils.attemptRepeatedly(() -> { - Assert.assertEquals(expectedFlush, tm.getCount()); + Assertions.assertEquals(expectedFlush, tm.getCount()); return null; }, 50, HUNDRED_MILLIS, "expectedFlush == tm.getCount()", null); - Assert.assertTrue(tm.getMeanRate() > 0); + Assertions.assertTrue(tm.getMeanRate() > 0); // Test jmx ObjectName oname = new ObjectName(RATIS_APPLICATION_NAME_METRICS, "name", flushTimeMetric); - Assert.assertEquals(expectedFlush, + Assertions.assertEquals(expectedFlush, ((Long) ManagementFactory.getPlatformMBeanServer().getAttribute(oname, "Count")) .intValue()); } @@ -152,42 +152,42 @@ static void assertRaftLogWritePathMetrics(RaftServer.Division server) throws Exc //Test sync count Timer tm = (Timer) ratisMetricRegistry.get(RAFT_LOG_SYNC_TIME); - Assert.assertNotNull(tm); + Assertions.assertNotNull(tm); final MetricsStateMachine stateMachine = MetricsStateMachine.get(server); final int expectedFlush = stateMachine.getFlushCount(); - Assert.assertEquals(expectedFlush, tm.getCount()); // Ideally, flushCount should be same as syncCount. - Assert.assertTrue(tm.getMeanRate() > 0); + Assertions.assertEquals(expectedFlush, tm.getCount()); // Ideally, flushCount should be same as syncCount. + Assertions.assertTrue(tm.getMeanRate() > 0); // Test jmx. Just testing one metric's JMX is good enough. ObjectName oname = new ObjectName(RATIS_APPLICATION_NAME_METRICS, "name", syncTimeMetric); - Assert.assertEquals(expectedFlush, + Assertions.assertEquals(expectedFlush, ((Long) ManagementFactory.getPlatformMBeanServer().getAttribute(oname, "Count")) .intValue()); long cacheMissCount = ratisMetricRegistry.counter(RAFT_LOG_CACHE_MISS_COUNT).getCount(); - Assert.assertEquals(0, cacheMissCount); + Assertions.assertEquals(0, cacheMissCount); long cacheHitsCount = ratisMetricRegistry.counter(RAFT_LOG_CACHE_HIT_COUNT).getCount(); - Assert.assertTrue(cacheHitsCount > 0); + Assertions.assertTrue(cacheHitsCount > 0); - Assert.assertTrue(ratisMetricRegistry.counter(RAFT_LOG_FLUSH_COUNT).getCount() > 0); - Assert.assertTrue(ratisMetricRegistry.counter(RAFT_LOG_APPEND_ENTRY_COUNT).getCount() > 0); + Assertions.assertTrue(ratisMetricRegistry.counter(RAFT_LOG_FLUSH_COUNT).getCount() > 0); + Assertions.assertTrue(ratisMetricRegistry.counter(RAFT_LOG_APPEND_ENTRY_COUNT).getCount() > 0); final DefaultTimekeeperImpl appendEntry = (DefaultTimekeeperImpl) ratisMetricRegistry.timer(RAFT_LOG_APPEND_ENTRY_LATENCY); - Assert.assertTrue(appendEntry.getTimer().getMeanRate() > 0); + Assertions.assertTrue(appendEntry.getTimer().getMeanRate() > 0); final DefaultTimekeeperImpl taskQueue = (DefaultTimekeeperImpl) ratisMetricRegistry.timer(RAFT_LOG_TASK_QUEUE_TIME); - Assert.assertTrue(taskQueue.getTimer().getMeanRate() > 0); + Assertions.assertTrue(taskQueue.getTimer().getMeanRate() > 0); final DefaultTimekeeperImpl enqueueDelay = (DefaultTimekeeperImpl) ratisMetricRegistry.timer(RAFT_LOG_TASK_ENQUEUE_DELAY); - Assert.assertTrue(enqueueDelay.getTimer().getMeanRate() > 0); + Assertions.assertTrue(enqueueDelay.getTimer().getMeanRate() > 0); final DefaultTimekeeperImpl write = (DefaultTimekeeperImpl) ratisMetricRegistry.timer( String.format(RAFT_LOG_TASK_EXECUTION_TIME, "writelog")); - Assert.assertTrue(write.getTimer().getMeanRate() > 0); + Assertions.assertTrue(write.getTimer().getMeanRate() > 0); - Assert.assertNotNull(ratisMetricRegistry.get(RAFT_LOG_DATA_QUEUE_SIZE)); - Assert.assertNotNull(ratisMetricRegistry.get(RAFT_LOG_WORKER_QUEUE_SIZE)); - Assert.assertNotNull(ratisMetricRegistry.get(RAFT_LOG_SYNC_BATCH_SIZE)); + Assertions.assertNotNull(ratisMetricRegistry.get(RAFT_LOG_DATA_QUEUE_SIZE)); + Assertions.assertNotNull(ratisMetricRegistry.get(RAFT_LOG_WORKER_QUEUE_SIZE)); + Assertions.assertNotNull(ratisMetricRegistry.get(RAFT_LOG_SYNC_BATCH_SIZE)); } } \ No newline at end of file diff --git a/ratis-test/src/test/java/org/apache/ratis/server/raftlog/memory/MemoryRaftLogTest.java b/ratis-test/src/test/java/org/apache/ratis/server/raftlog/memory/MemoryRaftLogTest.java index 5d8d090a38..5a41f9ed9a 100644 --- a/ratis-test/src/test/java/org/apache/ratis/server/raftlog/memory/MemoryRaftLogTest.java +++ b/ratis-test/src/test/java/org/apache/ratis/server/raftlog/memory/MemoryRaftLogTest.java @@ -17,7 +17,7 @@ */ package org.apache.ratis.server.raftlog.memory; -import static org.junit.Assert.assertEquals; +import static org.junit.jupiter.api.Assertions.assertEquals; import java.util.ArrayList; import java.util.List; @@ -34,7 +34,7 @@ import org.apache.ratis.statemachine.impl.SimpleStateMachine4Testing; import org.apache.ratis.statemachine.StateMachine; import org.apache.ratis.util.Slf4jUtils; -import org.junit.Test; +import org.junit.jupiter.api.Test; import org.slf4j.event.Level; public class MemoryRaftLogTest extends BaseTest { diff --git a/ratis-test/src/test/java/org/apache/ratis/server/raftlog/segmented/TestBufferedWriteChannel.java b/ratis-test/src/test/java/org/apache/ratis/server/raftlog/segmented/TestBufferedWriteChannel.java index c9d7928555..cefd5f6747 100644 --- a/ratis-test/src/test/java/org/apache/ratis/server/raftlog/segmented/TestBufferedWriteChannel.java +++ b/ratis-test/src/test/java/org/apache/ratis/server/raftlog/segmented/TestBufferedWriteChannel.java @@ -18,9 +18,8 @@ package org.apache.ratis.server.raftlog.segmented; import org.apache.ratis.BaseTest; -import org.apache.ratis.util.StringUtils; -import org.junit.Assert; -import org.junit.Test; +import org.junit.jupiter.api.Assertions; +import org.junit.jupiter.api.Test; import java.io.IOException; import java.nio.ByteBuffer; @@ -40,8 +39,8 @@ class FakeFileChannel extends FileChannel { private long forcedPosition = 0; void assertValues(long expectedPosition, long expectedForcedPosition) { - Assert.assertEquals(expectedPosition, position); - Assert.assertEquals(expectedForcedPosition, forcedPosition); + Assertions.assertEquals(expectedPosition, position); + Assertions.assertEquals(expectedForcedPosition, forcedPosition); } @Override @@ -200,7 +199,7 @@ static void flush(BufferedWriteChannel out, FakeFileChannel fake, AtomicInteger pos, AtomicInteger force) throws IOException { final int existing = out.writeBufferPosition(); out.flush(); - Assert.assertEquals(0, out.writeBufferPosition()); + Assertions.assertEquals(0, out.writeBufferPosition()); pos.addAndGet(existing); force.set(pos.get()); fake.assertValues(pos.get(), force.get()); @@ -213,9 +212,9 @@ static void writeToBuffer(BufferedWriteChannel out, FakeFileChannel fake, Atomic out.writeToBuffer(n, b -> b.put(buffer)); if (existing + n > bufferCapacity) { pos.addAndGet(existing); - Assert.assertEquals(n, out.writeBufferPosition()); + Assertions.assertEquals(n, out.writeBufferPosition()); } else { - Assert.assertEquals(existing + n, out.writeBufferPosition()); + Assertions.assertEquals(existing + n, out.writeBufferPosition()); } fake.assertValues(pos.get(), force.get()); } diff --git a/ratis-test/src/test/java/org/apache/ratis/server/raftlog/segmented/TestCacheEviction.java b/ratis-test/src/test/java/org/apache/ratis/server/raftlog/segmented/TestCacheEviction.java index 996f7ef527..dacccf8a58 100644 --- a/ratis-test/src/test/java/org/apache/ratis/server/raftlog/segmented/TestCacheEviction.java +++ b/ratis-test/src/test/java/org/apache/ratis/server/raftlog/segmented/TestCacheEviction.java @@ -39,8 +39,8 @@ import org.apache.ratis.statemachine.StateMachine; import org.apache.ratis.util.JavaUtils; import org.apache.ratis.util.SizeInBytes; -import org.junit.Assert; -import org.junit.Test; +import org.junit.jupiter.api.Assertions; +import org.junit.jupiter.api.Test; import org.mockito.Mockito; import java.io.File; @@ -55,7 +55,7 @@ public class TestCacheEviction extends BaseTest { private static final CacheInvalidationPolicy policy = new CacheInvalidationPolicyDefault(); static LogSegmentList prepareSegments(int numSegments, boolean[] cached, long start, long size) { - Assert.assertEquals(numSegments, cached.length); + Assertions.assertEquals(numSegments, cached.length); final LogSegmentList segments = new LogSegmentList(JavaUtils.getClassSimpleName(TestCacheEviction.class)); for (int i = 0; i < numSegments; i++) { LogSegment s = LogSegment.newCloseSegment(null, start, start + size - 1, MAX_OP_SIZE, null); @@ -77,35 +77,35 @@ public void testBasicEviction() throws Exception { // case 1, make sure we do not evict cache for segments behind local flushed index List evicted = policy.evict(null, 5, 15, segments, maxCached); - Assert.assertEquals(0, evicted.size()); + Assertions.assertEquals(0, evicted.size()); // case 2, suppose the local flushed index is in the 3rd segment, then we // can evict the first two segment evicted = policy.evict(null, 25, 30, segments, maxCached); - Assert.assertEquals(2, evicted.size()); - Assert.assertSame(evicted.get(0), segments.get(0)); - Assert.assertSame(evicted.get(1), segments.get(1)); + Assertions.assertEquals(2, evicted.size()); + Assertions.assertSame(evicted.get(0), segments.get(0)); + Assertions.assertSame(evicted.get(1), segments.get(1)); // case 3, similar with case 2, but the local applied index is less than // the local flushed index. evicted = policy.evict(null, 25, 15, segments, maxCached); - Assert.assertEquals(1, evicted.size()); - Assert.assertSame(evicted.get(0), segments.get(0)); + Assertions.assertEquals(1, evicted.size()); + Assertions.assertSame(evicted.get(0), segments.get(0)); // case 4, the local applied index is very small, then evict cache behind it // first and let the state machine load the segments later evicted = policy.evict(null, 35, 5, segments, maxCached); - Assert.assertEquals(1, evicted.size()); - Assert.assertSame(evicted.get(0), segments.get(2)); + Assertions.assertEquals(1, evicted.size()); + Assertions.assertSame(evicted.get(0), segments.get(2)); Mockito.when(segments.get(2).hasCache()).thenReturn(false); evicted = policy.evict(null, 35, 5, segments, maxCached); - Assert.assertEquals(1, evicted.size()); - Assert.assertSame(evicted.get(0), segments.get(1)); + Assertions.assertEquals(1, evicted.size()); + Assertions.assertSame(evicted.get(0), segments.get(1)); Mockito.when(segments.get(1).hasCache()).thenReturn(false); evicted = policy.evict(null, 35, 5, segments, maxCached); - Assert.assertEquals(0, evicted.size()); + Assertions.assertEquals(0, evicted.size()); } @Test @@ -118,37 +118,37 @@ public void testEvictionWithFollowerIndices() throws Exception { // flushed index List evicted = policy.evict(new long[]{20, 40, 40}, 5, 15, segments, maxCached); - Assert.assertEquals(0, evicted.size()); + Assertions.assertEquals(0, evicted.size()); // case 2, the follower indices are behind the local flushed index evicted = policy.evict(new long[]{30, 40, 45}, 25, 30, segments, maxCached); - Assert.assertEquals(2, evicted.size()); - Assert.assertSame(evicted.get(0), segments.get(0)); - Assert.assertSame(evicted.get(1), segments.get(1)); + Assertions.assertEquals(2, evicted.size()); + Assertions.assertSame(evicted.get(0), segments.get(0)); + Assertions.assertSame(evicted.get(1), segments.get(1)); // case 3, similar with case 3 in basic eviction test evicted = policy.evict(new long[]{30, 40, 45}, 25, 15, segments, maxCached); - Assert.assertEquals(1, evicted.size()); - Assert.assertSame(evicted.get(0), segments.get(0)); + Assertions.assertEquals(1, evicted.size()); + Assertions.assertSame(evicted.get(0), segments.get(0)); // case 4, the followers are slower than local flush evicted = policy.evict(new long[]{15, 45, 45}, 55, 50, segments, maxCached); - Assert.assertEquals(1, evicted.size()); - Assert.assertSame(evicted.get(0), segments.get(0)); + Assertions.assertEquals(1, evicted.size()); + Assertions.assertSame(evicted.get(0), segments.get(0)); Mockito.when(segments.get(0).hasCache()).thenReturn(false); evicted = policy.evict(new long[]{15, 45, 45}, 55, 50, segments, maxCached); - Assert.assertEquals(1, evicted.size()); - Assert.assertSame(evicted.get(0), segments.get(2)); + Assertions.assertEquals(1, evicted.size()); + Assertions.assertSame(evicted.get(0), segments.get(2)); Mockito.when(segments.get(2).hasCache()).thenReturn(false); evicted = policy.evict(new long[]{15, 45, 45}, 55, 50, segments, maxCached); - Assert.assertEquals(1, evicted.size()); - Assert.assertSame(evicted.get(0), segments.get(3)); + Assertions.assertEquals(1, evicted.size()); + Assertions.assertSame(evicted.get(0), segments.get(3)); Mockito.when(segments.get(3).hasCache()).thenReturn(false); evicted = policy.evict(new long[]{15, 45, 45}, 55, 50, segments, maxCached); - Assert.assertEquals(0, evicted.size()); + Assertions.assertEquals(0, evicted.size()); } @Test @@ -177,7 +177,7 @@ public void testEvictionInSegmentedLog() throws Exception { raftLog.append(entries).forEach(CompletableFuture::join); // check the current cached segment number: the last segment is still open - Assert.assertEquals(maxCachedNum - 1, + Assertions.assertEquals(maxCachedNum - 1, raftLog.getRaftLogCache().getCachedSegmentNum()); Mockito.when(info.getLastAppliedIndex()).thenReturn(35L); @@ -188,7 +188,7 @@ public void testEvictionInSegmentedLog() throws Exception { // check the cached segment number again. since the slowest follower is on // index 21, the eviction should happen and evict 3 segments - Assert.assertEquals(maxCachedNum + 1 - 3, + Assertions.assertEquals(maxCachedNum + 1 - 3, raftLog.getRaftLogCache().getCachedSegmentNum()); } diff --git a/ratis-test/src/test/java/org/apache/ratis/server/raftlog/segmented/TestLogSegment.java b/ratis-test/src/test/java/org/apache/ratis/server/raftlog/segmented/TestLogSegment.java index ece17a0524..8355c67333 100644 --- a/ratis-test/src/test/java/org/apache/ratis/server/raftlog/segmented/TestLogSegment.java +++ b/ratis-test/src/test/java/org/apache/ratis/server/raftlog/segmented/TestLogSegment.java @@ -37,10 +37,10 @@ import org.apache.ratis.util.ReferenceCountedObject; import org.apache.ratis.util.SizeInBytes; import org.apache.ratis.util.TraditionalBinaryPrefix; -import org.junit.After; -import org.junit.Assert; -import org.junit.Before; -import org.junit.Test; +import org.junit.jupiter.api.AfterEach; +import org.junit.jupiter.api.Assertions; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.Test; import java.io.File; import java.io.IOException; @@ -65,7 +65,7 @@ public class TestLogSegment extends BaseTest { private long preallocatedSize; private int bufferSize; - @Before + @BeforeEach public void setup() { RaftProperties properties = new RaftProperties(); storageDir = getTestDir(); @@ -78,7 +78,7 @@ public void setup() { RaftServerConfigKeys.Log.writeBufferSize(properties).getSizeInt(); } - @After + @AfterEach public void tearDown() throws Exception { if (storageDir != null) { FileUtils.deleteFully(storageDir.getParentFile()); @@ -110,11 +110,11 @@ File prepareLog(boolean isOpen, long startIndex, int numEntries, long term, bool // 0 < truncatedEntrySize < entrySize final long fileLength = file.length(); final long truncatedFileLength = fileLength - (entrySize - truncatedEntrySize); - Assert.assertTrue(truncatedFileLength < fileLength); + Assertions.assertTrue(truncatedFileLength < fileLength); LOG.info("truncate last entry: entry(size={}, truncated={}), file(length={}, truncated={})", entrySize, truncatedEntrySize, fileLength, truncatedFileLength); FileUtils.truncateFile(file, truncatedFileLength); - Assert.assertEquals(truncatedFileLength, file.length()); + Assertions.assertEquals(truncatedFileLength, file.length()); } storage.close(); @@ -128,18 +128,18 @@ static int size(LogEntryProto entry) { static void checkLogSegment(LogSegment segment, long start, long end, boolean isOpen, long totalSize, long term) throws Exception { - Assert.assertEquals(start, segment.getStartIndex()); - Assert.assertEquals(end, segment.getEndIndex()); - Assert.assertEquals(isOpen, segment.isOpen()); - Assert.assertEquals(totalSize, segment.getTotalFileSize()); + Assertions.assertEquals(start, segment.getStartIndex()); + Assertions.assertEquals(end, segment.getEndIndex()); + Assertions.assertEquals(isOpen, segment.isOpen()); + Assertions.assertEquals(totalSize, segment.getTotalFileSize()); long offset = SegmentedRaftLogFormat.getHeaderLength(); for (long i = start; i <= end; i++) { LogSegment.LogRecord record = segment.getLogRecord(i); final TermIndex ti = record.getTermIndex(); - Assert.assertEquals(i, ti.getIndex()); - Assert.assertEquals(term, ti.getTerm()); - Assert.assertEquals(offset, record.getOffset()); + Assertions.assertEquals(i, ti.getIndex()); + Assertions.assertEquals(term, ti.getTerm()); + Assertions.assertEquals(offset, record.getOffset()); LogEntryProto entry = segment.getEntryFromCache(ti); if (entry == null) { @@ -179,7 +179,7 @@ private void testLoadSegment(boolean loadInitial, boolean isLastEntryPartiallyWr checkLogSegment(openSegment, 0, 99 - delta, true, openSegmentFile.length(), 0); storage.close(); // for open segment we currently always keep log entries in the memory - Assert.assertEquals(0, openSegment.getLoadingTimes()); + Assertions.assertEquals(0, openSegment.getLoadingTimes()); // load a closed segment (1000-1099) final File closedSegmentFile = prepareLog(false, 1000, 100, 1, false); @@ -187,7 +187,7 @@ private void testLoadSegment(boolean loadInitial, boolean isLastEntryPartiallyWr LogSegmentStartEnd.valueOf(1000, 1099L), MAX_OP_SIZE, loadInitial, null, null); checkLogSegment(closedSegment, 1000, 1099, false, closedSegment.getTotalFileSize(), 1); - Assert.assertEquals(loadInitial ? 0 : 1, closedSegment.getLoadingTimes()); + Assertions.assertEquals(loadInitial ? 0 : 1, closedSegment.getLoadingTimes()); } @Test @@ -208,7 +208,7 @@ public void testAppendEntries() throws Exception { segment.appendToOpenSegment(Op.WRITE_CACHE_WITHOUT_STATE_MACHINE_CACHE, ReferenceCountedObject.wrap(entry)); } - Assert.assertTrue(segment.getTotalFileSize() >= max); + Assertions.assertTrue(segment.getTotalFileSize() >= max); checkLogSegment(segment, start, i - 1 + start, true, size, term); } @@ -224,9 +224,9 @@ public void testAppendEntryMetric() throws Exception { storage.close(); final DefaultTimekeeperImpl readEntryTimer = (DefaultTimekeeperImpl) raftLogMetrics.getReadEntryTimer(); - Assert.assertNotNull(readEntryTimer); - Assert.assertEquals(100, readEntryTimer.getTimer().getCount()); - Assert.assertTrue(readEntryTimer.getTimer().getMeanRate() > 0); + Assertions.assertNotNull(readEntryTimer); + Assertions.assertEquals(100, readEntryTimer.getTimer().getCount()); + Assertions.assertTrue(readEntryTimer.getTimer().getMeanRate() > 0); } @@ -238,7 +238,7 @@ public void testAppendWithGap() throws Exception { try { LogEntryProto entry = LogProtoUtils.toLogEntryProto(m, 0, 1001); segment.appendToOpenSegment(Op.WRITE_CACHE_WITHOUT_STATE_MACHINE_CACHE, ReferenceCountedObject.wrap(entry)); - Assert.fail("should fail since the entry's index needs to be 1000"); + Assertions.fail("should fail since the entry's index needs to be 1000"); } catch (IllegalStateException e) { // the exception is expected. } @@ -249,7 +249,7 @@ public void testAppendWithGap() throws Exception { try { entry = LogProtoUtils.toLogEntryProto(m, 0, 1002); segment.appendToOpenSegment(Op.WRITE_CACHE_WITHOUT_STATE_MACHINE_CACHE, ReferenceCountedObject.wrap(entry)); - Assert.fail("should fail since the entry's index needs to be 1001"); + Assertions.fail("should fail since the entry's index needs to be 1001"); } catch (IllegalStateException e) { // the exception is expected. } @@ -269,18 +269,18 @@ public void testTruncate() throws Exception { // truncate an open segment (remove 1080~1099) long newSize = segment.getLogRecord(start + 80).getOffset(); segment.truncate(start + 80); - Assert.assertEquals(80, segment.numOfEntries()); + Assertions.assertEquals(80, segment.numOfEntries()); checkLogSegment(segment, start, start + 79, false, newSize, term); // truncate a closed segment (remove 1050~1079) newSize = segment.getLogRecord(start + 50).getOffset(); segment.truncate(start + 50); - Assert.assertEquals(50, segment.numOfEntries()); + Assertions.assertEquals(50, segment.numOfEntries()); checkLogSegment(segment, start, start + 49, false, newSize, term); // truncate all the remaining entries segment.truncate(start); - Assert.assertEquals(0, segment.numOfEntries()); + Assertions.assertEquals(0, segment.numOfEntries()); checkLogSegment(segment, start, start - 1, false, SegmentedRaftLogFormat.getHeaderLength(), term); } @@ -300,12 +300,12 @@ public void testPreallocateSegment() throws Exception { for (int a : preallocated) { try(SegmentedRaftLogOutputStream ignored = new SegmentedRaftLogOutputStream(file, false, max, a, ByteBuffer.allocateDirect(bufferSize))) { - Assert.assertEquals("max=" + max + ", a=" + a, file.length(), Math.min(max, a)); + Assertions.assertEquals(file.length(), Math.min(max, a), "max=" + max + ", a=" + a); } try(SegmentedRaftLogInputStream in = SegmentedRaftLogTestUtils.newSegmentedRaftLogInputStream( file, 0, INVALID_LOG_INDEX, true)) { LogEntryProto entry = in.nextEntry(); - Assert.assertNull(entry); + Assertions.assertNull(entry); } } } @@ -321,14 +321,14 @@ public void testPreallocateSegment() throws Exception { size = LogSegment.getEntrySize(entry, Op.WRITE_CACHE_WITHOUT_STATE_MACHINE_CACHE); out.write(entry); } - Assert.assertEquals(file.length(), + Assertions.assertEquals(file.length(), size + SegmentedRaftLogFormat.getHeaderLength()); try (SegmentedRaftLogInputStream in = SegmentedRaftLogTestUtils.newSegmentedRaftLogInputStream( file, 0, INVALID_LOG_INDEX, true)) { LogEntryProto entry = in.nextEntry(); - Assert.assertArrayEquals(content, + Assertions.assertArrayEquals(content, entry.getStateMachineLogEntry().getLogData().toByteArray()); - Assert.assertNull(in.nextEntry()); + Assertions.assertNull(in.nextEntry()); } } @@ -351,19 +351,19 @@ public void testPreallocationAndAppend() throws Exception { long preallocated = 16 * 1024; try (SegmentedRaftLogOutputStream out = new SegmentedRaftLogOutputStream(file, false, max.getSize(), 16 * 1024, ByteBuffer.allocateDirect(10 * 1024))) { - Assert.assertEquals(preallocated, file.length()); + Assertions.assertEquals(preallocated, file.length()); while (totalSize + entrySize < max.getSize()) { totalSize += entrySize; out.write(entry); if (totalSize > preallocated) { - Assert.assertEquals("totalSize==" + totalSize, - preallocated + 16 * 1024, file.length()); + Assertions.assertEquals(preallocated + 16 * 1024, file.length(), + "totalSize==" + totalSize); preallocated += 16 * 1024; } } } - Assert.assertEquals(totalSize, file.length()); + Assertions.assertEquals(totalSize, file.length()); } @Test @@ -374,14 +374,14 @@ public void testZeroSizeInProgressFile() throws Exception { // create zero size in-progress file LOG.info("file: " + file); - Assert.assertTrue(file.createNewFile()); + Assertions.assertTrue(file.createNewFile()); final Path path = file.toPath(); - Assert.assertTrue(Files.exists(path)); - Assert.assertEquals(0, Files.size(path)); + Assertions.assertTrue(Files.exists(path)); + Assertions.assertEquals(0, Files.size(path)); // getLogSegmentPaths should remove it. final List logs = LogSegmentPath.getLogSegmentPaths(storage); - Assert.assertEquals(0, logs.size()); - Assert.assertFalse(Files.exists(path)); + Assertions.assertEquals(0, logs.size()); + Assertions.assertFalse(Files.exists(path)); } } diff --git a/ratis-test/src/test/java/org/apache/ratis/server/raftlog/segmented/TestRaftLogReadWrite.java b/ratis-test/src/test/java/org/apache/ratis/server/raftlog/segmented/TestRaftLogReadWrite.java index a020b43bd8..4f151c733f 100644 --- a/ratis-test/src/test/java/org/apache/ratis/server/raftlog/segmented/TestRaftLogReadWrite.java +++ b/ratis-test/src/test/java/org/apache/ratis/server/raftlog/segmented/TestRaftLogReadWrite.java @@ -29,10 +29,10 @@ import org.apache.ratis.thirdparty.com.google.protobuf.CodedOutputStream; import org.apache.ratis.proto.RaftProtos.LogEntryProto; import org.apache.ratis.util.FileUtils; -import org.junit.After; -import org.junit.Assert; -import org.junit.Before; -import org.junit.Test; +import org.junit.jupiter.api.AfterEach; +import org.junit.jupiter.api.Assertions; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.Test; import java.io.File; import java.io.IOException; @@ -53,7 +53,7 @@ public class TestRaftLogReadWrite extends BaseTest { private long preallocatedSize; private int bufferSize; - @Before + @BeforeEach public void setup() { storageDir = getTestDir(); RaftProperties properties = new RaftProperties(); @@ -66,7 +66,7 @@ public void setup() { RaftServerConfigKeys.Log.writeBufferSize(properties).getSizeInt(); } - @After + @AfterEach public void tearDown() throws Exception { if (storageDir != null) { FileUtils.deleteFully(storageDir.getParentFile()); @@ -116,10 +116,10 @@ public void testReadWriteLog() throws IOException { storage.close(); } - Assert.assertEquals(size, openSegment.length()); + Assertions.assertEquals(size, openSegment.length()); final LogEntryProto[] readEntries = readLog(openSegment, 0, RaftLog.INVALID_LOG_INDEX, true); - Assert.assertArrayEquals(entries, readEntries); + Assertions.assertArrayEquals(entries, readEntries); } @Test @@ -146,7 +146,7 @@ public void testAppendLog() throws IOException { } final LogEntryProto[] readEntries = readLog(openSegment, 0, RaftLog.INVALID_LOG_INDEX, true); - Assert.assertArrayEquals(entries, readEntries); + Assertions.assertArrayEquals(entries, readEntries); storage.close(); } @@ -168,16 +168,16 @@ public void testReadWithPadding() throws IOException { out.flush(); // make sure the file contains padding - Assert.assertEquals( + Assertions.assertEquals( RaftServerConfigKeys.Log.PREALLOCATED_SIZE_DEFAULT.getSize(), openSegment.length()); // check if the reader can correctly read the log file final LogEntryProto[] readEntries = readLog(openSegment, 0, RaftLog.INVALID_LOG_INDEX, true); - Assert.assertArrayEquals(entries, readEntries); + Assertions.assertArrayEquals(entries, readEntries); out.close(); - Assert.assertEquals(size, openSegment.length()); + Assertions.assertEquals(size, openSegment.length()); } /** @@ -200,12 +200,12 @@ public void testReadWithCorruptPadding() throws IOException { out.flush(); // make sure the file contains padding - Assert.assertEquals(4 * 1024 * 1024, openSegment.length()); + Assertions.assertEquals(4 * 1024 * 1024, openSegment.length()); try (FileChannel fout = FileUtils.newFileChannel(openSegment, StandardOpenOption.WRITE)) { final byte[] array = {-1, 1}; final int written = fout.write(ByteBuffer.wrap(array), 16 * 1024 * 1024 - 10); - Assert.assertEquals(array.length, written); + Assertions.assertEquals(array.length, written); } List list = new ArrayList<>(); @@ -215,7 +215,7 @@ public void testReadWithCorruptPadding() throws IOException { while ((entry = in.nextEntry()) != null) { list.add(entry); } - Assert.fail("should fail since we corrupt the padding"); + Assertions.fail("should fail since we corrupt the padding"); } catch (IOException e) { boolean findVerifyTerminator = false; for (StackTraceElement s : e.getStackTrace()) { @@ -224,9 +224,9 @@ public void testReadWithCorruptPadding() throws IOException { break; } } - Assert.assertTrue(findVerifyTerminator); + Assertions.assertTrue(findVerifyTerminator); } - Assert.assertArrayEquals(entries, + Assertions.assertArrayEquals(entries, list.toArray(new LogEntryProto[list.size()])); } @@ -259,7 +259,7 @@ public void testReadWithEntryCorruption() throws IOException { try { readLog(openSegment, 0, RaftLog.INVALID_LOG_INDEX, true); - Assert.fail("The read of corrupted log file should fail"); + Assertions.fail("The read of corrupted log file should fail"); } catch (ChecksumException e) { LOG.info("Caught ChecksumException as expected", e); } diff --git a/ratis-test/src/test/java/org/apache/ratis/server/raftlog/segmented/TestSegmentedRaftLog.java b/ratis-test/src/test/java/org/apache/ratis/server/raftlog/segmented/TestSegmentedRaftLog.java index 38fa45e6fa..8d6c97d815 100644 --- a/ratis-test/src/test/java/org/apache/ratis/server/raftlog/segmented/TestSegmentedRaftLog.java +++ b/ratis-test/src/test/java/org/apache/ratis/server/raftlog/segmented/TestSegmentedRaftLog.java @@ -17,8 +17,6 @@ */ package org.apache.ratis.server.raftlog.segmented; -import static org.junit.Assert.assertTrue; - import org.apache.ratis.BaseTest; import org.apache.ratis.RaftTestUtil.SimpleOperation; import org.apache.ratis.conf.RaftProperties; @@ -48,18 +46,12 @@ import org.apache.ratis.util.JavaUtils; import org.apache.ratis.util.SizeInBytes; import org.apache.ratis.util.TimeDuration; -import org.junit.After; -import org.junit.Assert; -import org.junit.Before; -import org.junit.Test; import java.io.File; import java.io.IOException; import java.nio.ByteBuffer; -import java.nio.charset.StandardCharsets; import java.util.ArrayList; import java.util.Arrays; -import java.util.Collection; import java.util.Collections; import java.util.List; import java.util.concurrent.CompletableFuture; @@ -67,12 +59,22 @@ import java.util.concurrent.TimeUnit; import java.util.function.LongSupplier; import java.util.function.Supplier; - -import org.junit.runner.RunWith; -import org.junit.runners.Parameterized; +import java.util.stream.Stream; + +import org.junit.jupiter.api.AfterEach; +import org.junit.jupiter.api.Assertions; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.Test; +import org.junit.jupiter.params.ParameterizedTest; +import org.junit.jupiter.params.provider.Arguments; +import org.junit.jupiter.params.provider.MethodSource; import org.slf4j.event.Level; -@RunWith(Parameterized.class) +import static java.lang.Boolean.FALSE; +import static java.lang.Boolean.TRUE; +import static org.junit.jupiter.api.Assertions.assertTrue; +import static org.junit.jupiter.params.provider.Arguments.arguments; + public class TestSegmentedRaftLog extends BaseTest { static { Slf4jUtils.setLogLevel(SegmentedRaftLogWorker.LOG, Level.INFO); @@ -80,18 +82,12 @@ public class TestSegmentedRaftLog extends BaseTest { Slf4jUtils.setLogLevel(SegmentedRaftLog.LOG, Level.INFO); } - private final Boolean smSyncFlush; - private final Boolean useAsyncFlush; - - public TestSegmentedRaftLog(Boolean raftLogAsync, Boolean smSync) { - this.useAsyncFlush = raftLogAsync; - this.smSyncFlush = smSync; - } - - @Parameterized.Parameters - public static Collection data() { - return Arrays.asList((new Boolean[][] {{Boolean.FALSE, Boolean.FALSE}, {Boolean.FALSE, Boolean.TRUE}, - {Boolean.TRUE, Boolean.FALSE}, {Boolean.TRUE, Boolean.TRUE}})); + public static Stream data() { + return Stream.of( + arguments(FALSE, FALSE), + arguments(FALSE, TRUE), + arguments(TRUE, FALSE), + arguments(TRUE, TRUE)); } public static long getOpenSegmentSize(RaftLog raftLog) { @@ -153,13 +149,11 @@ private SegmentedRaftLog newSegmentedRaftLogWithSnapshotIndex(RaftStorage storag .build(); } - @Before + @BeforeEach public void setup() throws Exception { storageDir = getTestDir(); properties = new RaftProperties(); RaftServerConfigKeys.setStorageDir(properties, Collections.singletonList(storageDir)); - RaftServerConfigKeys.Log.setAsyncFlushEnabled(properties, useAsyncFlush); - RaftServerConfigKeys.Log.StateMachineData.setSync(properties, smSyncFlush); storage = RaftStorageTestUtils.newRaftStorage(storageDir); this.segmentMaxSize = RaftServerConfigKeys.Log.segmentSizeMax(properties).getSize(); @@ -169,7 +163,7 @@ public void setup() throws Exception { RaftServerConfigKeys.Log.writeBufferSize(properties).getSizeInt(); } - @After + @AfterEach public void tearDown() throws Exception { if (storageDir != null) { FileUtils.deleteFully(storageDir.getParentFile()); @@ -211,8 +205,11 @@ private LogEntryProto getLastEntry(SegmentedRaftLog raftLog) return raftLog.get(raftLog.getLastEntryTermIndex().getIndex()); } - @Test - public void testLoadLogSegments() throws Exception { + @ParameterizedTest + @MethodSource("data") + public void testLoadLogSegments(Boolean useAsyncFlush, Boolean smSyncFlush) throws Exception { + RaftServerConfigKeys.Log.setAsyncFlushEnabled(properties, useAsyncFlush); + RaftServerConfigKeys.Log.StateMachineData.setSync(properties, smSyncFlush); // first generate log files List ranges = prepareRanges(0, 5, 100, 0); LogEntryProto[] entries = prepareLog(ranges); @@ -223,7 +220,7 @@ public void testLoadLogSegments() throws Exception { // check if log entries are loaded correctly for (LogEntryProto e : entries) { LogEntryProto entry = raftLog.get(e.getIndex()); - Assert.assertEquals(e, entry); + Assertions.assertEquals(e, entry); } final LogEntryHeader[] termIndices = raftLog.getEntries(0, 500); @@ -236,8 +233,8 @@ public void testLoadLogSegments() throws Exception { } }) .toArray(LogEntryProto[]::new); - Assert.assertArrayEquals(entries, entriesFromLog); - Assert.assertEquals(entries[entries.length - 1], getLastEntry(raftLog)); + Assertions.assertArrayEquals(entries, entriesFromLog); + Assertions.assertEquals(entries[entries.length - 1], getLastEntry(raftLog)); final RatisMetricRegistry metricRegistryForLogWorker = RaftLogMetricsBase.createRegistry(memberId); @@ -276,8 +273,11 @@ static LogEntryProto prepareLogEntry(long term, long index, Supplier str /** * Append entry one by one and check if log state is correct. */ - @Test - public void testAppendEntry() throws Exception { + @ParameterizedTest + @MethodSource("data") + public void testAppendEntry(Boolean useAsyncFlush, Boolean smSyncFlush) throws Exception { + RaftServerConfigKeys.Log.setAsyncFlushEnabled(properties, useAsyncFlush); + RaftServerConfigKeys.Log.StateMachineData.setSync(properties, smSyncFlush); List ranges = prepareRanges(0, 5, 200, 0); List entries = prepareLogEntries(ranges, null); @@ -331,8 +331,11 @@ public void testAppendEntry() throws Exception { } } - @Test - public void testAppendEntryAfterPurge() throws Exception { + @ParameterizedTest + @MethodSource("data") + public void testAppendEntryAfterPurge(Boolean useAsyncFlush, Boolean smSyncFlush) throws Exception { + RaftServerConfigKeys.Log.setAsyncFlushEnabled(properties, useAsyncFlush); + RaftServerConfigKeys.Log.StateMachineData.setSync(properties, smSyncFlush); List ranges = prepareRanges(0, 5, 200, 0); List entries = prepareLogEntries(ranges, null); @@ -362,8 +365,11 @@ public long getAsLong() { /** * Keep appending entries, make sure the rolling is correct. */ - @Test - public void testAppendAndRoll() throws Exception { + @ParameterizedTest + @MethodSource("data") + public void testAppendAndRoll(Boolean useAsyncFlush, Boolean smSyncFlush) throws Exception { + RaftServerConfigKeys.Log.setAsyncFlushEnabled(properties, useAsyncFlush); + RaftServerConfigKeys.Log.StateMachineData.setSync(properties, smSyncFlush); RaftServerConfigKeys.Log.setPreallocatedSize(properties, SizeInBytes.valueOf("16KB")); RaftServerConfigKeys.Log.setSegmentSizeMax(properties, SizeInBytes.valueOf("128KB")); @@ -382,12 +388,15 @@ public void testAppendAndRoll() throws Exception { raftLog.open(RaftLog.INVALID_LOG_INDEX, null); // check if the raft log is correct checkEntries(raftLog, entries, 0, entries.size()); - Assert.assertEquals(9, raftLog.getRaftLogCache().getNumOfSegments()); + Assertions.assertEquals(9, raftLog.getRaftLogCache().getNumOfSegments()); } } - @Test - public void testTruncate() throws Exception { + @ParameterizedTest + @MethodSource("data") + public void testTruncate(Boolean useAsyncFlush, Boolean smSyncFlush) throws Exception { + RaftServerConfigKeys.Log.setAsyncFlushEnabled(properties, useAsyncFlush); + RaftServerConfigKeys.Log.StateMachineData.setSync(properties, smSyncFlush); // prepare the log for truncation List ranges = prepareRanges(0, 5, 200, 0); List entries = prepareLogEntries(ranges, null); @@ -418,10 +427,10 @@ private void testTruncate(List entries, long fromIndex) raftLog.open(RaftLog.INVALID_LOG_INDEX, null); // check if the raft log is correct if (fromIndex > 0) { - Assert.assertEquals(entries.get((int) (fromIndex - 1)), + Assertions.assertEquals(entries.get((int) (fromIndex - 1)), getLastEntry(raftLog)); } else { - Assert.assertNull(raftLog.getLastEntryTermIndex()); + Assertions.assertNull(raftLog.getLastEntryTermIndex()); } checkEntries(raftLog, entries, 0, (int) fromIndex); } @@ -432,7 +441,7 @@ private void checkEntries(RaftLog raftLog, List expected, if (size > 0) { for (int i = offset; i < size + offset; i++) { LogEntryProto entry = raftLog.get(expected.get(i).getIndex()); - Assert.assertEquals(expected.get(i), entry); + Assertions.assertEquals(expected.get(i), entry); } final LogEntryHeader[] termIndices = raftLog.getEntries( expected.get(offset).getIndex(), @@ -448,7 +457,7 @@ private void checkEntries(RaftLog raftLog, List expected, .toArray(LogEntryProto[]::new); LogEntryProto[] expectedArray = expected.subList(offset, offset + size) .stream().toArray(LogEntryProto[]::new); - Assert.assertArrayEquals(expectedArray, entriesFromLog); + Assertions.assertArrayEquals(expectedArray, entriesFromLog); } } @@ -517,15 +526,18 @@ private void purgeAndVerify(int startTerm, int endTerm, int segmentSize, int pur final CompletableFuture f = raftLog.purge(purgeIndex); final Long purged = f.get(); LOG.info("purgeIndex = {}, purged = {}", purgeIndex, purged); - Assert.assertEquals(expectedIndex, raftLog.getRaftLogCache().getStartIndex()); + Assertions.assertEquals(expectedIndex, raftLog.getRaftLogCache().getStartIndex()); } } /** * Test append with inconsistent entries */ - @Test - public void testAppendEntriesWithInconsistency() throws Exception { + @ParameterizedTest + @MethodSource("data") + public void testAppendEntriesWithInconsistency(Boolean useAsyncFlush, Boolean smSyncFlush) throws Exception { + RaftServerConfigKeys.Log.setAsyncFlushEnabled(properties, useAsyncFlush); + RaftServerConfigKeys.Log.StateMachineData.setSync(properties, smSyncFlush); // prepare the log for truncation List ranges = prepareRanges(0, 5, 200, 0); List entries = prepareLogEntries(ranges, null); @@ -556,9 +568,9 @@ public void testAppendEntriesWithInconsistency() throws Exception { checkFailedEntries(entries, 650, retryCache); checkEntries(raftLog, entries, 0, 650); checkEntries(raftLog, newEntries, 100, 100); - Assert.assertEquals(newEntries.get(newEntries.size() - 1), + Assertions.assertEquals(newEntries.get(newEntries.size() - 1), getLastEntry(raftLog)); - Assert.assertEquals(newEntries.get(newEntries.size() - 1).getIndex(), + Assertions.assertEquals(newEntries.get(newEntries.size() - 1).getIndex(), raftLog.getFlushIndex()); } @@ -567,18 +579,21 @@ public void testAppendEntriesWithInconsistency() throws Exception { raftLog.open(RaftLog.INVALID_LOG_INDEX, null); checkEntries(raftLog, entries, 0, 650); checkEntries(raftLog, newEntries, 100, 100); - Assert.assertEquals(newEntries.get(newEntries.size() - 1), + Assertions.assertEquals(newEntries.get(newEntries.size() - 1), getLastEntry(raftLog)); - Assert.assertEquals(newEntries.get(newEntries.size() - 1).getIndex(), + Assertions.assertEquals(newEntries.get(newEntries.size() - 1).getIndex(), raftLog.getFlushIndex()); SegmentedRaftLogCache cache = raftLog.getRaftLogCache(); - Assert.assertEquals(5, cache.getNumOfSegments()); + Assertions.assertEquals(5, cache.getNumOfSegments()); } } - @Test - public void testSegmentedRaftLogStateMachineData() throws Exception { + @ParameterizedTest + @MethodSource("data") + public void testSegmentedRaftLogStateMachineData(Boolean useAsyncFlush, Boolean smSyncFlush) throws Exception { + RaftServerConfigKeys.Log.setAsyncFlushEnabled(properties, useAsyncFlush); + RaftServerConfigKeys.Log.StateMachineData.setSync(properties, smSyncFlush); final SegmentRange range = new SegmentRange(0, 10, 1, true); final List entries = prepareLogEntries(range, null, true, new ArrayList<>()); @@ -622,8 +637,11 @@ public void testSegmentedRaftLogStateMachineData() throws Exception { } } - @Test(expected = TimeoutIOException.class) - public void testServerShutdownOnTimeoutIOException() throws Throwable { + @ParameterizedTest + @MethodSource("data") + public void testServerShutdownOnTimeoutIOException(Boolean useAsyncFlush, Boolean smSyncFlush) throws Throwable { + RaftServerConfigKeys.Log.setAsyncFlushEnabled(properties, useAsyncFlush); + RaftServerConfigKeys.Log.StateMachineData.setSync(properties, smSyncFlush); RaftServerConfigKeys.Log.StateMachineData.setSync(properties, true); final TimeDuration syncTimeout = TimeDuration.valueOf(100, TimeUnit.MILLISECONDS); RaftServerConfigKeys.Log.StateMachineData.setSyncTimeout(properties, syncTimeout); @@ -645,14 +663,14 @@ public void notifyLogFailed(Throwable cause, LogEntryProto entry) { LOG.info("Test StateMachine: Ratis log failed notification received as expected.", cause); LOG.info("Test StateMachine: Transition to PAUSED state."); - Assert.assertNotNull(entry); + Assertions.assertNotNull(entry); getLifeCycle().transition(LifeCycle.State.PAUSING); getLifeCycle().transition(LifeCycle.State.PAUSED); } }; - Throwable ex = null; // TimeoutIOException + ExecutionException ex; try (SegmentedRaftLog raftLog = SegmentedRaftLog.newBuilder() .setMemberId(memberId) .setStateMachine(sm) @@ -663,15 +681,10 @@ public void notifyLogFailed(Throwable cause, LogEntryProto entry) { // SegmentedRaftLogWorker should catch TimeoutIOException CompletableFuture f = raftLog.appendEntry(entry); // Wait for async writeStateMachineData to finish - try { - f.get(); - } catch (ExecutionException e) { - ex = e.getCause(); - } + ex = Assertions.assertThrows(ExecutionException.class, f::get); } - Assert.assertNotNull(ex); - Assert.assertSame(LifeCycle.State.PAUSED, sm.getLifeCycleState()); - throw ex; + Assertions.assertSame(LifeCycle.State.PAUSED, sm.getLifeCycleState()); + Assertions.assertInstanceOf(TimeoutIOException.class, ex.getCause()); } static Thread startAppendEntryThread(RaftLog raftLog, LogEntryProto entry) { @@ -688,9 +701,9 @@ static Thread startAppendEntryThread(RaftLog raftLog, LogEntryProto entry) { void assertIndices(RaftLog raftLog, long expectedFlushIndex, long expectedNextIndex) { LOG.info("assert expectedFlushIndex={}", expectedFlushIndex); - Assert.assertEquals(expectedFlushIndex, raftLog.getFlushIndex()); + Assertions.assertEquals(expectedFlushIndex, raftLog.getFlushIndex()); LOG.info("assert expectedNextIndex={}", expectedNextIndex); - Assert.assertEquals(expectedNextIndex, raftLog.getNextIndex()); + Assertions.assertEquals(expectedNextIndex, raftLog.getNextIndex()); } void assertIndicesMultipleAttempts(RaftLog raftLog, long expectedFlushIndex, long expectedNextIndex) throws Exception { @@ -698,8 +711,11 @@ void assertIndicesMultipleAttempts(RaftLog raftLog, long expectedFlushIndex, lon 10, HUNDRED_MILLIS, "assertIndices", LOG); } - @Test - public void testAsyncFlushPerf1() throws Exception { + @ParameterizedTest + @MethodSource("data") + public void testAsyncFlushPerf1(Boolean useAsyncFlush, Boolean smSyncFlush) throws Exception { + RaftServerConfigKeys.Log.setAsyncFlushEnabled(properties, useAsyncFlush); + RaftServerConfigKeys.Log.StateMachineData.setSync(properties, smSyncFlush); List ranges = prepareRanges(0, 50, 20000, 0); List entries = prepareLogEntries(ranges, null); @@ -721,8 +737,11 @@ public void testAsyncFlushPerf1() throws Exception { } } - @Test - public void testAsyncFlushPerf2() throws Exception { + @ParameterizedTest + @MethodSource("data") + public void testAsyncFlushPerf2(Boolean useAsyncFlush, Boolean smSyncFlush) throws Exception { + RaftServerConfigKeys.Log.setAsyncFlushEnabled(properties, useAsyncFlush); + RaftServerConfigKeys.Log.StateMachineData.setSync(properties, smSyncFlush); List ranges = prepareRanges(0, 50, 20000, 0); List entries = prepareLogEntries(ranges, null); diff --git a/ratis-test/src/test/java/org/apache/ratis/server/raftlog/segmented/TestSegmentedRaftLogCache.java b/ratis-test/src/test/java/org/apache/ratis/server/raftlog/segmented/TestSegmentedRaftLogCache.java index 5be3c36578..fa892b2003 100644 --- a/ratis-test/src/test/java/org/apache/ratis/server/raftlog/segmented/TestSegmentedRaftLogCache.java +++ b/ratis-test/src/test/java/org/apache/ratis/server/raftlog/segmented/TestSegmentedRaftLogCache.java @@ -37,10 +37,10 @@ import org.apache.ratis.server.raftlog.segmented.LogSegment.Op; import org.apache.ratis.proto.RaftProtos.LogEntryProto; import org.apache.ratis.util.ReferenceCountedObject; -import org.junit.After; -import org.junit.Assert; -import org.junit.Before; -import org.junit.Test; +import org.junit.jupiter.api.AfterEach; +import org.junit.jupiter.api.Assertions; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.Test; public class TestSegmentedRaftLogCache { private static final RaftProperties prop = new RaftProperties(); @@ -49,14 +49,14 @@ public class TestSegmentedRaftLogCache { private SegmentedRaftLogMetrics raftLogMetrics; private RatisMetricRegistryImpl ratisMetricRegistry; - @Before + @BeforeEach public void setup() { raftLogMetrics = new SegmentedRaftLogMetrics(RaftServerTestUtil.TEST_MEMBER_ID); ratisMetricRegistry = (RatisMetricRegistryImpl) raftLogMetrics.getRegistry(); cache = new SegmentedRaftLogCache(null, null, prop, raftLogMetrics); } - @After + @AfterEach public void clear() { raftLogMetrics.unregister(); } @@ -75,14 +75,14 @@ private LogSegment prepareLogSegment(long start, long end, boolean isOpen) { } private void checkCache(long start, long end, int segmentSize) { - Assert.assertEquals(start, cache.getStartIndex()); - Assert.assertEquals(end, cache.getEndIndex()); + Assertions.assertEquals(start, cache.getStartIndex()); + Assertions.assertEquals(end, cache.getEndIndex()); for (long index = start; index <= end; index++) { final LogSegment segment = cache.getSegment(index); final LogRecord record = segment.getLogRecord(index); final LogEntryProto entry = segment.getEntryFromCache(record.getTermIndex()); - Assert.assertEquals(index, entry.getIndex()); + Assertions.assertEquals(index, entry.getIndex()); } long[] offsets = new long[]{start, start + 1, start + (end - start) / 2, @@ -99,9 +99,9 @@ private void checkCache(long start, long end, int segmentSize) { private void checkCacheEntries(long offset, int size, long end) { final LogEntryHeader[] entries = cache.getTermIndices(offset, offset + size); long realEnd = offset + size > end + 1 ? end + 1 : offset + size; - Assert.assertEquals(realEnd - offset, entries.length); + Assertions.assertEquals(realEnd - offset, entries.length); for (long i = offset; i < realEnd; i++) { - Assert.assertEquals(i, entries[(int) (i - offset)].getIndex()); + Assertions.assertEquals(i, entries[(int) (i - offset)].getIndex()); } } @@ -114,7 +114,7 @@ public void testAddSegments() throws Exception { try { LogSegment s = prepareLogSegment(102, 103, true); cache.addSegment(s); - Assert.fail("should fail since there is gap between two segments"); + Assertions.fail("should fail since there is gap between two segments"); } catch (IllegalStateException ignored) { } @@ -125,7 +125,7 @@ public void testAddSegments() throws Exception { try { LogSegment s = prepareLogSegment(201, 202, true); cache.addSegment(s); - Assert.fail("should fail since there is still an open segment in cache"); + Assertions.fail("should fail since there is still an open segment in cache"); } catch (IllegalStateException ignored) { } @@ -135,17 +135,17 @@ public void testAddSegments() throws Exception { try { LogSegment s = prepareLogSegment(202, 203, true); cache.addSegment(s); - Assert.fail("should fail since there is gap between two segments"); + Assertions.fail("should fail since there is gap between two segments"); } catch (IllegalStateException ignored) { } LogSegment s3 = prepareLogSegment(201, 300, true); cache.addSegment(s3); - Assert.assertNotNull(cache.getOpenSegment()); + Assertions.assertNotNull(cache.getOpenSegment()); checkCache(1, 300, 100); cache.rollOpenSegment(true); - Assert.assertNotNull(cache.getOpenSegment()); + Assertions.assertNotNull(cache.getOpenSegment()); checkCache(1, 300, 100); } @@ -159,7 +159,7 @@ public void testAppendEntry() { LogEntryProto entry = LogProtoUtils.toLogEntryProto(m.getLogEntryContent(), 0, 0); cache.appendEntry(Op.WRITE_CACHE_WITHOUT_STATE_MACHINE_CACHE, ReferenceCountedObject.wrap(entry) ); - Assert.fail("the open segment is null"); + Assertions.fail("the open segment is null"); } catch (IllegalStateException ignored) { } @@ -171,7 +171,7 @@ public void testAppendEntry() { ); } - Assert.assertNotNull(cache.getOpenSegment()); + Assertions.assertNotNull(cache.getOpenSegment()); checkCache(0, 199, 100); } @@ -188,7 +188,7 @@ public void testTruncate() throws Exception { cache.addSegment(s); long end = cache.getEndIndex(); - Assert.assertEquals(599, end); + Assertions.assertEquals(599, end); int numOfSegments = 6; // start truncation for (int i = 0; i < 10; i++) { // truncate 10 times @@ -200,46 +200,46 @@ public void testTruncate() throws Exception { // check TruncationSegments int currentNum= (int) (end / 100 + 1); if (currentNum < numOfSegments) { - Assert.assertEquals(1, ts.getToDelete().length); + Assertions.assertEquals(1, ts.getToDelete().length); numOfSegments = currentNum; } else { - Assert.assertEquals(0, ts.getToDelete().length); + Assertions.assertEquals(0, ts.getToDelete().length); } } // 230 entries remaining. truncate at the segment boundary TruncationSegments ts = cache.truncate(200); checkCache(0, 199, 100); - Assert.assertEquals(1, ts.getToDelete().length); - Assert.assertEquals(200, ts.getToDelete()[0].getStartIndex()); - Assert.assertEquals(229, ts.getToDelete()[0].getEndIndex()); - Assert.assertEquals(0, ts.getToDelete()[0].getTargetLength()); - Assert.assertFalse(ts.getToDelete()[0].isOpen()); - Assert.assertNull(ts.getToTruncate()); + Assertions.assertEquals(1, ts.getToDelete().length); + Assertions.assertEquals(200, ts.getToDelete()[0].getStartIndex()); + Assertions.assertEquals(229, ts.getToDelete()[0].getEndIndex()); + Assertions.assertEquals(0, ts.getToDelete()[0].getTargetLength()); + Assertions.assertFalse(ts.getToDelete()[0].isOpen()); + Assertions.assertNull(ts.getToTruncate()); // add another open segment and truncate it as a whole LogSegment newOpen = prepareLogSegment(200, 249, true); cache.addSegment(newOpen); ts = cache.truncate(200); checkCache(0, 199, 100); - Assert.assertEquals(1, ts.getToDelete().length); - Assert.assertEquals(200, ts.getToDelete()[0].getStartIndex()); - Assert.assertEquals(249, ts.getToDelete()[0].getEndIndex()); - Assert.assertEquals(0, ts.getToDelete()[0].getTargetLength()); - Assert.assertTrue(ts.getToDelete()[0].isOpen()); - Assert.assertNull(ts.getToTruncate()); + Assertions.assertEquals(1, ts.getToDelete().length); + Assertions.assertEquals(200, ts.getToDelete()[0].getStartIndex()); + Assertions.assertEquals(249, ts.getToDelete()[0].getEndIndex()); + Assertions.assertEquals(0, ts.getToDelete()[0].getTargetLength()); + Assertions.assertTrue(ts.getToDelete()[0].isOpen()); + Assertions.assertNull(ts.getToTruncate()); // add another open segment and truncate part of it newOpen = prepareLogSegment(200, 249, true); cache.addSegment(newOpen); ts = cache.truncate(220); checkCache(0, 219, 100); - Assert.assertNull(cache.getOpenSegment()); - Assert.assertEquals(0, ts.getToDelete().length); - Assert.assertTrue(ts.getToTruncate().isOpen()); - Assert.assertEquals(219, ts.getToTruncate().getNewEndIndex()); - Assert.assertEquals(200, ts.getToTruncate().getStartIndex()); - Assert.assertEquals(249, ts.getToTruncate().getEndIndex()); + Assertions.assertNull(cache.getOpenSegment()); + Assertions.assertEquals(0, ts.getToDelete().length); + Assertions.assertTrue(ts.getToTruncate().isOpen()); + Assertions.assertEquals(219, ts.getToTruncate().getNewEndIndex()); + Assertions.assertEquals(200, ts.getToTruncate().getStartIndex()); + Assertions.assertEquals(249, ts.getToTruncate().getEndIndex()); } @Test @@ -255,9 +255,9 @@ public void testOpenSegmentPurge() { int purgeIndex = sIndex; // open segment should never be purged TruncationSegments ts = cache.purge(purgeIndex); - Assert.assertNull(ts.getToTruncate()); - Assert.assertEquals(end - start, ts.getToDelete().length); - Assert.assertEquals(sIndex, cache.getStartIndex()); + Assertions.assertNull(ts.getToTruncate()); + Assertions.assertEquals(end - start, ts.getToDelete().length); + Assertions.assertEquals(sIndex, cache.getStartIndex()); } @Test @@ -272,9 +272,9 @@ public void testCloseSegmentPurge() { // overlapped close segment will not purged. Passing in index - 1 since // we purge a closed segment when end index == passed in purge index. TruncationSegments ts = cache.purge(purgeIndex - 1); - Assert.assertNull(ts.getToTruncate()); - Assert.assertEquals(end - start - 1, ts.getToDelete().length); - Assert.assertEquals(1, cache.getNumOfSegments()); + Assertions.assertNull(ts.getToTruncate()); + Assertions.assertEquals(end - start - 1, ts.getToDelete().length); + Assertions.assertEquals(1, cache.getNumOfSegments()); } private void populatedSegment(int start, int end, int segmentSize, boolean isOpen) { @@ -291,15 +291,15 @@ private void testIterator(long startIndex) throws IOException { TermIndex prev = null; while (iterator.hasNext()) { TermIndex termIndex = iterator.next(); - Assert.assertEquals(cache.getLogRecord(termIndex.getIndex()).getTermIndex(), termIndex); + Assertions.assertEquals(cache.getLogRecord(termIndex.getIndex()).getTermIndex(), termIndex); if (prev != null) { - Assert.assertEquals(prev.getIndex() + 1, termIndex.getIndex()); + Assertions.assertEquals(prev.getIndex() + 1, termIndex.getIndex()); } prev = termIndex; } if (startIndex <= cache.getEndIndex()) { - Assert.assertNotNull(prev); - Assert.assertEquals(cache.getEndIndex(), prev.getIndex()); + Assertions.assertNotNull(prev); + Assertions.assertEquals(cache.getEndIndex(), prev.getIndex()); } } @@ -321,7 +321,7 @@ public void testIterator() throws Exception { testIterator(299); Iterator iterator = cache.iterator(300); - Assert.assertFalse(iterator.hasNext()); + Assertions.assertFalse(iterator.hasNext()); } @Test @@ -332,14 +332,14 @@ public void testCacheMetric() { Long closedSegmentsNum = (Long) ratisMetricRegistry.getGauges((s, metric) -> s.contains(RAFT_LOG_CACHE_CLOSED_SEGMENTS_NUM)).values().iterator().next().getValue(); - Assert.assertEquals(2L, closedSegmentsNum.longValue()); + Assertions.assertEquals(2L, closedSegmentsNum.longValue()); Long closedSegmentsSizeInBytes = (Long) ratisMetricRegistry.getGauges((s, metric) -> s.contains(RAFT_LOG_CACHE_CLOSED_SEGMENTS_SIZE_IN_BYTES)).values().iterator().next().getValue(); - Assert.assertEquals(closedSegmentsSizeInBytes.longValue(), cache.getClosedSegmentsSizeInBytes()); + Assertions.assertEquals(closedSegmentsSizeInBytes.longValue(), cache.getClosedSegmentsSizeInBytes()); Long openSegmentSizeInBytes = (Long) ratisMetricRegistry.getGauges((s, metric) -> s.contains(RAFT_LOG_CACHE_OPEN_SEGMENT_SIZE_IN_BYTES)).values().iterator().next().getValue(); - Assert.assertEquals(openSegmentSizeInBytes.longValue(), cache.getOpenSegmentSizeInBytes()); + Assertions.assertEquals(openSegmentSizeInBytes.longValue(), cache.getOpenSegmentSizeInBytes()); } } diff --git a/ratis-test/src/test/java/org/apache/ratis/server/storage/TestRaftStorage.java b/ratis-test/src/test/java/org/apache/ratis/server/storage/TestRaftStorage.java index 6e7555703f..eb65f342e3 100644 --- a/ratis-test/src/test/java/org/apache/ratis/server/storage/TestRaftStorage.java +++ b/ratis-test/src/test/java/org/apache/ratis/server/storage/TestRaftStorage.java @@ -32,10 +32,10 @@ import org.apache.ratis.util.FileUtils; import org.apache.ratis.util.Preconditions; import org.apache.ratis.util.SizeInBytes; -import org.junit.After; -import org.junit.Assert; -import org.junit.Before; -import org.junit.Test; +import org.junit.jupiter.api.AfterEach; +import org.junit.jupiter.api.Assertions; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.Test; import org.mockito.Mockito; import java.io.File; @@ -65,12 +65,12 @@ static RaftStorageImpl newRaftStorage(File dir) throws IOException { private File storageDir; - @Before + @BeforeEach public void setup() { storageDir = getTestDir(); } - @After + @AfterEach public void tearDown() throws Exception { if (storageDir != null) { FileUtils.deleteFully(storageDir.getParentFile()); @@ -93,22 +93,22 @@ public void testNotExistent() throws IOException { // we will format the empty directory final RaftStorageImpl storage = newRaftStorage(storageDir); - Assert.assertEquals(StorageState.NORMAL, storage.getState()); + Assertions.assertEquals(StorageState.NORMAL, storage.getState()); try { formatRaftStorage(storageDir).close(); - Assert.fail("the format should fail since the storage is still locked"); + Assertions.fail("the format should fail since the storage is still locked"); } catch (IOException e) { - Assert.assertTrue(e.getMessage().contains("directory is already locked")); + Assertions.assertTrue(e.getMessage().contains("directory is already locked")); } storage.close(); FileUtils.deleteFully(storageDir); - Assert.assertTrue(storageDir.createNewFile()); + Assertions.assertTrue(storageDir.createNewFile()); try (RaftStorage ignored = newRaftStorage(storageDir)) { - Assert.fail(); + Assertions.fail(); } catch (IOException e) { - Assert.assertTrue( + Assertions.assertTrue( e.getMessage().contains(StorageState.NON_EXISTENT.name())); } } @@ -121,42 +121,42 @@ public void testStorage() throws Exception { final RaftStorageDirectoryImpl sd = new RaftStorageDirectoryImpl(storageDir, SizeInBytes.ZERO); try { StorageState state = sd.analyzeStorage(true); - Assert.assertEquals(StorageState.NOT_FORMATTED, state); - Assert.assertTrue(sd.isCurrentEmpty()); + Assertions.assertEquals(StorageState.NOT_FORMATTED, state); + Assertions.assertTrue(sd.isCurrentEmpty()); } finally { sd.unlock(); } RaftStorageImpl storage = newRaftStorage(storageDir); - Assert.assertEquals(StorageState.NORMAL, storage.getState()); + Assertions.assertEquals(StorageState.NORMAL, storage.getState()); storage.close(); - Assert.assertEquals(StorageState.NORMAL, sd.analyzeStorage(false)); + Assertions.assertEquals(StorageState.NORMAL, sd.analyzeStorage(false)); assertMetadataFile(sd.getMetaFile()); // test format storage = formatRaftStorage(storageDir); - Assert.assertEquals(StorageState.NORMAL, storage.getState()); + Assertions.assertEquals(StorageState.NORMAL, storage.getState()); final RaftStorageMetadataFile metaFile = new RaftStorageMetadataFileImpl(sd.getMetaFile()); - Assert.assertEquals(RaftStorageMetadata.getDefault(), metaFile.getMetadata()); + Assertions.assertEquals(RaftStorageMetadata.getDefault(), metaFile.getMetadata()); storage.close(); } static void assertMetadataFile(File m) throws Exception { - Assert.assertTrue(m.exists()); + Assertions.assertTrue(m.exists()); final RaftStorageMetadataFile metaFile = new RaftStorageMetadataFileImpl(m); - Assert.assertEquals(RaftStorageMetadata.getDefault(), metaFile.getMetadata()); + Assertions.assertEquals(RaftStorageMetadata.getDefault(), metaFile.getMetadata()); final RaftPeerId peer1 = RaftPeerId.valueOf("peer1"); final RaftStorageMetadata metadata = RaftStorageMetadata.valueOf(123, peer1); metaFile.persist(metadata); - Assert.assertEquals(metadata.getTerm(), 123); - Assert.assertEquals(metadata.getVotedFor(), peer1); - Assert.assertEquals(metadata, metaFile.getMetadata()); + Assertions.assertEquals(metadata.getTerm(), 123); + Assertions.assertEquals(metadata.getVotedFor(), peer1); + Assertions.assertEquals(metadata, metaFile.getMetadata()); final RaftStorageMetadataFile metaFile2 = new RaftStorageMetadataFileImpl(m); - Assert.assertNull(((AtomicReference) RaftTestUtil.getDeclaredField(metaFile2, "metadata")).get()); - Assert.assertEquals(metadata, metaFile2.getMetadata()); + Assertions.assertNull(((AtomicReference) RaftTestUtil.getDeclaredField(metaFile2, "metadata")).get()); + Assertions.assertEquals(metadata, metaFile2.getMetadata()); } @Test @@ -172,27 +172,27 @@ public void testMetaFile() throws Exception { @Test public void testCleanMetaTmpFile() throws Exception { RaftStorageImpl storage = newRaftStorage(storageDir); - Assert.assertEquals(StorageState.NORMAL, storage.getState()); + Assertions.assertEquals(StorageState.NORMAL, storage.getState()); storage.close(); final RaftStorageDirectoryImpl sd = new RaftStorageDirectoryImpl(storageDir, SizeInBytes.ZERO); File metaFile = sd.getMetaFile(); FileUtils.move(metaFile, sd.getMetaTmpFile()); - Assert.assertEquals(StorageState.NOT_FORMATTED, sd.analyzeStorage(false)); + Assertions.assertEquals(StorageState.NOT_FORMATTED, sd.analyzeStorage(false)); // RaftStorage initialization should succeed as the raft-meta.tmp is // always cleaned. newRaftStorage(storageDir).close(); - Assert.assertTrue(sd.getMetaFile().exists()); - Assert.assertTrue(sd.getMetaTmpFile().createNewFile()); - Assert.assertTrue(sd.getMetaTmpFile().exists()); + Assertions.assertTrue(sd.getMetaFile().exists()); + Assertions.assertTrue(sd.getMetaTmpFile().createNewFile()); + Assertions.assertTrue(sd.getMetaTmpFile().exists()); try { storage = newRaftStorage(storageDir); - Assert.assertEquals(StorageState.NORMAL, storage.getState()); - Assert.assertFalse(sd.getMetaTmpFile().exists()); - Assert.assertTrue(sd.getMetaFile().exists()); + Assertions.assertEquals(StorageState.NORMAL, storage.getState()); + Assertions.assertFalse(sd.getMetaTmpFile().exists()); + Assertions.assertTrue(sd.getMetaFile().exists()); } finally { storage.close(); } @@ -207,14 +207,14 @@ public void testSnapshotFileName() { final File file = new File(storageDir, name); final TermIndex ti = SimpleStateMachineStorage.getTermIndexFromSnapshotFile(file); System.out.println("file = " + file); - Assert.assertEquals(term, ti.getTerm()); - Assert.assertEquals(index, ti.getIndex()); + Assertions.assertEquals(term, ti.getTerm()); + Assertions.assertEquals(index, ti.getIndex()); System.out.println("ti = " + ti); final File foo = new File(storageDir, "foo"); try { SimpleStateMachineStorage.getTermIndexFromSnapshotFile(foo); - Assert.fail(); + Assertions.fail(); } catch(IllegalArgumentException iae) { System.out.println("Good " + iae); } @@ -244,7 +244,7 @@ public int getNumSnapshotsRetained() { final long index = ThreadLocalRandom.current().nextLong(100, 1000L); if (termIndexSet.add(TermIndex.valueOf(term, index))) { File file = simpleStateMachineStorage.getSnapshotFile(term, index); - Assert.assertTrue(file.createNewFile()); + Assertions.assertTrue(file.createNewFile()); } } // create MD5 files that will not be deleted in older version @@ -254,7 +254,7 @@ public int getNumSnapshotsRetained() { if (termIndexSet.add(TermIndex.valueOf(term, index))) { File file = simpleStateMachineStorage.getSnapshotFile(term, index); File snapshotFile = new File(file.getParent(), file.getName() + MD5_SUFFIX); - Assert.assertTrue(snapshotFile.createNewFile()); + Assertions.assertTrue(snapshotFile.createNewFile()); } } @@ -274,7 +274,7 @@ public int getNumSnapshotsRetained() { System.out.println(file.getName()); Matcher matcher = SNAPSHOT_REGEX.matcher(file.getName()); if (matcher.matches()) { - Assert.assertTrue(remainingIndices.contains(Long.parseLong(matcher.group(2)))); + Assertions.assertTrue(remainingIndices.contains(Long.parseLong(matcher.group(2)))); } } @@ -288,7 +288,7 @@ public int getNumSnapshotsRetained() { final long term = ThreadLocalRandom.current().nextLong(1, 10L); final long index = ThreadLocalRandom.current().nextLong(1000L); File file = simpleStateMachineStorage.getSnapshotFile(term, index); - Assert.assertTrue(file.createNewFile()); + Assertions.assertTrue(file.createNewFile()); } simpleStateMachineStorage.cleanupOldSnapshots(new SnapshotRetentionPolicy() { }); @@ -297,8 +297,8 @@ public int getNumSnapshotsRetained() { private static File[] assertFileCount(File dir, int expected) { File[] files = dir.listFiles(); - Assert.assertNotNull(files); - Assert.assertEquals(Arrays.toString(files), expected, files.length); + Assertions.assertNotNull(files); + Assertions.assertEquals(expected, files.length, Arrays.toString(files)); return files; } @@ -309,6 +309,6 @@ public void testNotEnoughSpace() throws IOException { final RaftStorageDirectoryImpl sd = new RaftStorageDirectoryImpl(mockStorageDir, SizeInBytes.valueOf("100M")); StorageState state = sd.analyzeStorage(false); - Assert.assertEquals(StorageState.NO_SPACE, state); + Assertions.assertEquals(StorageState.NO_SPACE, state); } } diff --git a/ratis-test/src/test/java/org/apache/ratis/server/storage/TestStorageImplUtils.java b/ratis-test/src/test/java/org/apache/ratis/server/storage/TestStorageImplUtils.java index e8661e2689..4f75f793fc 100644 --- a/ratis-test/src/test/java/org/apache/ratis/server/storage/TestStorageImplUtils.java +++ b/ratis-test/src/test/java/org/apache/ratis/server/storage/TestStorageImplUtils.java @@ -22,9 +22,9 @@ import org.apache.ratis.server.RaftServerConfigKeys; import org.apache.ratis.util.FileUtils; import org.apache.ratis.util.JavaUtils; -import org.junit.AfterClass; -import org.junit.Assert; -import org.junit.Test; +import org.junit.jupiter.api.AfterAll; +import org.junit.jupiter.api.Assertions; +import org.junit.jupiter.api.Test; import java.io.File; import java.io.IOException; @@ -55,7 +55,7 @@ static File chooseNewStorageDir(List volumes, String sub) throws IOExcepti return new File(vol, sub); } - @AfterClass + @AfterAll public static void tearDown() throws IOException { FileUtils.deleteFully(rootTestDir.get()); } @@ -75,9 +75,9 @@ public void testChooseStorageDirWithOneVolume() throws IOException { final File storageDirTwo = chooseNewStorageDir(directories, subDirTwo); File expectedOne = new File(testDir, subDirOne); File expectedTwo = new File(testDir, subDirTwo); - Assert.assertEquals(expectedOne.getCanonicalPath(), + Assertions.assertEquals(expectedOne.getCanonicalPath(), storageDirOne.getCanonicalPath()); - Assert.assertEquals(expectedTwo.getCanonicalPath(), + Assertions.assertEquals(expectedTwo.getCanonicalPath(), storageDirTwo.getCanonicalPath()); } @@ -113,7 +113,7 @@ public void testChooseStorageDirWithMultipleVolumes() throws IOException { String subDir = UUID.randomUUID().toString(); final File storageDirectory = chooseNewStorageDir(directories, subDir); File expected = new File(directories.get(6), subDir); - Assert.assertEquals(expected.getCanonicalPath(), + Assertions.assertEquals(expected.getCanonicalPath(), storageDirectory.getCanonicalPath()); } @@ -124,10 +124,10 @@ public void testChooseStorageDirWithMultipleVolumes() throws IOException { public void testChooseStorageDirWithNoVolume() { try { chooseNewStorageDir(Collections.emptyList(), UUID.randomUUID().toString()); - Assert.fail(); + Assertions.fail(); } catch (IOException ex) { String expectedErrMsg = "No storage directory found."; - Assert.assertEquals(expectedErrMsg, ex.getMessage()); + Assertions.assertEquals(expectedErrMsg, ex.getMessage()); } } @@ -144,7 +144,7 @@ public void testAutoFormatSingleDirectory() throws Exception { final RaftStorageImpl storage = StorageImplUtils.initRaftStorage( "group-1", RaftStorage.StartupOption.RECOVER, properties); - Assert.assertNotNull(storage); + Assertions.assertNotNull(storage); storage.close(); } @@ -172,6 +172,6 @@ public void testAutoFormatMultiDirectories() throws Exception { () -> StorageImplUtils.initRaftStorage( "group-1", RaftStorage.StartupOption.RECOVER, properties), IOException.class); - Assert.assertTrue(ioe.getMessage().contains("Failed to RECOVER: Storage directory not found")); + Assertions.assertTrue(ioe.getMessage().contains("Failed to RECOVER: Storage directory not found")); } } \ No newline at end of file diff --git a/ratis-test/src/test/java/org/apache/ratis/shell/cli/sh/ElectionCommandIntegrationTest.java b/ratis-test/src/test/java/org/apache/ratis/shell/cli/sh/ElectionCommandIntegrationTest.java index e708153aa5..a61bb19f73 100644 --- a/ratis-test/src/test/java/org/apache/ratis/shell/cli/sh/ElectionCommandIntegrationTest.java +++ b/ratis-test/src/test/java/org/apache/ratis/shell/cli/sh/ElectionCommandIntegrationTest.java @@ -30,8 +30,8 @@ import org.apache.ratis.util.Slf4jUtils; import org.apache.ratis.util.SizeInBytes; import org.apache.ratis.util.TimeDuration; -import org.junit.Assert; -import org.junit.Test; +import org.junit.jupiter.api.Assertions; +import org.junit.jupiter.api.Test; import org.slf4j.event.Level; import java.util.concurrent.TimeUnit; @@ -63,13 +63,13 @@ void runTestElectionTransferCommand(MiniRaftCluster cluster) throws Exception { RaftServer.Division newLeader = cluster.getFollowers().get(0); final StringPrintStream out = new StringPrintStream(); RatisShell shell = new RatisShell(out.getPrintStream()); - Assert.assertNotEquals(cluster.getLeader().getId(), newLeader.getId()); + Assertions.assertNotEquals(cluster.getLeader().getId(), newLeader.getId()); int ret = shell.run("election", "transfer", "-peers", address, "-address", newLeader.getPeer().getAddress()); - Assert.assertEquals(0, ret); + Assertions.assertEquals(0, ret); JavaUtils.attempt(() -> { - Assert.assertEquals(cluster.getLeader().getId(), newLeader.getId()); + Assertions.assertEquals(cluster.getLeader().getId(), newLeader.getId()); }, 10, TimeDuration.valueOf(1, TimeUnit.SECONDS), "testElectionTransferCommand", LOG); } @@ -85,28 +85,28 @@ void runTestElectionTransferCommandToHigherPriority(MiniRaftCluster cluster) thr RaftServer.Division newLeader = cluster.getFollowers().get(0); final StringPrintStream out = new StringPrintStream(); RatisShell shell = new RatisShell(out.getPrintStream()); - Assert.assertTrue(cluster.getFollowers().contains(newLeader)); + Assertions.assertTrue(cluster.getFollowers().contains(newLeader)); // set current leader's priority to 2 int ret = shell.run("peer", "setPriority", "-peers", address, "-addressPriority", leader.getPeer().getAddress()+ "|" + 2); - Assert.assertEquals(0, ret); + Assertions.assertEquals(0, ret); // transfer to new leader will set its priority to 2 (with timeout 1s) ret = shell.run("election", "transfer", "-peers", address, "-address", newLeader.getPeer().getAddress(), "-timeout", "1"); - Assert.assertEquals(0, ret); + Assertions.assertEquals(0, ret); - JavaUtils.attempt(() -> Assert.assertEquals(cluster.getLeader().getId(), newLeader.getId()), + JavaUtils.attempt(() -> Assertions.assertEquals(cluster.getLeader().getId(), newLeader.getId()), 10, TimeDuration.valueOf(1, TimeUnit.SECONDS), "testElectionTransferLeaderCommand", LOG); // verify that priorities of new leader and old leader are both 2 ret = shell.run("group", "info", "-peers", address); - Assert.assertEquals(0 , ret); + Assertions.assertEquals(0 , ret); String expected = String.format("\"%s\"%n priority: %d", newLeader.getPeer().getAddress(), 2); String expected2 = String.format("\"%s\"%n priority: %d", leader.getPeer().getAddress(), 2); - Assert.assertTrue(out.toString().contains(expected)); - Assert.assertTrue(out.toString().contains(expected2)); + Assertions.assertTrue(out.toString().contains(expected)); + Assertions.assertTrue(out.toString().contains(expected2)); } @Test @@ -120,25 +120,25 @@ void runTestElectionPauseResumeCommand(MiniRaftCluster cluster) throws Exception RaftServer.Division newLeader = cluster.getFollowers().get(0); final StringPrintStream out = new StringPrintStream(); RatisShell shell = new RatisShell(out.getPrintStream()); - Assert.assertNotEquals(cluster.getLeader().getId(), newLeader.getId()); + Assertions.assertNotEquals(cluster.getLeader().getId(), newLeader.getId()); int ret = shell.run("election", "pause", "-peers", sb.toString(), "-address", newLeader.getPeer().getAddress()); - Assert.assertEquals(0, ret); + Assertions.assertEquals(0, ret); ret = shell.run("peer", "setPriority", "-peers", sb.toString(), "-addressPriority", newLeader.getPeer().getAddress() + "|" + 2); - Assert.assertEquals(0, ret); + Assertions.assertEquals(0, ret); JavaUtils.attempt(() -> { - Assert.assertNotEquals(cluster.getLeader().getId(), newLeader.getId()); + Assertions.assertNotEquals(cluster.getLeader().getId(), newLeader.getId()); }, 10, TimeDuration.valueOf(1, TimeUnit.SECONDS), "testElectionPauseResumeCommand", LOG); ret = shell.run("election", "resume", "-peers", sb.toString(), "-address", newLeader.getPeer().getAddress()); - Assert.assertEquals(0, ret); + Assertions.assertEquals(0, ret); JavaUtils.attempt(() -> { - Assert.assertEquals(cluster.getLeader().getId(), newLeader.getId()); + Assertions.assertEquals(cluster.getLeader().getId(), newLeader.getId()); }, 10, TimeDuration.valueOf(1, TimeUnit.SECONDS), "testElectionPauseResumeCommand", LOG); } @@ -153,10 +153,10 @@ void runTestElectionStepDownCommand(MiniRaftCluster cluster) throws Exception { RaftServer.Division newLeader = cluster.getFollowers().get(0); final StringPrintStream out = new StringPrintStream(); RatisShell shell = new RatisShell(out.getPrintStream()); - Assert.assertNotEquals(cluster.getLeader().getId(), newLeader.getId()); - Assert.assertEquals(2, cluster.getFollowers().size()); + Assertions.assertNotEquals(cluster.getLeader().getId(), newLeader.getId()); + Assertions.assertEquals(2, cluster.getFollowers().size()); int ret = shell.run("election", "stepDown", "-peers", sb.toString()); - Assert.assertEquals(0, ret); - Assert.assertEquals(3, cluster.getFollowers().size()); + Assertions.assertEquals(0, ret); + Assertions.assertEquals(3, cluster.getFollowers().size()); } } diff --git a/ratis-test/src/test/java/org/apache/ratis/shell/cli/sh/GroupCommandIntegrationTest.java b/ratis-test/src/test/java/org/apache/ratis/shell/cli/sh/GroupCommandIntegrationTest.java index c745307d2d..83c05bfddd 100644 --- a/ratis-test/src/test/java/org/apache/ratis/shell/cli/sh/GroupCommandIntegrationTest.java +++ b/ratis-test/src/test/java/org/apache/ratis/shell/cli/sh/GroupCommandIntegrationTest.java @@ -29,8 +29,8 @@ import org.apache.ratis.util.Slf4jUtils; import org.apache.ratis.util.SizeInBytes; -import org.junit.Assert; -import org.junit.Test; +import org.junit.jupiter.api.Assertions; +import org.junit.jupiter.api.Test; import org.slf4j.event.Level; public abstract class GroupCommandIntegrationTest @@ -64,11 +64,11 @@ void runTestGroupListCommand(MiniRaftCluster cluster) throws Exception { RatisShell shell = new RatisShell(out.getPrintStream()); int ret = shell.run("group", "list", "-peers", address, "-peerId", leader.getPeer().getId().toString()); - Assert.assertEquals(0, ret); + Assertions.assertEquals(0, ret); String info = out.toString().trim(); String expected = String.format("The peerId %s (server %s) is in 1 groups, and the groupIds is: [%s]", leader.getId(), leader.getPeer().getAddress(), leader.getGroup().getGroupId()); - Assert.assertEquals(expected, info); + Assertions.assertEquals(expected, info); } @Test @@ -82,12 +82,12 @@ void runTestGroupInfoCommand(MiniRaftCluster cluster) throws Exception { final StringPrintStream out = new StringPrintStream(); RatisShell shell = new RatisShell(out.getPrintStream()); int ret = shell.run("group", "info", "-peers", address); - Assert.assertEquals(0 , ret); + Assertions.assertEquals(0 , ret); String result = out.toString().trim(); String hearder = String.format("group id: %s%sleader info: %s(%s)%s%s", cluster.getGroupId().getUuid(), NEW_LINE, leader.getId(), cluster.getLeader().getPeer().getAddress(), NEW_LINE, NEW_LINE); String info = result.substring(0, hearder.length()); - Assert.assertEquals(hearder, info); + Assertions.assertEquals(hearder, info); } } diff --git a/ratis-test/src/test/java/org/apache/ratis/shell/cli/sh/PeerCommandIntegrationTest.java b/ratis-test/src/test/java/org/apache/ratis/shell/cli/sh/PeerCommandIntegrationTest.java index 80400309ec..b3c35147a9 100644 --- a/ratis-test/src/test/java/org/apache/ratis/shell/cli/sh/PeerCommandIntegrationTest.java +++ b/ratis-test/src/test/java/org/apache/ratis/shell/cli/sh/PeerCommandIntegrationTest.java @@ -33,8 +33,8 @@ import org.apache.ratis.util.Slf4jUtils; import org.apache.ratis.util.SizeInBytes; import org.apache.ratis.util.TimeDuration; -import org.junit.Assert; -import org.junit.Test; +import org.junit.jupiter.api.Assertions; +import org.junit.jupiter.api.Test; import org.slf4j.event.Level; import java.util.concurrent.TimeUnit; @@ -68,11 +68,11 @@ void runTestPeerRemoveCommand(MiniRaftCluster cluster) throws Exception { RaftPeer[] peers = new RaftPeer[]{cluster.getFollowers().get(1).getPeer(), leader.getPeer()}; final StringPrintStream out = new StringPrintStream(); RatisShell shell = new RatisShell(out.getPrintStream()); - Assert.assertTrue(cluster.getFollowers().contains(toRemove)); + Assertions.assertTrue(cluster.getFollowers().contains(toRemove)); int ret = shell.run("peer", "remove", "-peers", address, "-peerId", toRemove.getPeer().getId().toString()); - Assert.assertEquals(0, ret); + Assertions.assertEquals(0, ret); RaftServerTestUtil.waitAndCheckNewConf(cluster, peers,1, null); } @@ -95,7 +95,7 @@ void runTestPeerAddCommand(MiniRaftCluster cluster) throws Exception { int ret = shell.run("peer", "add", "-peers", sb.toString(), "-address", newPeers[0].getAdminAddress(), "-peerId", newPeers[0].getId().toString()); - Assert.assertEquals(0, ret); + Assertions.assertEquals(0, ret); RaftServerTestUtil.waitAndCheckNewConf(cluster, ObjectArrays.concat(peers, newPeers[0]), 0, null); } @@ -112,12 +112,12 @@ void runTestPeerSetPriorityCommand(MiniRaftCluster cluster) throws Exception { RaftServer.Division newLeader = cluster.getFollowers().get(0); final StringPrintStream out = new StringPrintStream(); RatisShell shell = new RatisShell(out.getPrintStream()); - Assert.assertTrue(cluster.getFollowers().contains(newLeader)); + Assertions.assertTrue(cluster.getFollowers().contains(newLeader)); int ret = shell.run("peer", "setPriority", "-peers", address, "-addressPriority", newLeader.getPeer().getAddress()+ "|" + 2); - Assert.assertEquals(0, ret); + Assertions.assertEquals(0, ret); JavaUtils.attempt(() -> { - Assert.assertEquals(cluster.getLeader().getId(), newLeader.getId()); + Assertions.assertEquals(cluster.getLeader().getId(), newLeader.getId()); }, 10, TimeDuration.valueOf(1, TimeUnit.SECONDS), "testPeerSetPriorityCommand", LOG); } diff --git a/ratis-test/src/test/java/org/apache/ratis/shell/cli/sh/SnapshotCommandIntegrationTest.java b/ratis-test/src/test/java/org/apache/ratis/shell/cli/sh/SnapshotCommandIntegrationTest.java index 3c0ee9772c..f3f4e43077 100644 --- a/ratis-test/src/test/java/org/apache/ratis/shell/cli/sh/SnapshotCommandIntegrationTest.java +++ b/ratis-test/src/test/java/org/apache/ratis/shell/cli/sh/SnapshotCommandIntegrationTest.java @@ -30,8 +30,8 @@ import org.apache.ratis.statemachine.StateMachine; import org.apache.ratis.util.Slf4jUtils; import org.apache.ratis.util.SizeInBytes; -import org.junit.Assert; -import org.junit.Test; +import org.junit.jupiter.api.Assertions; +import org.junit.jupiter.api.Test; import org.slf4j.event.Level; import java.io.File; @@ -67,7 +67,7 @@ void runTestSnapshotCreateCommand(MiniRaftCluster cluster) throws Exception { try (final RaftClient client = cluster.createClient(leaderId)) { for (int i = 0; i < RaftServerConfigKeys.Snapshot.creationGap(getProperties()); i++) { RaftClientReply reply = client.io().send(new RaftTestUtil.SimpleMessage("m" + i)); - Assert.assertTrue(reply.isSuccess()); + Assertions.assertTrue(reply.isSuccess()); } } final String address = getClusterAddress(cluster); @@ -75,14 +75,14 @@ void runTestSnapshotCreateCommand(MiniRaftCluster cluster) throws Exception { RatisShell shell = new RatisShell(out.getPrintStream()); int ret = shell.run("snapshot", "create", "-peers", address, "-peerId", leader.getPeer().getId().toString()); - Assert.assertEquals(0, ret); + Assertions.assertEquals(0, ret); String[] str = out.toString().trim().split(" "); int snapshotIndex = Integer.parseInt(str[str.length-1]); LOG.info("snapshotIndex = {}", snapshotIndex); final File snapshotFile = SimpleStateMachine4Testing.get(leader) .getStateMachineStorage().getSnapshotFile(leader.getInfo().getCurrentTerm(), snapshotIndex); - Assert.assertTrue(snapshotFile.exists()); + Assertions.assertTrue(snapshotFile.exists()); } void runTestSnapshotCreateCommandOnSpecificServer(MiniRaftCluster cluster) throws Exception { @@ -91,16 +91,16 @@ void runTestSnapshotCreateCommandOnSpecificServer(MiniRaftCluster cluster) throw try (final RaftClient client = cluster.createClient(leaderId)) { for (int i = 0; i < RaftServerConfigKeys.Snapshot.creationGap(getProperties()); i++) { RaftClientReply reply = client.io().send(new RaftTestUtil.SimpleMessage("m" + i)); - Assert.assertTrue(reply.isSuccess()); + Assertions.assertTrue(reply.isSuccess()); } } final String address = getClusterAddress(cluster); final StringPrintStream out = new StringPrintStream(); RatisShell shell = new RatisShell(out.getPrintStream()); - Assert.assertEquals(2, cluster.getFollowers().size()); + Assertions.assertEquals(2, cluster.getFollowers().size()); int ret = shell.run("snapshot", "create", "-peers", address, "-peerId", cluster.getFollowers().get(0).getId().toString()); - Assert.assertEquals(0, ret); + Assertions.assertEquals(0, ret); String[] str = out.toString().trim().split(" "); int snapshotIndex = Integer.parseInt(str[str.length-1]); LOG.info("snapshotIndex = {}", snapshotIndex); @@ -108,7 +108,7 @@ void runTestSnapshotCreateCommandOnSpecificServer(MiniRaftCluster cluster) throw final File snapshotFile = SimpleStateMachine4Testing.get(cluster.getFollowers().get(0)) .getStateMachineStorage() .getSnapshotFile(cluster.getFollowers().get(0).getInfo().getCurrentTerm(), snapshotIndex); - Assert.assertTrue(snapshotFile.exists()); + Assertions.assertTrue(snapshotFile.exists()); } } diff --git a/ratis-test/src/test/java/org/apache/ratis/shell/cli/sh/TestRatisShell.java b/ratis-test/src/test/java/org/apache/ratis/shell/cli/sh/TestRatisShell.java index 6e22276475..21f4685121 100644 --- a/ratis-test/src/test/java/org/apache/ratis/shell/cli/sh/TestRatisShell.java +++ b/ratis-test/src/test/java/org/apache/ratis/shell/cli/sh/TestRatisShell.java @@ -27,8 +27,8 @@ import org.apache.ratis.shell.cli.sh.command.PeerCommand; import org.apache.ratis.shell.cli.sh.command.SnapshotCommand; import org.apache.ratis.util.ReflectionUtils; -import org.junit.Assert; -import org.junit.Test; +import org.junit.jupiter.api.Assertions; +import org.junit.jupiter.api.Test; import org.reflections.Reflections; import java.io.PrintStream; @@ -49,12 +49,12 @@ public class TestRatisShell extends BaseTest { static final Class[] ARG_CLASSES = new Class[] {Context.class}; static void assertCommand(String message, Command expected, Command computed) { - Assert.assertEquals(message, expected.getClass(), computed.getClass()); - Assert.assertEquals(message, expected.getCommandName(), computed.getCommandName()); + Assertions.assertEquals(expected.getClass(), computed.getClass(), message); + Assertions.assertEquals(expected.getCommandName(), computed.getCommandName(), message); } static void assertCommands(List expected, List computed) { - Assert.assertEquals(expected.size(), computed.size()); + Assertions.assertEquals(expected.size(), computed.size()); for(int i = 0; i < expected.size(); i++) { assertCommand("Command " + i, expected.get(i), computed.get(i)); } @@ -103,7 +103,7 @@ static void runTestFullCommandList(Function pare final List computed = new ArrayList<>(parent.getSubCommands().values()); Collections.sort(computed); - Assert.assertFalse(computed.isEmpty()); + Assertions.assertFalse(computed.isEmpty()); final Package pkg = computed.iterator().next().getClass().getPackage(); final List expected = new ArrayList<>(loadCommands(pkg)); diff --git a/ratis-test/src/test/java/org/apache/ratis/statemachine/TestStateMachine.java b/ratis-test/src/test/java/org/apache/ratis/statemachine/TestStateMachine.java index 8e37947d3b..094189827d 100644 --- a/ratis-test/src/test/java/org/apache/ratis/statemachine/TestStateMachine.java +++ b/ratis-test/src/test/java/org/apache/ratis/statemachine/TestStateMachine.java @@ -35,7 +35,8 @@ import org.apache.ratis.server.simulation.MiniRaftClusterWithSimulatedRpc; import org.apache.ratis.statemachine.impl.SimpleStateMachine4Testing; import org.apache.ratis.util.Slf4jUtils; -import org.junit.*; +import org.junit.jupiter.api.Assertions; +import org.junit.jupiter.api.Test; import org.slf4j.event.Level; import java.util.Collections; @@ -48,7 +49,10 @@ import java.util.concurrent.atomic.AtomicLong; import java.util.concurrent.atomic.AtomicReference; -import static org.junit.Assert.*; +import static org.junit.jupiter.api.Assertions.assertEquals; +import static org.junit.jupiter.api.Assertions.assertNotNull; +import static org.junit.jupiter.api.Assertions.assertNull; +import static org.junit.jupiter.api.Assertions.assertTrue; /** * Test StateMachine related functionality @@ -162,9 +166,9 @@ static void runTestTransactionContextIsPassedBack(MiniRaftCluster cluster) throw final SMTransactionContext sm = SMTransactionContext.get(raftServer); final List ll = new ArrayList<>(sm.applied); Collections.sort(ll); - assertEquals(ll.toString(), ll.size(), numTrx); + assertEquals(ll.size(), numTrx, ll.toString()); for (int i=0; i < numTrx; i++) { - assertEquals(ll.toString(), Long.valueOf(i+1), ll.get(i)); + assertEquals(Long.valueOf(i+1), ll.get(i), ll.toString()); } } @@ -193,7 +197,7 @@ public void testStateMachineRegistry() throws Throwable { final RaftServer server = cluster.getServer(id); for(Map.Entry e: registry.entrySet()) { - Assert.assertSame(e.getValue(), server.getDivision(e.getKey()).getStateMachine()); + Assertions.assertSame(e.getValue(), server.getDivision(e.getKey()).getStateMachine()); } } } diff --git a/ratis-test/src/test/java/org/apache/ratis/util/TestDataBlockingQueue.java b/ratis-test/src/test/java/org/apache/ratis/util/TestDataBlockingQueue.java index df0dd2f8ed..e9dcf61d09 100644 --- a/ratis-test/src/test/java/org/apache/ratis/util/TestDataBlockingQueue.java +++ b/ratis-test/src/test/java/org/apache/ratis/util/TestDataBlockingQueue.java @@ -17,8 +17,9 @@ */ package org.apache.ratis.util; -import org.junit.Assert; -import org.junit.Test; +import org.junit.jupiter.api.Assertions; +import org.junit.jupiter.api.Test; +import org.junit.jupiter.api.Timeout; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -36,34 +37,38 @@ public class TestDataBlockingQueue { final TimeDuration slow = TimeDuration.valueOf(100, TimeUnit.MILLISECONDS); final TimeDuration fast = TimeDuration.valueOf(10, TimeUnit.MILLISECONDS); - @Test(timeout = 1000) + @Test + @Timeout(value = 1000) public void testElementLimit() { TestDataQueue.runTestElementLimit(q); } - @Test(timeout = 1000) + @Test + @Timeout(value = 1000) public void testByteLimit() { TestDataQueue.runTestByteLimit(q); } - @Test(timeout = 10_000) + @Test + @Timeout(value = 1000) public void testSlowOfferFastPoll() throws Exception { runTestBlockingCalls(slow, fast, q); } - @Test(timeout = 10_000) + @Test + @Timeout(value = 1000) public void testFastOfferSlowPoll() throws Exception { runTestBlockingCalls(fast, slow, q); } static void assertOfferPull(long offering, long polled, long elementLimit) { - Assert.assertTrue(offering >= polled); - Assert.assertTrue(offering - polled <= elementLimit + 1); + Assertions.assertTrue(offering >= polled); + Assertions.assertTrue(offering - polled <= elementLimit + 1); } static void runTestBlockingCalls(TimeDuration offerSleepTime, TimeDuration pollSleepTime, DataBlockingQueue q) throws Exception { - Assert.assertTrue(q.isEmpty()); + Assertions.assertTrue(q.isEmpty()); ExitUtils.disableSystemExit(); final int elementLimit = q.getElementLimit(); final TimeDuration timeout = CollectionUtils.min(offerSleepTime, pollSleepTime); @@ -74,11 +79,11 @@ static void runTestBlockingCalls(TimeDuration offerSleepTime, TimeDuration pollS final Thread pollThread = new Thread(() -> { try { - for(; polledValue.get() < endValue;) { + while (polledValue.get() < endValue) { pollSleepTime.sleep(); final Long polled = q.poll(timeout); if (polled != null) { - Assert.assertEquals(polledValue.incrementAndGet(), polled.intValue()); + Assertions.assertEquals(polledValue.incrementAndGet(), polled.intValue()); LOG.info("polled {}", polled); } assertOfferPull(offeringValue.get(), polledValue.get(), elementLimit); @@ -109,10 +114,10 @@ static void runTestBlockingCalls(TimeDuration offerSleepTime, TimeDuration pollS offerThread.join(); pollThread.join(); - Assert.assertEquals(endValue + 1, offeringValue.get()); - Assert.assertEquals(endValue, polledValue.get()); + Assertions.assertEquals(endValue + 1, offeringValue.get()); + Assertions.assertEquals(endValue, polledValue.get()); - Assert.assertTrue(q.isEmpty()); + Assertions.assertTrue(q.isEmpty()); ExitUtils.assertNotTerminated(); } } diff --git a/ratis-test/src/test/java/org/apache/ratis/util/TestDataQueue.java b/ratis-test/src/test/java/org/apache/ratis/util/TestDataQueue.java index fe8a7ac246..2dafa7e91c 100644 --- a/ratis-test/src/test/java/org/apache/ratis/util/TestDataQueue.java +++ b/ratis-test/src/test/java/org/apache/ratis/util/TestDataQueue.java @@ -18,8 +18,9 @@ package org.apache.ratis.util; import org.apache.ratis.util.function.TriConsumer; -import org.junit.Assert; -import org.junit.Test; +import org.junit.jupiter.api.Assertions; +import org.junit.jupiter.api.Test; +import org.junit.jupiter.api.Timeout; import java.util.ArrayList; import java.util.Collections; @@ -37,15 +38,16 @@ static TriConsumer getTimeoutHandler(bool } static void assertSizes(long expectedNumElements, long expectedNumBytes, DataQueue q) { - Assert.assertEquals(expectedNumElements, q.getNumElements()); - Assert.assertEquals(expectedNumBytes, q.getNumBytes()); + Assertions.assertEquals(expectedNumElements, q.getNumElements()); + Assertions.assertEquals(expectedNumBytes, q.getNumBytes()); } final SizeInBytes byteLimit = SizeInBytes.valueOf(100); final int elementLimit = 5; final DataQueue q = new DataQueue<>(null, byteLimit, elementLimit, Long::longValue); - @Test(timeout = 1000) + @Test + @Timeout(value = 1000) public void testElementLimit() { runTestElementLimit(q); } @@ -56,30 +58,31 @@ static void runTestElementLimit(DataQueue q) { final int elementLimit = q.getElementLimit(); long numBytes = 0; for (long i = 0; i < elementLimit; i++) { - Assert.assertEquals(i, q.getNumElements()); - Assert.assertEquals(numBytes, q.getNumBytes()); + Assertions.assertEquals(i, q.getNumElements()); + Assertions.assertEquals(numBytes, q.getNumBytes()); final boolean offered = q.offer(i); - Assert.assertTrue(offered); + Assertions.assertTrue(offered); numBytes += i; assertSizes(i+1, numBytes, q); } { final boolean offered = q.offer(0L); - Assert.assertFalse(offered); + Assertions.assertFalse(offered); assertSizes(elementLimit, numBytes, q); } { // poll all elements final List polled = q.pollList(100, (i, timeout) -> i, getTimeoutHandler(false)); - Assert.assertEquals(elementLimit, polled.size()); + Assertions.assertEquals(elementLimit, polled.size()); for (int i = 0; i < polled.size(); i++) { - Assert.assertEquals(i, polled.get(i).intValue()); + Assertions.assertEquals(i, polled.get(i).intValue()); } } assertSizes(0, 0, q); } - @Test(timeout = 1000) + @Test + @Timeout(value = 1000) public void testByteLimit() { runTestByteLimit(q); } @@ -90,53 +93,54 @@ static void runTestByteLimit(DataQueue q) { final long byteLimit = q.getByteLimit(); try { q.offer(byteLimit + 1); - Assert.fail(); + Assertions.fail(); } catch (IllegalStateException ignored) { } final long halfBytes = byteLimit / 2; { final boolean offered = q.offer(halfBytes); - Assert.assertTrue(offered); + Assertions.assertTrue(offered); assertSizes(1, halfBytes, q); } { final boolean offered = q.offer(halfBytes + 1); - Assert.assertFalse(offered); + Assertions.assertFalse(offered); assertSizes(1, halfBytes, q); } { final boolean offered = q.offer(halfBytes); - Assert.assertTrue(offered); + Assertions.assertTrue(offered); assertSizes(2, byteLimit, q); } { final boolean offered = q.offer(1L); - Assert.assertFalse(offered); + Assertions.assertFalse(offered); assertSizes(2, byteLimit, q); } { final boolean offered = q.offer(0L); - Assert.assertTrue(offered); + Assertions.assertTrue(offered); assertSizes(3, byteLimit, q); } { // poll all elements final List polled = q.pollList(100, (i, timeout) -> i, getTimeoutHandler(false)); - Assert.assertEquals(3, polled.size()); - Assert.assertEquals(halfBytes, polled.get(0).intValue()); - Assert.assertEquals(halfBytes, polled.get(1).intValue()); - Assert.assertEquals(0, polled.get(2).intValue()); + Assertions.assertEquals(3, polled.size()); + Assertions.assertEquals(halfBytes, polled.get(0).intValue()); + Assertions.assertEquals(halfBytes, polled.get(1).intValue()); + Assertions.assertEquals(0, polled.get(2).intValue()); } assertSizes(0, 0, q); } - @Test(timeout = 1000) + @Test + @Timeout(value = 1000) public void testIteratorAndRemove() { runTestIteratorAndRemove(q); } @@ -149,7 +153,7 @@ static void runTestIteratorAndRemove(DataQueue q) { long numBytes = 0; for(long i = 0; i < elementLimit; i++) { final boolean offered = q.offer(i); - Assert.assertTrue(offered); + Assertions.assertTrue(offered); numElements++; numBytes += i; assertSizes(numElements, numBytes, q); @@ -158,7 +162,7 @@ static void runTestIteratorAndRemove(DataQueue q) { { // test iterator() final Iterator i = q.iterator(); for (long expected = 0; expected < elementLimit; expected++) { - Assert.assertEquals(expected, i.next().longValue()); + Assertions.assertEquals(expected, i.next().longValue()); } } @@ -180,23 +184,24 @@ static void runTestIteratorAndRemove(DataQueue q) { assertSizes(0, 0, q); } - @Test(timeout = 1000) + @Test + @Timeout(value = 1000) public void testTimeout() { assertSizes(0, 0, q); long numBytes = 0; for (long i = 0; i < elementLimit; i++) { - Assert.assertEquals(i, q.getNumElements()); - Assert.assertEquals(numBytes, q.getNumBytes()); + Assertions.assertEquals(i, q.getNumElements()); + Assertions.assertEquals(numBytes, q.getNumBytes()); final boolean offered = q.offer(i); - Assert.assertTrue(offered); + Assertions.assertTrue(offered); numBytes += i; assertSizes(i+1, numBytes, q); } { // poll with zero time final List polled = q.pollList(0, (i, timeout) -> i, getTimeoutHandler(false)); - Assert.assertTrue(polled.isEmpty()); + Assertions.assertTrue(polled.isEmpty()); assertSizes(elementLimit, numBytes, q); } @@ -209,9 +214,9 @@ public void testTimeout() { } return i; }, getTimeoutHandler(true)); - Assert.assertEquals(halfElements, polled.size()); + Assertions.assertEquals(halfElements, polled.size()); for (int i = 0; i < polled.size(); i++) { - Assert.assertEquals(i, polled.get(i).intValue()); + Assertions.assertEquals(i, polled.get(i).intValue()); numBytes -= i; } assertSizes(elementLimit - halfElements, numBytes, q); @@ -219,9 +224,9 @@ public void testTimeout() { { // poll the remaining elements final List polled = q.pollList(100, (i, timeout) -> i, getTimeoutHandler(false)); - Assert.assertEquals(elementLimit - halfElements, polled.size()); + Assertions.assertEquals(elementLimit - halfElements, polled.size()); for (int i = 0; i < polled.size(); i++) { - Assert.assertEquals(halfElements + i, polled.get(i).intValue()); + Assertions.assertEquals(halfElements + i, polled.get(i).intValue()); } } assertSizes(0, 0, q); diff --git a/ratis-test/src/test/java/org/apache/ratis/util/TestExitUtils.java b/ratis-test/src/test/java/org/apache/ratis/util/TestExitUtils.java index 25b05fd0df..1dd5b270bb 100644 --- a/ratis-test/src/test/java/org/apache/ratis/util/TestExitUtils.java +++ b/ratis-test/src/test/java/org/apache/ratis/util/TestExitUtils.java @@ -19,15 +19,17 @@ import org.apache.ratis.BaseTest; import org.apache.ratis.util.ExitUtils.ExitException; -import org.junit.Assert; -import org.junit.Test; +import org.junit.jupiter.api.Assertions; +import org.junit.jupiter.api.Test; +import org.junit.jupiter.api.Timeout; public class TestExitUtils extends BaseTest { /** Test if {@link BaseTest} can handle uncaught exception. */ - @Test(timeout = 1000) + @Test + @Timeout(value = 1000) public void testUncaughtException() throws Exception { - Assert.assertFalse(ExitUtils.isTerminated()); - Assert.assertFalse(ExitUtils.clear()); + Assertions.assertFalse(ExitUtils.isTerminated()); + Assertions.assertFalse(ExitUtils.clear()); final Thread t = new Thread(null, () -> { throw new AssertionError("Testing"); @@ -35,25 +37,26 @@ public void testUncaughtException() throws Exception { t.start(); t.join(); - Assert.assertTrue(ExitUtils.isTerminated()); - Assert.assertTrue(ExitUtils.clear()); + Assertions.assertTrue(ExitUtils.isTerminated()); + Assertions.assertTrue(ExitUtils.clear()); } /** Test if {@link BaseTest} can handle ExitUtils.terminate(..). */ - @Test(timeout = 1000) + @Test + @Timeout(value = 1000) public void testExitStatus() { - Assert.assertFalse(ExitUtils.isTerminated()); - Assert.assertFalse(ExitUtils.clear()); + Assertions.assertFalse(ExitUtils.isTerminated()); + Assertions.assertFalse(ExitUtils.clear()); final int status = -1; try { ExitUtils.terminate(status, "testExitStatus", LOG); - Assert.fail(); + Assertions.fail(); } catch (ExitException e) { - Assert.assertEquals(status, e.getStatus()); + Assertions.assertEquals(status, e.getStatus()); } - Assert.assertTrue(ExitUtils.isTerminated()); - Assert.assertTrue(ExitUtils.clear()); + Assertions.assertTrue(ExitUtils.isTerminated()); + Assertions.assertTrue(ExitUtils.clear()); } } diff --git a/ratis-test/src/test/java/org/apache/ratis/util/TestLifeCycle.java b/ratis-test/src/test/java/org/apache/ratis/util/TestLifeCycle.java index 3faf2a4959..f817449995 100644 --- a/ratis-test/src/test/java/org/apache/ratis/util/TestLifeCycle.java +++ b/ratis-test/src/test/java/org/apache/ratis/util/TestLifeCycle.java @@ -18,12 +18,13 @@ package org.apache.ratis.util; import org.apache.ratis.util.function.TriConsumer; -import org.junit.Test; +import org.junit.jupiter.api.Test; +import org.junit.jupiter.api.Timeout; import static org.apache.ratis.util.LifeCycle.State.*; -import static org.junit.Assert.assertEquals; -import static org.junit.Assert.assertFalse; -import static org.junit.Assert.assertTrue; +import static org.junit.jupiter.api.Assertions.assertEquals; +import static org.junit.jupiter.api.Assertions.assertFalse; +import static org.junit.jupiter.api.Assertions.assertTrue; import java.util.*; @@ -33,7 +34,8 @@ public class TestLifeCycle { * {@link LifeCycle} uses predecessors to validate transitions * while this test uses successors. */ - @Test(timeout = 1000) + @Test + @Timeout(value = 1000) public void testIsValid() { final Map> successors = new EnumMap<>(LifeCycle.State.class); @@ -49,9 +51,8 @@ public void testIsValid() { final List states = Arrays.asList(LifeCycle.State.values()); states.forEach( from -> states.forEach( - to -> assertEquals(from + " -> " + to, - successors.get(from).contains(to), - isValid(from, to)))); + to -> assertEquals(successors.get(from).contains(to), + isValid(from, to), from + " -> " + to))); } @Test @@ -90,7 +91,7 @@ private static void testInvalidTransition(TriConsumer n).limit(10)); @@ -43,11 +45,11 @@ static void runTestMinMax(LongStream stream) { final List list = stream.collect(ArrayList::new, List::add, List::addAll); final LongMinMax longMinMax = toLongStream(list).collect(LongMinMax::new, LongMinMax::accumulate, LongMinMax::combine); if (longMinMax.isInitialized()) { - Assert.assertEquals(toLongStream(list).min().getAsLong(), longMinMax.getMin()); - Assert.assertEquals(toLongStream(list).max().getAsLong(), longMinMax.getMax()); + Assertions.assertEquals(toLongStream(list).min().getAsLong(), longMinMax.getMin()); + Assertions.assertEquals(toLongStream(list).max().getAsLong(), longMinMax.getMax()); } else { - Assert.assertEquals(OptionalLong.empty(), toLongStream(list).min()); - Assert.assertEquals(OptionalLong.empty(), toLongStream(list).max()); + Assertions.assertEquals(OptionalLong.empty(), toLongStream(list).min()); + Assertions.assertEquals(OptionalLong.empty(), toLongStream(list).max()); } } diff --git a/ratis-test/src/test/java/org/apache/ratis/util/TestPeerProxyMap.java b/ratis-test/src/test/java/org/apache/ratis/util/TestPeerProxyMap.java index 549fbc53f8..118830bda8 100644 --- a/ratis-test/src/test/java/org/apache/ratis/util/TestPeerProxyMap.java +++ b/ratis-test/src/test/java/org/apache/ratis/util/TestPeerProxyMap.java @@ -20,8 +20,9 @@ import org.apache.ratis.BaseTest; import org.apache.ratis.protocol.RaftPeer; import org.apache.ratis.protocol.RaftPeerId; -import org.junit.Assert; -import org.junit.Test; +import org.junit.jupiter.api.Assertions; +import org.junit.jupiter.api.Test; +import org.junit.jupiter.api.Timeout; import java.io.Closeable; import java.io.IOException; @@ -51,7 +52,8 @@ public String toString() { } } - @Test(timeout = 10_000) + @Test + @Timeout(value = 10_000) public void testCloseDeadLock() throws Exception { final PeerProxyMap map = new PeerProxyMap<>("test", DummyProxy::new); final RaftPeerId id = RaftPeerId.valueOf("s0"); @@ -68,7 +70,7 @@ public void testCloseDeadLock() throws Exception { HUNDRED_MILLIS.sleep(); LOG.info("Try getProxy"); final DummyProxy newProxy = map.getProxy(id); - Assert.assertNotSame(proxy, newProxy); + Assertions.assertNotSame(proxy, newProxy); } catch (Exception e) { setFirstException(e); } @@ -115,7 +117,8 @@ public String toString() { } } - @Test(timeout = 1000) + @Test + @Timeout(value = 1000) public void testStackTrace() { final RaftPeerId id = RaftPeerId.valueOf("s0"); final RaftPeer peer = RaftPeer.newBuilder().setId(id).build(); @@ -123,7 +126,7 @@ public void testStackTrace() { final ExceptionProxy ignored = map.computeIfAbsent(peer).get()) { } catch (IOException e) { assertThrowable("closeProxy", e, AnnotatedConnectException.class, LOG, ConnectException.class); - Assert.assertEquals(0, e.getStackTrace().length); + Assertions.assertEquals(0, e.getStackTrace().length); } } } diff --git a/ratis-test/src/test/java/org/apache/ratis/util/TestPreconditions.java b/ratis-test/src/test/java/org/apache/ratis/util/TestPreconditions.java index 884c1e5d59..e99c11fdde 100644 --- a/ratis-test/src/test/java/org/apache/ratis/util/TestPreconditions.java +++ b/ratis-test/src/test/java/org/apache/ratis/util/TestPreconditions.java @@ -18,7 +18,8 @@ package org.apache.ratis.util; import org.apache.ratis.BaseTest; -import org.junit.Test; +import org.junit.jupiter.api.Test; +import org.junit.jupiter.api.Timeout; import java.util.Arrays; import java.util.Collections; @@ -28,7 +29,8 @@ import java.util.Set; public class TestPreconditions extends BaseTest { - @Test(timeout = 1000) + @Test + @Timeout(value = 1000) public void testAssertUnique() { final Set empty = Collections.emptySet(); Preconditions.assertUnique(empty); @@ -53,7 +55,8 @@ public void testAssertUnique() { Preconditions.assertUnique(three, Arrays.asList(4, 5, 6)); } - @Test(timeout = 1000) + @Test + @Timeout(value = 1000) public void testAssertNull() { final Map map = new HashMap<>(); final String key = "abc1234"; diff --git a/ratis-test/src/test/java/org/apache/ratis/util/TestReferenceCountedObject.java b/ratis-test/src/test/java/org/apache/ratis/util/TestReferenceCountedObject.java index fe58b92da6..502261ec23 100644 --- a/ratis-test/src/test/java/org/apache/ratis/util/TestReferenceCountedObject.java +++ b/ratis-test/src/test/java/org/apache/ratis/util/TestReferenceCountedObject.java @@ -18,8 +18,9 @@ package org.apache.ratis.util; import org.apache.ratis.util.function.UncheckedAutoCloseableSupplier; -import org.junit.Assert; -import org.junit.Test; +import org.junit.jupiter.api.Assertions; +import org.junit.jupiter.api.Test; +import org.junit.jupiter.api.Timeout; import java.util.concurrent.atomic.AtomicInteger; @@ -27,8 +28,8 @@ public class TestReferenceCountedObject { static void assertValues( AtomicInteger retained, int expectedRetained, AtomicInteger released, int expectedReleased) { - Assert.assertEquals("retained", expectedRetained, retained.get()); - Assert.assertEquals("released", expectedReleased, released.get()); + Assertions.assertEquals(expectedRetained, retained.get(), "retained"); + Assertions.assertEquals(expectedReleased, released.get(), "retained"); } static void assertRelease(ReferenceCountedObject ref, @@ -36,10 +37,11 @@ static void assertRelease(ReferenceCountedObject ref, AtomicInteger released, int expectedReleased) { final boolean returned = ref.release(); assertValues(retained, expectedRetained, released, expectedReleased); - Assert.assertEquals(expectedRetained == expectedReleased, returned); + Assertions.assertEquals(expectedRetained == expectedReleased, returned); } - @Test(timeout = 1000) + @Test + @Timeout(value = 1000) public void testWrap() { final String value = "testWrap"; final AtomicInteger retained = new AtomicInteger(); @@ -50,19 +52,19 @@ public void testWrap() { assertValues(retained, 0, released, 0); try { ref.get(); - Assert.fail(); + Assertions.fail(); } catch (IllegalStateException e) { e.printStackTrace(System.out); } assertValues(retained, 0, released, 0); - Assert.assertEquals(value, ref.retain()); + Assertions.assertEquals(value, ref.retain()); assertValues(retained, 1, released, 0); try(UncheckedAutoCloseableSupplier auto = ref.retainAndReleaseOnClose()) { final String got = auto.get(); - Assert.assertEquals(value, got); - Assert.assertSame(got, auto.get()); // it should return the same object. + Assertions.assertEquals(value, got); + Assertions.assertSame(got, auto.get()); // it should return the same object. assertValues(retained, 2, released, 0); } catch (IllegalStateException e) { e.printStackTrace(System.out); @@ -70,12 +72,12 @@ public void testWrap() { assertValues(retained, 2, released, 1); final UncheckedAutoCloseableSupplier notClosing = ref.retainAndReleaseOnClose(); - Assert.assertEquals(value, notClosing.get()); + Assertions.assertEquals(value, notClosing.get()); assertValues(retained, 3, released, 1); assertRelease(ref, retained, 3, released, 2); final UncheckedAutoCloseableSupplier auto = ref.retainAndReleaseOnClose(); - Assert.assertEquals(value, auto.get()); + Assertions.assertEquals(value, auto.get()); assertValues(retained, 4, released, 2); auto.close(); assertValues(retained, 4, released, 3); @@ -87,59 +89,60 @@ public void testWrap() { try { ref.get(); - Assert.fail(); + Assertions.fail(); } catch (IllegalStateException e) { e.printStackTrace(System.out); } try { ref.retain(); - Assert.fail(); + Assertions.fail(); } catch (IllegalStateException e) { e.printStackTrace(System.out); } try(UncheckedAutoCloseable ignore = ref.retainAndReleaseOnClose()) { - Assert.fail(); + Assertions.fail(); } catch (IllegalStateException e) { e.printStackTrace(System.out); } try { ref.release(); - Assert.fail(); + Assertions.fail(); } catch (IllegalStateException e) { e.printStackTrace(System.out); } } - @Test(timeout = 1000) + @Test + @Timeout(value = 1000) public void testReleaseWithoutRetaining() { final ReferenceCountedObject ref = ReferenceCountedObject.wrap(""); try { ref.release(); - Assert.fail(); + Assertions.fail(); } catch (IllegalStateException e) { e.printStackTrace(System.out); } try { ref.get(); - Assert.fail(); + Assertions.fail(); } catch (IllegalStateException e) { e.printStackTrace(System.out); } try { ref.retain(); - Assert.fail(); + Assertions.fail(); } catch (IllegalStateException e) { e.printStackTrace(System.out); } try(UncheckedAutoCloseable ignore = ref.retainAndReleaseOnClose()) { - Assert.fail(); + Assertions.fail(); } catch (IllegalStateException e) { e.printStackTrace(System.out); } diff --git a/ratis-test/src/test/java/org/apache/ratis/util/TestResourceSemaphore.java b/ratis-test/src/test/java/org/apache/ratis/util/TestResourceSemaphore.java index 6fe1aed7ef..0f3612dce4 100644 --- a/ratis-test/src/test/java/org/apache/ratis/util/TestResourceSemaphore.java +++ b/ratis-test/src/test/java/org/apache/ratis/util/TestResourceSemaphore.java @@ -19,15 +19,17 @@ import org.apache.ratis.BaseTest; import org.apache.ratis.RaftTestUtil; -import org.junit.Assert; -import org.junit.Test; +import org.junit.jupiter.api.Assertions; +import org.junit.jupiter.api.Test; +import org.junit.jupiter.api.Timeout; import java.util.concurrent.TimeoutException; import static org.apache.ratis.util.ResourceSemaphore.Group.SUCCESS; public class TestResourceSemaphore extends BaseTest { - @Test(timeout = 5000) + @Test + @Timeout(value = 5000) public void testGroup() throws InterruptedException, TimeoutException { final int FAILED_IN_ELEMENT_LIMIT = 0; final int FAILED_IN_BYTE_SIZE_LIMIT = 1; @@ -80,15 +82,15 @@ public void testGroup() throws InterruptedException, TimeoutException { } static void assertUsed(ResourceSemaphore.Group g, int... expected) { - Assert.assertEquals(expected.length, g.resourceSize()); + Assertions.assertEquals(expected.length, g.resourceSize()); for(int i = 0; i < expected.length; i++) { - Assert.assertEquals(expected[i], g.get(i).used()); + Assertions.assertEquals(expected[i], g.get(i).used()); } } static void assertAcquire(ResourceSemaphore.Group g, int expected, int... permits) { final int computed = g.tryAcquire(permits); - Assert.assertEquals(expected, computed); + Assertions.assertEquals(expected, computed); } static Runnable acquire(ResourceSemaphore.Group g, int... permits) { diff --git a/ratis-test/src/test/java/org/apache/ratis/util/TestTimeDuration.java b/ratis-test/src/test/java/org/apache/ratis/util/TestTimeDuration.java index a2c180d922..f2d58f48db 100644 --- a/ratis-test/src/test/java/org/apache/ratis/util/TestTimeDuration.java +++ b/ratis-test/src/test/java/org/apache/ratis/util/TestTimeDuration.java @@ -17,8 +17,9 @@ */ package org.apache.ratis.util; -import org.junit.Assert; -import org.junit.Test; +import org.junit.jupiter.api.Assertions; +import org.junit.jupiter.api.Test; +import org.junit.jupiter.api.Timeout; import org.slf4j.event.Level; import java.util.Arrays; @@ -28,17 +29,18 @@ import static org.apache.ratis.util.TimeDuration.Abbreviation; import static org.apache.ratis.util.TimeDuration.parse; -import static org.junit.Assert.assertEquals; -import static org.junit.Assert.assertFalse; -import static org.junit.Assert.assertNotNull; -import static org.junit.Assert.assertTrue; +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.assertTrue; public class TestTimeDuration { { Slf4jUtils.setLogLevel(TimeDuration.LOG, Level.DEBUG); } - @Test(timeout = 1000) + @Test + @Timeout(value = 1000) public void testAbbreviation() { Arrays.asList(TimeUnit.values()) .forEach(a -> assertNotNull(Abbreviation.valueOf(a.name()))); @@ -51,10 +53,11 @@ public void testAbbreviation() { Arrays.asList(TimeUnit.values()).forEach(unit -> allSymbols.stream() .map(s -> "0" + s) - .forEach(s -> assertEquals(s, 0L, parse(s, unit)))); + .forEach(s -> assertEquals(0L, parse(s, unit), s))); } - @Test(timeout = 1000) + @Test + @Timeout(value = 1000) public void testParse() { assertEquals(1L, parse("1_000_000 ns", TimeUnit.MILLISECONDS)); assertEquals(10L, parse("10_000_000 nanos", TimeUnit.MILLISECONDS)); @@ -93,7 +96,8 @@ public void testParse() { assertEquals(2400, parse("100 days", TimeUnit.HOURS)); } - @Test(timeout = 1000) + @Test + @Timeout(value = 1000) public void testRoundUp() { final long nanosPerSecond = 1_000_000_000L; final TimeDuration oneSecond = TimeDuration.valueOf(1, TimeUnit.SECONDS); @@ -108,7 +112,8 @@ public void testRoundUp() { assertEquals(2*nanosPerSecond, oneSecond.roundUpNanos(nanosPerSecond + 1)); } - @Test(timeout = 1000) + @Test + @Timeout(value = 1000) public void testTo() { final TimeDuration oneSecond = TimeDuration.valueOf(1, TimeUnit.SECONDS); assertTo(1000, "1000ms", oneSecond, TimeUnit.MILLISECONDS); @@ -131,7 +136,8 @@ static TimeDuration assertTo(long expected, String expectedString, TimeDuration return computed; } - @Test(timeout = 1000) + @Test + @Timeout(value = 1000) public void testAddAndSubtract() { final TimeDuration oneSecond = TimeDuration.valueOf(1, TimeUnit.SECONDS); final TimeDuration tenSecond = TimeDuration.valueOf(10, TimeUnit.SECONDS); @@ -184,28 +190,30 @@ public void testAddAndSubtract() { } } - @Test(timeout = 1000) + @Test + @Timeout(value = 1000) public void testNegate() { assertNegate(0); assertNegate(1); assertNegate(-1); assertNegate(Long.MAX_VALUE); - Assert.assertEquals( + Assertions.assertEquals( TimeDuration.valueOf(Long.MAX_VALUE, TimeUnit.SECONDS), TimeDuration.valueOf(Long.MIN_VALUE, TimeUnit.SECONDS).negate()); } private static void assertNegate(long n) { - Assert.assertEquals( + Assertions.assertEquals( TimeDuration.valueOf(-n, TimeUnit.SECONDS), TimeDuration.valueOf(n, TimeUnit.SECONDS).negate()); - Assert.assertEquals( + Assertions.assertEquals( TimeDuration.valueOf(n, TimeUnit.SECONDS), TimeDuration.valueOf(-n, TimeUnit.SECONDS).negate()); } - @Test(timeout = 1000) + @Test + @Timeout(value = 1000) public void testMultiply() { assertMultiply(0, TimeDuration.ONE_SECOND, TimeDuration.valueOf(0, TimeUnit.SECONDS)); assertMultiply(0.001, TimeDuration.ONE_SECOND, TimeDuration.ONE_MILLISECOND); @@ -230,12 +238,13 @@ private static void assertMultiply(double multiplier, TimeDuration t, TimeDurati private static void assertMultiply(TimeDuration t, double multiplier, TimeDuration expected) { final TimeDuration computed = t.multiply(multiplier); TimeDuration.LOG.info("assertMultiply: {} x {} = {} ?= {}\n\n", t, multiplier, computed, expected); - Assert.assertEquals(expected.getUnit(), computed.getUnit()); + Assertions.assertEquals(expected.getUnit(), computed.getUnit()); final long d = Math.abs(computed.getDuration() - expected.getDuration()); - Assert.assertTrue(d <= Math.abs(expected.getDuration()) * TimeDuration.ERROR_THRESHOLD); + Assertions.assertTrue(d <= Math.abs(expected.getDuration()) * TimeDuration.ERROR_THRESHOLD); } - @Test(timeout = 1000) + @Test + @Timeout(value = 1000) public void testHigherLower() { final TimeUnit[] units = {TimeUnit.NANOSECONDS, TimeUnit.MICROSECONDS, TimeUnit.MILLISECONDS, TimeUnit.SECONDS, TimeUnit.MINUTES, TimeUnit.HOURS, TimeUnit.DAYS}; @@ -243,11 +252,12 @@ public void testHigherLower() { assertHigherLower(units[i-1], units[i]); } - Assert.assertSame(TimeUnit.NANOSECONDS, TimeDuration.lowerUnit(TimeUnit.NANOSECONDS)); - Assert.assertSame(TimeUnit.DAYS, TimeDuration.higherUnit(TimeUnit.DAYS)); + Assertions.assertSame(TimeUnit.NANOSECONDS, TimeDuration.lowerUnit(TimeUnit.NANOSECONDS)); + Assertions.assertSame(TimeUnit.DAYS, TimeDuration.higherUnit(TimeUnit.DAYS)); } - @Test(timeout = 1000) + @Test + @Timeout(value = 1000) public void testCompareTo() { assertTimeDurationCompareTo(TimeDuration.ONE_MINUTE, TimeDuration.ONE_SECOND); @@ -281,7 +291,7 @@ static void assertTimeDurationCompareTo(TimeDuration larger, TimeDuration smalle } private static void assertHigherLower(TimeUnit lower, TimeUnit higher) { - Assert.assertSame(lower, TimeDuration.lowerUnit(higher)); - Assert.assertSame(higher, TimeDuration.higherUnit(lower)); + Assertions.assertSame(lower, TimeDuration.lowerUnit(higher)); + Assertions.assertSame(higher, TimeDuration.higherUnit(lower)); } } diff --git a/ratis-test/src/test/java/org/apache/ratis/util/TestTimeoutScheduler.java b/ratis-test/src/test/java/org/apache/ratis/util/TestTimeoutScheduler.java index 848428f5b8..79c97a6ede 100644 --- a/ratis-test/src/test/java/org/apache/ratis/util/TestTimeoutScheduler.java +++ b/ratis-test/src/test/java/org/apache/ratis/util/TestTimeoutScheduler.java @@ -18,8 +18,9 @@ package org.apache.ratis.util; import org.apache.ratis.BaseTest; -import org.junit.Assert; -import org.junit.Test; +import org.junit.jupiter.api.Assertions; +import org.junit.jupiter.api.Test; +import org.junit.jupiter.api.Timeout; import org.slf4j.event.Level; import java.util.concurrent.TimeUnit; @@ -41,52 +42,54 @@ public void accept(RuntimeException e) { } void assertNoError() { - Assert.assertFalse(hasError.get()); + Assertions.assertFalse(hasError.get()); } } - @Test(timeout = 1000) + @Test + @Timeout(value = 1000) public void testSingleTask() throws Exception { final TimeoutScheduler scheduler = TimeoutScheduler.newInstance(); final TimeDuration grace = TimeDuration.valueOf(100, TimeUnit.MILLISECONDS); scheduler.setGracePeriod(grace); - Assert.assertFalse(scheduler.hasScheduler()); + Assertions.assertFalse(scheduler.hasScheduler()); final ErrorHandler errorHandler = new ErrorHandler(); final AtomicBoolean fired = new AtomicBoolean(false); scheduler.onTimeout(TimeDuration.valueOf(250, TimeUnit.MILLISECONDS), () -> { - Assert.assertFalse(fired.get()); + Assertions.assertFalse(fired.get()); fired.set(true); }, errorHandler); - Assert.assertTrue(scheduler.hasScheduler()); + Assertions.assertTrue(scheduler.hasScheduler()); Thread.sleep(100); - Assert.assertFalse(fired.get()); - Assert.assertTrue(scheduler.hasScheduler()); + Assertions.assertFalse(fired.get()); + Assertions.assertTrue(scheduler.hasScheduler()); Thread.sleep(100); - Assert.assertFalse(fired.get()); - Assert.assertTrue(scheduler.hasScheduler()); + Assertions.assertFalse(fired.get()); + Assertions.assertTrue(scheduler.hasScheduler()); Thread.sleep(100); - Assert.assertTrue(fired.get()); - Assert.assertTrue(scheduler.hasScheduler()); + Assertions.assertTrue(fired.get()); + Assertions.assertTrue(scheduler.hasScheduler()); Thread.sleep(100); - Assert.assertTrue(fired.get()); - Assert.assertFalse(scheduler.hasScheduler()); + Assertions.assertTrue(fired.get()); + Assertions.assertFalse(scheduler.hasScheduler()); errorHandler.assertNoError(); scheduler.setGracePeriod(grace); } - @Test(timeout = 1000) + @Test + @Timeout(value = 1000) public void testMultipleTasks() throws Exception { final TimeoutScheduler scheduler = TimeoutScheduler.newInstance(); final TimeDuration grace = TimeDuration.valueOf(100, TimeUnit.MILLISECONDS); scheduler.setGracePeriod(grace); - Assert.assertFalse(scheduler.hasScheduler()); + Assertions.assertFalse(scheduler.hasScheduler()); final ErrorHandler errorHandler = new ErrorHandler(); @@ -94,126 +97,129 @@ public void testMultipleTasks() throws Exception { for(int i = 0; i < fired.length; i++) { final AtomicBoolean f = fired[i] = new AtomicBoolean(false); scheduler.onTimeout(TimeDuration.valueOf(100*i + 50, TimeUnit.MILLISECONDS), () -> { - Assert.assertFalse(f.get()); + Assertions.assertFalse(f.get()); f.set(true); }, errorHandler); - Assert.assertTrue(scheduler.hasScheduler()); + Assertions.assertTrue(scheduler.hasScheduler()); } Thread.sleep(100); - Assert.assertTrue(fired[0].get()); - Assert.assertFalse(fired[1].get()); - Assert.assertFalse(fired[2].get()); - Assert.assertTrue(scheduler.hasScheduler()); + Assertions.assertTrue(fired[0].get()); + Assertions.assertFalse(fired[1].get()); + Assertions.assertFalse(fired[2].get()); + Assertions.assertTrue(scheduler.hasScheduler()); Thread.sleep(100); - Assert.assertTrue(fired[0].get()); - Assert.assertTrue(fired[1].get()); - Assert.assertFalse(fired[2].get()); - Assert.assertTrue(scheduler.hasScheduler()); + Assertions.assertTrue(fired[0].get()); + Assertions.assertTrue(fired[1].get()); + Assertions.assertFalse(fired[2].get()); + Assertions.assertTrue(scheduler.hasScheduler()); Thread.sleep(100); - Assert.assertTrue(fired[0].get()); - Assert.assertTrue(fired[1].get()); - Assert.assertTrue(fired[2].get()); - Assert.assertTrue(scheduler.hasScheduler()); + Assertions.assertTrue(fired[0].get()); + Assertions.assertTrue(fired[1].get()); + Assertions.assertTrue(fired[2].get()); + Assertions.assertTrue(scheduler.hasScheduler()); Thread.sleep(100); - Assert.assertTrue(fired[0].get()); - Assert.assertTrue(fired[1].get()); - Assert.assertTrue(fired[2].get()); - Assert.assertFalse(scheduler.hasScheduler()); + Assertions.assertTrue(fired[0].get()); + Assertions.assertTrue(fired[1].get()); + Assertions.assertTrue(fired[2].get()); + Assertions.assertFalse(scheduler.hasScheduler()); errorHandler.assertNoError(); } - @Test(timeout = 1000) + @Test + @Timeout(value = 1000) public void testExtendingGracePeriod() throws Exception { final TimeoutScheduler scheduler = TimeoutScheduler.newInstance(); final TimeDuration grace = TimeDuration.valueOf(100, TimeUnit.MILLISECONDS); scheduler.setGracePeriod(grace); - Assert.assertFalse(scheduler.hasScheduler()); + Assertions.assertFalse(scheduler.hasScheduler()); final ErrorHandler errorHandler = new ErrorHandler(); { final AtomicBoolean fired = new AtomicBoolean(false); scheduler.onTimeout(TimeDuration.valueOf(150, TimeUnit.MILLISECONDS), () -> { - Assert.assertFalse(fired.get()); + Assertions.assertFalse(fired.get()); fired.set(true); }, errorHandler); - Assert.assertTrue(scheduler.hasScheduler()); + Assertions.assertTrue(scheduler.hasScheduler()); Thread.sleep(100); - Assert.assertFalse(fired.get()); - Assert.assertTrue(scheduler.hasScheduler()); + Assertions.assertFalse(fired.get()); + Assertions.assertTrue(scheduler.hasScheduler()); Thread.sleep(100); - Assert.assertTrue(fired.get()); - Assert.assertTrue(scheduler.hasScheduler()); + Assertions.assertTrue(fired.get()); + Assertions.assertTrue(scheduler.hasScheduler()); } { // submit another task during grace period final AtomicBoolean fired2 = new AtomicBoolean(false); scheduler.onTimeout(TimeDuration.valueOf(150, TimeUnit.MILLISECONDS), () -> { - Assert.assertFalse(fired2.get()); + Assertions.assertFalse(fired2.get()); fired2.set(true); }, errorHandler); Thread.sleep(100); - Assert.assertFalse(fired2.get()); - Assert.assertTrue(scheduler.hasScheduler()); + Assertions.assertFalse(fired2.get()); + Assertions.assertTrue(scheduler.hasScheduler()); Thread.sleep(100); - Assert.assertTrue(fired2.get()); - Assert.assertTrue(scheduler.hasScheduler()); + Assertions.assertTrue(fired2.get()); + Assertions.assertTrue(scheduler.hasScheduler()); Thread.sleep(100); - Assert.assertTrue(fired2.get()); - Assert.assertFalse(scheduler.hasScheduler()); + Assertions.assertTrue(fired2.get()); + Assertions.assertFalse(scheduler.hasScheduler()); } errorHandler.assertNoError(); } - @Test(timeout = 1000) + @Test + @Timeout(value = 1000) public void testRestartingScheduler() throws Exception { final TimeoutScheduler scheduler = TimeoutScheduler.newInstance(); final TimeDuration grace = TimeDuration.valueOf(100, TimeUnit.MILLISECONDS); scheduler.setGracePeriod(grace); - Assert.assertFalse(scheduler.hasScheduler()); + Assertions.assertFalse(scheduler.hasScheduler()); final ErrorHandler errorHandler = new ErrorHandler(); for(int i = 0; i < 2; i++) { final AtomicBoolean fired = new AtomicBoolean(false); scheduler.onTimeout(TimeDuration.valueOf(150, TimeUnit.MILLISECONDS), () -> { - Assert.assertFalse(fired.get()); + Assertions.assertFalse(fired.get()); fired.set(true); }, errorHandler); - Assert.assertTrue(scheduler.hasScheduler()); + Assertions.assertTrue(scheduler.hasScheduler()); Thread.sleep(100); - Assert.assertFalse(fired.get()); - Assert.assertTrue(scheduler.hasScheduler()); + Assertions.assertFalse(fired.get()); + Assertions.assertTrue(scheduler.hasScheduler()); Thread.sleep(100); - Assert.assertTrue(fired.get()); - Assert.assertTrue(scheduler.hasScheduler()); + Assertions.assertTrue(fired.get()); + Assertions.assertTrue(scheduler.hasScheduler()); Thread.sleep(100); - Assert.assertTrue(fired.get()); - Assert.assertFalse(scheduler.hasScheduler()); + Assertions.assertTrue(fired.get()); + Assertions.assertFalse(scheduler.hasScheduler()); } errorHandler.assertNoError(); } - @Test(timeout = 10_000) + @Test + @Timeout(value = 10_000) public void testShutdown() throws Exception { final TimeoutScheduler scheduler = TimeoutScheduler.newInstance(); - Assert.assertEquals(TimeoutScheduler.DEFAULT_GRACE_PERIOD, scheduler.getGracePeriod()); + Assertions.assertEquals(TimeoutScheduler.DEFAULT_GRACE_PERIOD, scheduler.getGracePeriod()); final ErrorHandler errorHandler = new ErrorHandler(); final int numTasks = 100; @@ -223,7 +229,7 @@ public void testShutdown() throws Exception { } HUNDRED_MILLIS.sleep(); HUNDRED_MILLIS.sleep(); - JavaUtils.attempt(() -> Assert.assertEquals(1, scheduler.getTaskCount()), + JavaUtils.attempt(() -> Assertions.assertEquals(1, scheduler.getTaskCount()), 10, HUNDRED_MILLIS, "only 1 shutdown task is scheduled", LOG); final TimeDuration oneMillis = TimeDuration.valueOf(1, TimeUnit.MILLISECONDS); @@ -234,7 +240,7 @@ public void testShutdown() throws Exception { oneMillis.sleep(); } HUNDRED_MILLIS.sleep(); - JavaUtils.attempt(() -> Assert.assertEquals(1, scheduler.getTaskCount()), + JavaUtils.attempt(() -> Assertions.assertEquals(1, scheduler.getTaskCount()), 10, HUNDRED_MILLIS, "only 1 shutdown task is scheduled", LOG); errorHandler.assertNoError(); diff --git a/ratis-test/src/test/java/org/apache/ratis/util/TestTraditionalBinaryPrefix.java b/ratis-test/src/test/java/org/apache/ratis/util/TestTraditionalBinaryPrefix.java index 8b6b7a378f..988cc5c710 100644 --- a/ratis-test/src/test/java/org/apache/ratis/util/TestTraditionalBinaryPrefix.java +++ b/ratis-test/src/test/java/org/apache/ratis/util/TestTraditionalBinaryPrefix.java @@ -17,15 +17,18 @@ */ package org.apache.ratis.util; -import org.junit.Test; + +import org.junit.jupiter.api.Test; +import org.junit.jupiter.api.Timeout; import static org.apache.ratis.util.TraditionalBinaryPrefix.long2String; import static org.apache.ratis.util.TraditionalBinaryPrefix.string2long; -import static org.junit.Assert.assertEquals; -import static org.junit.Assert.fail; +import static org.junit.jupiter.api.Assertions.assertEquals; +import static org.junit.jupiter.api.Assertions.fail; public class TestTraditionalBinaryPrefix { - @Test(timeout = 1000) + @Test + @Timeout(value = 1000) public void testTraditionalBinaryPrefix() { //test string2long(..) String[] symbol = {"k", "m", "g", "t", "p", "e"}; @@ -108,19 +111,19 @@ public void testTraditionalBinaryPrefix() { { // n = 2^e final long n = 1L << e; final String expected = (n/p.getValue()) + " " + p.getSymbol(); - assertEquals("n=" + n, expected, long2String(n, null, 2)); + assertEquals(expected, long2String(n, null, 2), "n=" + n); } { // n = 2^e + 1 final long n = (1L << e) + 1; final String expected = (n/p.getValue()) + trailingZeros + p.getSymbol(); - assertEquals("n=" + n, expected, long2String(n, null, decimalPlace)); + assertEquals(expected, long2String(n, null, decimalPlace), "n=" + n); } { // n = 2^e - 1 final long n = (1L << e) - 1; final String expected = ((n+1)/p.getValue()) + trailingZeros + p.getSymbol(); - assertEquals("n=" + n, expected, long2String(n, null, decimalPlace)); + assertEquals(expected, long2String(n, null, decimalPlace), "n=" + n); } } } @@ -143,7 +146,8 @@ private static String byteDescription(long len) { return long2String(len, "B", 2); } - @Test(timeout = 1000) + @Test + @Timeout(value = 1000) public void testUnderscore() { final SizeInBytes value = SizeInBytes.valueOf("1_000_000_000_000_000"); assertEquals(1_000_000_000_000_000L, value.getSize()); From fe037eac32323a3d895874a19d807a19724d7cae Mon Sep 17 00:00:00 2001 From: Tsz-Wo Nicholas Sze Date: Wed, 21 Feb 2024 03:32:07 -0800 Subject: [PATCH 006/397] RATIS-2027. Ratis Streaming: Remote Stream copy data to heap. (#1044) --- .../client/impl/DataStreamClientImpl.java | 7 ++++++- .../ratis/client/impl/OrderedStreamAsync.java | 4 ++++ .../impl}/DataStreamRequestByteBuf.java | 3 +-- .../ratis/netty/NettyDataStreamUtils.java | 16 +++++++++++++++- .../netty/client/NettyClientStreamRpc.java | 12 ++++++++++++ .../netty/server/DataStreamManagement.java | 19 ++++++++++--------- .../netty/server/NettyServerStreamRpc.java | 12 +++++++----- 7 files changed, 55 insertions(+), 18 deletions(-) rename {ratis-netty/src/main/java/org/apache/ratis/netty/server => ratis-common/src/main/java/org/apache/ratis/datastream/impl}/DataStreamRequestByteBuf.java (96%) diff --git a/ratis-client/src/main/java/org/apache/ratis/client/impl/DataStreamClientImpl.java b/ratis-client/src/main/java/org/apache/ratis/client/impl/DataStreamClientImpl.java index 26d01c356f..ba91866d71 100644 --- a/ratis-client/src/main/java/org/apache/ratis/client/impl/DataStreamClientImpl.java +++ b/ratis-client/src/main/java/org/apache/ratis/client/impl/DataStreamClientImpl.java @@ -40,6 +40,7 @@ import org.apache.ratis.protocol.RaftPeer; import org.apache.ratis.protocol.exceptions.AlreadyClosedException; import org.apache.ratis.rpc.CallId; +import org.apache.ratis.thirdparty.io.netty.buffer.ByteBuf; import org.apache.ratis.util.IOUtils; import org.apache.ratis.protocol.*; import org.apache.ratis.thirdparty.com.google.protobuf.ByteString; @@ -169,6 +170,10 @@ private CompletableFuture writeAsyncImpl(Object data, long leng return f; } + public CompletableFuture writeAsync(ByteBuf src, Iterable options) { + return writeAsyncImpl(src, src.readableBytes(), options); + } + @Override public CompletableFuture writeAsync(ByteBuffer src, Iterable options) { return writeAsyncImpl(src, src.remaining(), options); @@ -235,7 +240,7 @@ public DataStreamClientRpc getClientRpc() { } @Override - public DataStreamOutputRpc stream(RaftClientRequest request) { + public DataStreamOutputImpl stream(RaftClientRequest request) { return new DataStreamOutputImpl(request); } diff --git a/ratis-client/src/main/java/org/apache/ratis/client/impl/OrderedStreamAsync.java b/ratis-client/src/main/java/org/apache/ratis/client/impl/OrderedStreamAsync.java index 989c00cbbc..275755514f 100644 --- a/ratis-client/src/main/java/org/apache/ratis/client/impl/OrderedStreamAsync.java +++ b/ratis-client/src/main/java/org/apache/ratis/client/impl/OrderedStreamAsync.java @@ -21,12 +21,14 @@ import org.apache.ratis.client.RaftClientConfigKeys; import org.apache.ratis.conf.RaftProperties; import org.apache.ratis.datastream.impl.DataStreamPacketByteBuffer; +import org.apache.ratis.datastream.impl.DataStreamRequestByteBuf; import org.apache.ratis.datastream.impl.DataStreamRequestByteBuffer; import org.apache.ratis.datastream.impl.DataStreamRequestFilePositionCount; import org.apache.ratis.io.FilePositionCount; import org.apache.ratis.protocol.DataStreamReply; import org.apache.ratis.protocol.DataStreamRequest; import org.apache.ratis.protocol.DataStreamRequestHeader; +import org.apache.ratis.thirdparty.io.netty.buffer.ByteBuf; import org.apache.ratis.util.IOUtils; import org.apache.ratis.util.JavaUtils; import org.apache.ratis.util.SlidingWindow; @@ -56,6 +58,8 @@ static class DataStreamWindowRequest implements SlidingWindow.ClientSideRequest< DataStreamRequest getDataStreamRequest() { if (header.getDataLength() == 0) { return new DataStreamRequestByteBuffer(header, DataStreamPacketByteBuffer.EMPTY_BYTE_BUFFER); + } else if (data instanceof ByteBuf) { + return new DataStreamRequestByteBuf(header, (ByteBuf)data); } else if (data instanceof ByteBuffer) { return new DataStreamRequestByteBuffer(header, (ByteBuffer)data); } else if (data instanceof FilePositionCount) { diff --git a/ratis-netty/src/main/java/org/apache/ratis/netty/server/DataStreamRequestByteBuf.java b/ratis-common/src/main/java/org/apache/ratis/datastream/impl/DataStreamRequestByteBuf.java similarity index 96% rename from ratis-netty/src/main/java/org/apache/ratis/netty/server/DataStreamRequestByteBuf.java rename to ratis-common/src/main/java/org/apache/ratis/datastream/impl/DataStreamRequestByteBuf.java index 2542b1ec6f..1873bec9b4 100644 --- a/ratis-netty/src/main/java/org/apache/ratis/netty/server/DataStreamRequestByteBuf.java +++ b/ratis-common/src/main/java/org/apache/ratis/datastream/impl/DataStreamRequestByteBuf.java @@ -16,9 +16,8 @@ * limitations under the License. */ -package org.apache.ratis.netty.server; +package org.apache.ratis.datastream.impl; -import org.apache.ratis.datastream.impl.DataStreamPacketImpl; import org.apache.ratis.io.WriteOption; import org.apache.ratis.proto.RaftProtos.DataStreamPacketHeaderProto.Type; import org.apache.ratis.protocol.ClientId; diff --git a/ratis-netty/src/main/java/org/apache/ratis/netty/NettyDataStreamUtils.java b/ratis-netty/src/main/java/org/apache/ratis/netty/NettyDataStreamUtils.java index bd03fefcc3..aa46cba53f 100644 --- a/ratis-netty/src/main/java/org/apache/ratis/netty/NettyDataStreamUtils.java +++ b/ratis-netty/src/main/java/org/apache/ratis/netty/NettyDataStreamUtils.java @@ -23,7 +23,7 @@ import org.apache.ratis.io.FilePositionCount; import org.apache.ratis.io.StandardWriteOption; import org.apache.ratis.io.WriteOption; -import org.apache.ratis.netty.server.DataStreamRequestByteBuf; +import org.apache.ratis.datastream.impl.DataStreamRequestByteBuf; import org.apache.ratis.proto.RaftProtos.DataStreamReplyHeaderProto; import org.apache.ratis.proto.RaftProtos.DataStreamRequestHeaderProto; import org.apache.ratis.proto.RaftProtos.DataStreamPacketHeaderProto; @@ -128,6 +128,20 @@ static void encodeByteBuffer(ByteBuffer buffer, Consumer out) { out.accept(Unpooled.wrappedBuffer(buffer)); } + static void encodeDataStreamRequestByteBuf(DataStreamRequestByteBuf request, Consumer out, + ByteBufAllocator allocator) { + encodeDataStreamRequestHeader(request, out, allocator); + encodeByteBuf(request.slice(), out); + } + + static void encodeByteBuf(ByteBuf buffer, Consumer out) { + if (buffer.readableBytes() == 0) { + out.accept(Unpooled.EMPTY_BUFFER); // to avoid EncoderException: must produce at least one message + return; + } + out.accept(buffer); + } + static void encodeDataStreamRequestFilePositionCount( DataStreamRequestFilePositionCount request, Consumer out, ByteBufAllocator allocator) { encodeDataStreamRequestHeader(request, out, allocator); diff --git a/ratis-netty/src/main/java/org/apache/ratis/netty/client/NettyClientStreamRpc.java b/ratis-netty/src/main/java/org/apache/ratis/netty/client/NettyClientStreamRpc.java index 020acc2fdb..b2dc3812f2 100644 --- a/ratis-netty/src/main/java/org/apache/ratis/netty/client/NettyClientStreamRpc.java +++ b/ratis-netty/src/main/java/org/apache/ratis/netty/client/NettyClientStreamRpc.java @@ -21,6 +21,7 @@ import org.apache.ratis.client.DataStreamClientRpc; import org.apache.ratis.client.RaftClientConfigKeys; import org.apache.ratis.conf.RaftProperties; +import org.apache.ratis.datastream.impl.DataStreamRequestByteBuf; import org.apache.ratis.datastream.impl.DataStreamRequestByteBuffer; import org.apache.ratis.datastream.impl.DataStreamRequestFilePositionCount; import org.apache.ratis.io.StandardWriteOption; @@ -370,6 +371,7 @@ public void initChannel(SocketChannel ch) { p.addLast(ENCODER); p.addLast(ENCODER_FILE_POSITION_COUNT); p.addLast(ENCODER_BYTE_BUFFER); + p.addLast(ENCODER_BYTE_BUF); p.addLast(newDecoder()); p.addLast(handler); } @@ -386,6 +388,16 @@ protected void encode(ChannelHandlerContext context, DataStreamRequestByteBuffer } } + static final MessageToMessageEncoder ENCODER_BYTE_BUF = new EncoderByteBuf(); + + @ChannelHandler.Sharable + static class EncoderByteBuf extends MessageToMessageEncoder { + @Override + protected void encode(ChannelHandlerContext context, DataStreamRequestByteBuf request, List out) { + NettyDataStreamUtils.encodeDataStreamRequestByteBuf(request, out::add, context.alloc()); + } + } + static final MessageToMessageEncoder ENCODER_FILE_POSITION_COUNT = new EncoderFilePositionCount(); diff --git a/ratis-netty/src/main/java/org/apache/ratis/netty/server/DataStreamManagement.java b/ratis-netty/src/main/java/org/apache/ratis/netty/server/DataStreamManagement.java index 276a365cea..302aed998e 100644 --- a/ratis-netty/src/main/java/org/apache/ratis/netty/server/DataStreamManagement.java +++ b/ratis-netty/src/main/java/org/apache/ratis/netty/server/DataStreamManagement.java @@ -18,10 +18,11 @@ package org.apache.ratis.netty.server; -import org.apache.ratis.client.DataStreamOutputRpc; import org.apache.ratis.client.impl.ClientProtoUtils; +import org.apache.ratis.client.impl.DataStreamClientImpl.DataStreamOutputImpl; import org.apache.ratis.conf.RaftProperties; import org.apache.ratis.datastream.impl.DataStreamReplyByteBuffer; +import org.apache.ratis.datastream.impl.DataStreamRequestByteBuf; import org.apache.ratis.io.StandardWriteOption; import org.apache.ratis.io.WriteOption; import org.apache.ratis.metrics.Timekeeper; @@ -111,12 +112,12 @@ void cleanUp() { } static class RemoteStream { - private final DataStreamOutputRpc out; + private final DataStreamOutputImpl out; private final AtomicReference> sendFuture = new AtomicReference<>(CompletableFuture.completedFuture(null)); private final RequestMetrics metrics; - RemoteStream(DataStreamOutputRpc out, RequestMetrics metrics) { + RemoteStream(DataStreamOutputImpl out, RequestMetrics metrics) { this.metrics = metrics; this.out = out; } @@ -132,7 +133,7 @@ static Iterable addFlush(List original) { CompletableFuture write(DataStreamRequestByteBuf request, Executor executor) { final Timekeeper.Context context = metrics.start(); return composeAsync(sendFuture, executor, - n -> out.writeAsync(request.slice().nioBuffer(), addFlush(request.getWriteOptionList())) + n -> out.writeAsync(request.slice().retain(), addFlush(request.getWriteOptionList())) .whenComplete((l, e) -> metrics.stop(context, e == null))); } } @@ -147,7 +148,7 @@ static class StreamInfo { = new AtomicReference<>(CompletableFuture.completedFuture(null)); StreamInfo(RaftClientRequest request, boolean primary, CompletableFuture stream, RaftServer server, - CheckedBiFunction, Set, IOException> getStreams, + CheckedBiFunction, Set, IOException> getStreams, Function metricsConstructor) throws IOException { this.request = request; @@ -155,7 +156,7 @@ static class StreamInfo { this.local = new LocalStream(stream, metricsConstructor.apply(RequestType.LOCAL_WRITE)); this.server = server; final Set successors = getSuccessors(server.getId()); - final Set outs = getStreams.apply(request, successors); + final Set outs = getStreams.apply(request, successors); this.remotes = outs.stream() .map(o -> new RemoteStream(o, metricsConstructor.apply(RequestType.REMOTE_WRITE))) .collect(Collectors.toSet()); @@ -315,7 +316,7 @@ private CompletableFuture computeDataStreamIfAbsent(RaftClientReques } private StreamInfo newStreamInfo(ByteBuf buf, - CheckedBiFunction, Set, IOException> getStreams) { + CheckedBiFunction, Set, IOException> getStreams) { try { final RaftClientRequest request = ClientProtoUtils.toRaftClientRequest( RaftClientRequestProto.parseFrom(buf.nioBuffer())); @@ -449,7 +450,7 @@ void cleanUpOnChannelInactive(ChannelId channelId, TimeDuration channelInactiveG } void read(DataStreamRequestByteBuf request, ChannelHandlerContext ctx, - CheckedBiFunction, Set, IOException> getStreams) { + CheckedBiFunction, Set, IOException> getStreams) { LOG.debug("{}: read {}", this, request); try { readImpl(request, ctx, getStreams); @@ -459,7 +460,7 @@ void read(DataStreamRequestByteBuf request, ChannelHandlerContext ctx, } private void readImpl(DataStreamRequestByteBuf request, ChannelHandlerContext ctx, - CheckedBiFunction, Set, IOException> getStreams) { + CheckedBiFunction, Set, IOException> getStreams) { final boolean close = request.getWriteOptionList().contains(StandardWriteOption.CLOSE); ClientInvocationId key = ClientInvocationId.valueOf(request.getClientId(), request.getStreamId()); diff --git a/ratis-netty/src/main/java/org/apache/ratis/netty/server/NettyServerStreamRpc.java b/ratis-netty/src/main/java/org/apache/ratis/netty/server/NettyServerStreamRpc.java index c5f24b0587..451040bb62 100644 --- a/ratis-netty/src/main/java/org/apache/ratis/netty/server/NettyServerStreamRpc.java +++ b/ratis-netty/src/main/java/org/apache/ratis/netty/server/NettyServerStreamRpc.java @@ -20,9 +20,11 @@ import org.apache.ratis.client.DataStreamClient; import org.apache.ratis.client.DataStreamOutputRpc; +import org.apache.ratis.client.impl.DataStreamClientImpl.DataStreamOutputImpl; import org.apache.ratis.conf.Parameters; import org.apache.ratis.conf.RaftProperties; import org.apache.ratis.datastream.impl.DataStreamReplyByteBuffer; +import org.apache.ratis.datastream.impl.DataStreamRequestByteBuf; import org.apache.ratis.netty.NettyConfigKeys; import org.apache.ratis.netty.NettyDataStreamUtils; import org.apache.ratis.netty.NettyUtils; @@ -90,8 +92,8 @@ void addPeers(Collection newPeers) { map.addRaftPeers(newPeers); } - Set getDataStreamOutput(RaftClientRequest request, Set peers) throws IOException { - final Set outs = new HashSet<>(); + Set getDataStreamOutput(RaftClientRequest request, Set peers) throws IOException { + final Set outs = new HashSet<>(); try { getDataStreamOutput(request, peers, outs); } catch (IOException e) { @@ -101,11 +103,11 @@ Set getDataStreamOutput(RaftClientRequest request, Set peers, Set outs) + private void getDataStreamOutput(RaftClientRequest request, Set peers, Set outs) throws IOException { for (RaftPeer peer : peers) { try { - outs.add((DataStreamOutputRpc) map.computeIfAbsent(peer).get().stream(request)); + outs.add((DataStreamOutputImpl) map.computeIfAbsent(peer).get().stream(request)); } catch (IOException e) { map.handleException(peer.getId(), e, true); throw new IOException(map.getName() + ": Failed to getDataStreamOutput for " + peer, e); @@ -238,7 +240,7 @@ public void channelRead(ChannelHandlerContext ctx, Object msg) { } @Override - public void channelInactive(ChannelHandlerContext ctx) throws Exception { + public void channelInactive(ChannelHandlerContext ctx) { requests.cleanUpOnChannelInactive(ctx.channel().id(), channelInactiveGracePeriod); } From 54cdff2bc25200aa1752082a7096046f8cf777a5 Mon Sep 17 00:00:00 2001 From: Tsz-Wo Nicholas Sze Date: Fri, 1 Mar 2024 10:52:17 -0800 Subject: [PATCH 007/397] RATIS-2038. IllegalStateException when ServerState is closed before start. (#1048) --- .../src/main/java/org/apache/ratis/server/impl/ServerState.java | 2 +- .../java/org/apache/ratis/server/impl/StateMachineUpdater.java | 2 ++ 2 files changed, 3 insertions(+), 1 deletion(-) diff --git a/ratis-server/src/main/java/org/apache/ratis/server/impl/ServerState.java b/ratis-server/src/main/java/org/apache/ratis/server/impl/ServerState.java index d02994ec54..3099b5b840 100644 --- a/ratis-server/src/main/java/org/apache/ratis/server/impl/ServerState.java +++ b/ratis-server/src/main/java/org/apache/ratis/server/impl/ServerState.java @@ -158,6 +158,7 @@ void writeRaftConfiguration(LogEntryProto conf) { } void start() { + // initialize stateMachineUpdater stateMachineUpdater.get().start(); } @@ -421,7 +422,6 @@ void close() { } LOG.warn(getMemberId() + ": Failed to join " + getStateMachineUpdater(), e); } - LOG.info("{}: applyIndex: {}", getMemberId(), getLastAppliedIndex()); try { if (log.isInitialized()) { diff --git a/ratis-server/src/main/java/org/apache/ratis/server/impl/StateMachineUpdater.java b/ratis-server/src/main/java/org/apache/ratis/server/impl/StateMachineUpdater.java index 43fbdd8843..5f6e972e2c 100644 --- a/ratis-server/src/main/java/org/apache/ratis/server/impl/StateMachineUpdater.java +++ b/ratis-server/src/main/java/org/apache/ratis/server/impl/StateMachineUpdater.java @@ -133,6 +133,8 @@ void start() { private void stop() { state = State.STOP; try { + LOG.info("{}: closing {}, lastApplied={}", name, + JavaUtils.getClassSimpleName(stateMachine.getClass()), stateMachine.getLastAppliedTermIndex()); stateMachine.close(); if (stateMachineMetrics.isInitialized()) { stateMachineMetrics.get().unregister(); From f24172de3178aaf275d91a7fd9c15ff22382d55d Mon Sep 17 00:00:00 2001 From: Tsz-Wo Nicholas Sze Date: Wed, 6 Mar 2024 22:29:50 -0800 Subject: [PATCH 008/397] RATIS-2035. Refactor streaming code for Read. (#1046) --- .../apache/ratis/netty/server/ChannelMap.java | 48 ++++++++++++++++ .../netty/server/DataStreamManagement.java | 47 +--------------- .../apache/ratis/netty/server/StreamMap.java | 55 +++++++++++++++++++ .../ratis/server/DataStreamServerRpc.java | 10 +--- .../apache/ratis/server/RaftServerRpc.java | 9 +-- .../org/apache/ratis/server/ServerRpc.java | 33 +++++++++++ .../ratis/datastream/DataStreamBaseTest.java | 4 +- 7 files changed, 141 insertions(+), 65 deletions(-) create mode 100644 ratis-netty/src/main/java/org/apache/ratis/netty/server/ChannelMap.java create mode 100644 ratis-netty/src/main/java/org/apache/ratis/netty/server/StreamMap.java create mode 100644 ratis-server-api/src/main/java/org/apache/ratis/server/ServerRpc.java diff --git a/ratis-netty/src/main/java/org/apache/ratis/netty/server/ChannelMap.java b/ratis-netty/src/main/java/org/apache/ratis/netty/server/ChannelMap.java new file mode 100644 index 0000000000..7b0d761840 --- /dev/null +++ b/ratis-netty/src/main/java/org/apache/ratis/netty/server/ChannelMap.java @@ -0,0 +1,48 @@ +/* + * 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.ratis.netty.server; + +import org.apache.ratis.protocol.ClientInvocationId; +import org.apache.ratis.thirdparty.io.netty.channel.ChannelId; + +import java.util.Collections; +import java.util.Map; +import java.util.Optional; +import java.util.Set; +import java.util.concurrent.ConcurrentHashMap; + +/** Map: {@link ChannelId} -> {@link ClientInvocationId}s. */ +class ChannelMap { + private final Map> map = new ConcurrentHashMap<>(); + + void add(ChannelId channelId, ClientInvocationId clientInvocationId) { + map.computeIfAbsent(channelId, (e) -> new ConcurrentHashMap<>()) + .put(clientInvocationId, clientInvocationId); + } + + void remove(ChannelId channelId, ClientInvocationId clientInvocationId) { + Optional.ofNullable(map.get(channelId)) + .ifPresent((ids) -> ids.remove(clientInvocationId)); + } + + Set remove(ChannelId channelId) { + return Optional.ofNullable(map.remove(channelId)) + .map(Map::keySet) + .orElse(Collections.emptySet()); + } +} diff --git a/ratis-netty/src/main/java/org/apache/ratis/netty/server/DataStreamManagement.java b/ratis-netty/src/main/java/org/apache/ratis/netty/server/DataStreamManagement.java index 302aed998e..a6e9b815ee 100644 --- a/ratis-netty/src/main/java/org/apache/ratis/netty/server/DataStreamManagement.java +++ b/ratis-netty/src/main/java/org/apache/ratis/netty/server/DataStreamManagement.java @@ -70,13 +70,10 @@ import java.util.Collection; import java.util.Collections; import java.util.List; -import java.util.Map; import java.util.Optional; import java.util.Set; import java.util.concurrent.CompletableFuture; import java.util.concurrent.CompletionException; -import java.util.concurrent.ConcurrentHashMap; -import java.util.concurrent.ConcurrentMap; import java.util.concurrent.Executor; import java.util.concurrent.ExecutorService; import java.util.concurrent.atomic.AtomicReference; @@ -219,52 +216,10 @@ private Set getSuccessors(RaftPeerId peerId) throws IOException { } } - static class StreamMap { - private final ConcurrentMap map = new ConcurrentHashMap<>(); - - StreamInfo computeIfAbsent(ClientInvocationId key, Function function) { - final StreamInfo info = map.computeIfAbsent(key, function); - LOG.debug("computeIfAbsent({}) returns {}", key, info); - return info; - } - - StreamInfo get(ClientInvocationId key) { - final StreamInfo info = map.get(key); - LOG.debug("get({}) returns {}", key, info); - return info; - } - - StreamInfo remove(ClientInvocationId key) { - final StreamInfo info = map.remove(key); - LOG.debug("remove({}) returns {}", key, info); - return info; - } - } - - public static class ChannelMap { - private final Map> map = new ConcurrentHashMap<>(); - - public void add(ChannelId channelId, - ClientInvocationId clientInvocationId) { - map.computeIfAbsent(channelId, (e) -> new ConcurrentHashMap<>()).put(clientInvocationId, clientInvocationId); - } - - public void remove(ChannelId channelId, - ClientInvocationId clientInvocationId) { - Optional.ofNullable(map.get(channelId)).ifPresent((ids) -> ids.remove(clientInvocationId)); - } - - public Set remove(ChannelId channelId) { - return Optional.ofNullable(map.remove(channelId)) - .map(Map::keySet) - .orElse(Collections.emptySet()); - } - } - private final RaftServer server; private final String name; - private final StreamMap streams = new StreamMap(); + private final StreamMap streams = new StreamMap<>(); private final ChannelMap channels; private final ExecutorService requestExecutor; private final ExecutorService writeExecutor; diff --git a/ratis-netty/src/main/java/org/apache/ratis/netty/server/StreamMap.java b/ratis-netty/src/main/java/org/apache/ratis/netty/server/StreamMap.java new file mode 100644 index 0000000000..073698cb80 --- /dev/null +++ b/ratis-netty/src/main/java/org/apache/ratis/netty/server/StreamMap.java @@ -0,0 +1,55 @@ +/* + * 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.ratis.netty.server; + +import org.apache.ratis.protocol.ClientInvocationId; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.util.concurrent.ConcurrentHashMap; +import java.util.concurrent.ConcurrentMap; +import java.util.function.Function; + +/** + * Map: {@link ClientInvocationId} -> {@link STREAM}. + * + * @param the stream type. + */ +class StreamMap { + public static final Logger LOG = LoggerFactory.getLogger(StreamMap.class); + + private final ConcurrentMap map = new ConcurrentHashMap<>(); + + STREAM computeIfAbsent(ClientInvocationId key, Function function) { + final STREAM info = map.computeIfAbsent(key, function); + LOG.debug("computeIfAbsent({}) returns {}", key, info); + return info; + } + + STREAM get(ClientInvocationId key) { + final STREAM info = map.get(key); + LOG.debug("get({}) returns {}", key, info); + return info; + } + + STREAM remove(ClientInvocationId key) { + final STREAM info = map.remove(key); + LOG.debug("remove({}) returns {}", key, info); + return info; + } +} diff --git a/ratis-server-api/src/main/java/org/apache/ratis/server/DataStreamServerRpc.java b/ratis-server-api/src/main/java/org/apache/ratis/server/DataStreamServerRpc.java index 4e948c6f81..6316ef6073 100644 --- a/ratis-server-api/src/main/java/org/apache/ratis/server/DataStreamServerRpc.java +++ b/ratis-server-api/src/main/java/org/apache/ratis/server/DataStreamServerRpc.java @@ -20,18 +20,10 @@ import org.apache.ratis.protocol.RaftPeer; import java.io.Closeable; -import java.net.InetSocketAddress; /** * A server interface handling incoming streams * Relays those streams to other servers after persisting */ -public interface DataStreamServerRpc extends RaftPeer.Add, Closeable { - /** - * start server - */ - void start(); - - /** @return the address where this RPC server is listening to. */ - InetSocketAddress getInetSocketAddress(); +public interface DataStreamServerRpc extends ServerRpc, RaftPeer.Add, Closeable { } diff --git a/ratis-server-api/src/main/java/org/apache/ratis/server/RaftServerRpc.java b/ratis-server-api/src/main/java/org/apache/ratis/server/RaftServerRpc.java index d81f9cc8bc..76bd817f5e 100644 --- a/ratis-server-api/src/main/java/org/apache/ratis/server/RaftServerRpc.java +++ b/ratis-server-api/src/main/java/org/apache/ratis/server/RaftServerRpc.java @@ -26,20 +26,13 @@ import org.apache.ratis.util.JavaUtils; import java.io.Closeable; -import java.io.IOException; import java.net.InetSocketAddress; /** * An server-side interface for supporting different RPC implementations * such as Netty, gRPC and Hadoop. */ -public interface RaftServerRpc extends RaftServerProtocol, RpcType.Get, RaftPeer.Add, Closeable { - /** Start the RPC service. */ - void start() throws IOException; - - /** @return the address where this RPC server is listening */ - InetSocketAddress getInetSocketAddress(); - +public interface RaftServerRpc extends RaftServerProtocol, ServerRpc, RpcType.Get, RaftPeer.Add, Closeable { /** @return the address where this RPC server is listening for client requests */ default InetSocketAddress getClientServerAddress() { return getInetSocketAddress(); diff --git a/ratis-server-api/src/main/java/org/apache/ratis/server/ServerRpc.java b/ratis-server-api/src/main/java/org/apache/ratis/server/ServerRpc.java new file mode 100644 index 0000000000..6ad5eacf14 --- /dev/null +++ b/ratis-server-api/src/main/java/org/apache/ratis/server/ServerRpc.java @@ -0,0 +1,33 @@ +/* + * 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.ratis.server; + +import java.io.Closeable; +import java.io.IOException; +import java.net.InetSocketAddress; + +/** + * A general server interface. + */ +public interface ServerRpc extends Closeable { + /** Start the RPC service. */ + void start() throws IOException; + + /** @return the address where this RPC server is listening to. */ + InetSocketAddress getInetSocketAddress(); +} diff --git a/ratis-test/src/test/java/org/apache/ratis/datastream/DataStreamBaseTest.java b/ratis-test/src/test/java/org/apache/ratis/datastream/DataStreamBaseTest.java index 70e26af249..2ac01ac1f5 100644 --- a/ratis-test/src/test/java/org/apache/ratis/datastream/DataStreamBaseTest.java +++ b/ratis-test/src/test/java/org/apache/ratis/datastream/DataStreamBaseTest.java @@ -67,7 +67,7 @@ RaftServer getRaftServer() { return raftServer; } - void start() { + void start() throws IOException { dataStreamServer.getServerRpc().start(); } @@ -90,7 +90,7 @@ Server getPrimaryServer() { return servers.get(0); } - void setup(RaftGroupId groupId, List peers, List raftServers) { + void setup(RaftGroupId groupId, List peers, List raftServers) throws Exception { raftGroup = RaftGroup.valueOf(groupId, peers); this.peers = peers; servers = new ArrayList<>(peers.size()); From 5951acd467ee804ce6617355e5f8735d3e2cfd1e Mon Sep 17 00:00:00 2001 From: DaveTeng0 <109315747+DaveTeng0@users.noreply.github.com> Date: Thu, 7 Mar 2024 16:37:16 -0800 Subject: [PATCH 009/397] Ratis-2031. Add peer info to response of GroupInfoCommand CLI (#1047) --- .../ratis/client/impl/ClientProtoUtils.java | 4 +- .../apache/ratis/protocol/GroupInfoReply.java | 13 +++-- ratis-proto/src/main/proto/Raft.proto | 10 ++++ .../ratis/server/impl/RaftServerImpl.java | 16 +++++- .../shell/cli/sh/group/GroupInfoCommand.java | 1 + .../cli/sh/GroupCommandIntegrationTest.java | 49 +++++++++++++++++++ 6 files changed, 88 insertions(+), 5 deletions(-) diff --git a/ratis-client/src/main/java/org/apache/ratis/client/impl/ClientProtoUtils.java b/ratis-client/src/main/java/org/apache/ratis/client/impl/ClientProtoUtils.java index db19831955..003f202bd9 100644 --- a/ratis-client/src/main/java/org/apache/ratis/client/impl/ClientProtoUtils.java +++ b/ratis-client/src/main/java/org/apache/ratis/client/impl/ClientProtoUtils.java @@ -364,6 +364,7 @@ static GroupInfoReplyProto toGroupInfoReplyProto(GroupInfoReply reply) { b.setIsRaftStorageHealthy(reply.isRaftStorageHealthy()); b.setRole(reply.getRoleInfoProto()); b.addAllCommitInfos(reply.getCommitInfos()); + b.setLogInfo(reply.getLogInfoProto()); } } return b.build(); @@ -506,7 +507,8 @@ static GroupInfoReply toGroupInfoReply(GroupInfoReplyProto replyProto) { ProtoUtils.toRaftGroup(replyProto.getGroup()), replyProto.getRole(), replyProto.getIsRaftStorageHealthy(), - replyProto.hasConf()? replyProto.getConf(): null); + replyProto.hasConf()? replyProto.getConf(): null, + replyProto.getLogInfo()); } static Message toMessage(final ClientMessageEntryProto p) { diff --git a/ratis-common/src/main/java/org/apache/ratis/protocol/GroupInfoReply.java b/ratis-common/src/main/java/org/apache/ratis/protocol/GroupInfoReply.java index 632fa65293..bfac81a2b0 100644 --- a/ratis-common/src/main/java/org/apache/ratis/protocol/GroupInfoReply.java +++ b/ratis-common/src/main/java/org/apache/ratis/protocol/GroupInfoReply.java @@ -19,6 +19,7 @@ import org.apache.ratis.proto.RaftProtos.RaftConfigurationProto; import org.apache.ratis.proto.RaftProtos.CommitInfoProto; +import org.apache.ratis.proto.RaftProtos.LogInfoProto; import org.apache.ratis.proto.RaftProtos.RoleInfoProto; import java.util.Collection; @@ -33,25 +34,27 @@ public class GroupInfoReply extends RaftClientReply { private final RoleInfoProto roleInfoProto; private final boolean isRaftStorageHealthy; private final RaftConfigurationProto conf; + private final LogInfoProto logInfoProto; public GroupInfoReply(RaftClientRequest request, Collection commitInfos, RaftGroup group, RoleInfoProto roleInfoProto, boolean isRaftStorageHealthy, - RaftConfigurationProto conf) { + RaftConfigurationProto conf, LogInfoProto logInfoProto) { this(request.getClientId(), request.getServerId(), request.getRaftGroupId(), request.getCallId(), commitInfos, - group, roleInfoProto, isRaftStorageHealthy, conf); + group, roleInfoProto, isRaftStorageHealthy, conf, logInfoProto); } @SuppressWarnings("parameternumber") public GroupInfoReply(ClientId clientId, RaftPeerId serverId, RaftGroupId groupId, long callId, Collection commitInfos, RaftGroup group, RoleInfoProto roleInfoProto, boolean isRaftStorageHealthy, - RaftConfigurationProto conf) { + RaftConfigurationProto conf, LogInfoProto logInfoProto) { super(clientId, serverId, groupId, callId, true, null, null, 0L, commitInfos); this.group = group; this.roleInfoProto = roleInfoProto; this.isRaftStorageHealthy = isRaftStorageHealthy; this.conf = conf; + this.logInfoProto = logInfoProto; } public RaftGroup getGroup() { @@ -69,4 +72,8 @@ public boolean isRaftStorageHealthy() { public Optional getConf() { return Optional.ofNullable(conf); } + + public LogInfoProto getLogInfoProto() { + return logInfoProto; + } } diff --git a/ratis-proto/src/main/proto/Raft.proto b/ratis-proto/src/main/proto/Raft.proto index 586ec1b052..edc57ec65e 100644 --- a/ratis-proto/src/main/proto/Raft.proto +++ b/ratis-proto/src/main/proto/Raft.proto @@ -556,4 +556,14 @@ message GroupInfoReplyProto { bool isRaftStorageHealthy = 4; repeated CommitInfoProto commitInfos = 5; RaftConfigurationProto conf = 6; + LogInfoProto logInfo = 7; +} + +/** Add new LogInfoProto for RATIS-2030, allow GroupInfoCommand to show each server's last committed log, + last applied log, last snapshot log, last entry log.*/ +message LogInfoProto { + TermIndexProto lastSnapshot = 1; + TermIndexProto applied = 2; + TermIndexProto committed = 3; + TermIndexProto lastEntry = 4; } diff --git a/ratis-server/src/main/java/org/apache/ratis/server/impl/RaftServerImpl.java b/ratis-server/src/main/java/org/apache/ratis/server/impl/RaftServerImpl.java index 08ccabac6d..6ad9e181bb 100644 --- a/ratis-server/src/main/java/org/apache/ratis/server/impl/RaftServerImpl.java +++ b/ratis-server/src/main/java/org/apache/ratis/server/impl/RaftServerImpl.java @@ -24,6 +24,7 @@ import org.apache.ratis.proto.RaftProtos.AppendEntriesReplyProto.AppendResult; import org.apache.ratis.proto.RaftProtos.AppendEntriesRequestProto; import org.apache.ratis.proto.RaftProtos.CommitInfoProto; +import org.apache.ratis.proto.RaftProtos.LogInfoProto; import org.apache.ratis.proto.RaftProtos.InstallSnapshotReplyProto; import org.apache.ratis.proto.RaftProtos.InstallSnapshotRequestProto; import org.apache.ratis.proto.RaftProtos.InstallSnapshotResult; @@ -642,7 +643,20 @@ GroupInfoReply getGroupInfo(GroupInfoRequest request) { final RaftConfigurationProto conf = LogProtoUtils.toRaftConfigurationProtoBuilder(getRaftConf()).build(); return new GroupInfoReply(request, getCommitInfos(), getGroup(), getRoleInfoProto(), - dir.isHealthy(), conf); + dir.isHealthy(), conf, getLogInfo()); + } + + LogInfoProto getLogInfo(){ + final RaftLog log = getRaftLog(); + LogInfoProto.Builder logInfoBuilder = LogInfoProto.newBuilder() + .setApplied(getStateMachine().getLastAppliedTermIndex().toProto()) + .setCommitted(log.getTermIndex(log.getLastCommittedIndex()).toProto()) + .setLastEntry(log.getLastEntryTermIndex().toProto()); + final SnapshotInfo snapshot = getStateMachine().getLatestSnapshot(); + if (snapshot != null) { + logInfoBuilder.setLastSnapshot(snapshot.getTermIndex().toProto()); + } + return logInfoBuilder.build(); } RoleInfoProto getRoleInfoProto() { diff --git a/ratis-shell/src/main/java/org/apache/ratis/shell/cli/sh/group/GroupInfoCommand.java b/ratis-shell/src/main/java/org/apache/ratis/shell/cli/sh/group/GroupInfoCommand.java index d2c4e65991..0125440e90 100644 --- a/ratis-shell/src/main/java/org/apache/ratis/shell/cli/sh/group/GroupInfoCommand.java +++ b/ratis-shell/src/main/java/org/apache/ratis/shell/cli/sh/group/GroupInfoCommand.java @@ -54,6 +54,7 @@ public int run(CommandLine cl) throws IOException { printf("leader info: %s(%s)%n%n", leader.getId().toStringUtf8(), leader.getAddress()); } println(reply.getCommitInfos()); + println(reply.getLogInfoProto()); return 0; } diff --git a/ratis-test/src/test/java/org/apache/ratis/shell/cli/sh/GroupCommandIntegrationTest.java b/ratis-test/src/test/java/org/apache/ratis/shell/cli/sh/GroupCommandIntegrationTest.java index 83c05bfddd..6900d4804a 100644 --- a/ratis-test/src/test/java/org/apache/ratis/shell/cli/sh/GroupCommandIntegrationTest.java +++ b/ratis-test/src/test/java/org/apache/ratis/shell/cli/sh/GroupCommandIntegrationTest.java @@ -20,6 +20,7 @@ import org.apache.ratis.RaftTestUtil; import org.apache.ratis.client.RaftClient; import org.apache.ratis.conf.RaftProperties; +import org.apache.ratis.protocol.RaftClientReply; import org.apache.ratis.server.RaftServer; import org.apache.ratis.server.RaftServerConfigKeys; import org.apache.ratis.server.impl.MiniRaftCluster; @@ -90,4 +91,52 @@ void runTestGroupInfoCommand(MiniRaftCluster cluster) throws Exception { String info = result.substring(0, hearder.length()); Assertions.assertEquals(hearder, info); } + + @Test + public void testGroupInfoCommandIncludesCorrectLogInfo() throws Exception { + // set number of server to 1 so that we can make sure which server returns the LogInfoProto + // since information of applied index, snapshot index, and last entry index are not shared between servers + runWithNewCluster(1, this::runTestGroupInfoCommandWithLogInfoVerification); + } + + void runTestGroupInfoCommandWithLogInfoVerification(MiniRaftCluster cluster) throws Exception { + RaftServer.Division leader = RaftTestUtil.waitForLeader(cluster); + + try (final RaftClient client = cluster.createClient(leader.getId())) { + for (int i = 0; i < RaftServerConfigKeys.Snapshot.creationGap(getProperties()); i++) { + RaftClientReply + reply = client.io().send(new RaftTestUtil.SimpleMessage("m" + i)); + Assertions.assertTrue(reply.isSuccess()); + } + } + + leader.getStateMachine().takeSnapshot(); + + final String address = getClusterAddress(cluster); + final StringPrintStream out = new StringPrintStream(); + RatisShell shell = new RatisShell(out.getPrintStream()); + int ret = shell.run("group", "info", "-peers", address); + Assertions.assertEquals(0 , ret); + String result = out.toString().trim(); + String hearder = String.format("group id: %s%sleader info: %s(%s)%s%s", + cluster.getGroupId().getUuid(), NEW_LINE, leader.getId(), + cluster.getLeader().getPeer().getAddress(), NEW_LINE, NEW_LINE); + String info = result.substring(0, hearder.length()); + Assertions.assertEquals(hearder, info); + long currentTerm = leader.getInfo().getCurrentTerm(); + String LogInfoProtoFormat = "%s {" + NEW_LINE + " term: " + currentTerm + NEW_LINE + " index: %s"; + Assertions.assertTrue(result.contains( + String.format(LogInfoProtoFormat, "applied", + leader.getStateMachine().getLastAppliedTermIndex().getIndex()))); + Assertions.assertTrue(result.contains( + String.format(LogInfoProtoFormat, "committed", + leader.getRaftLog().getLastCommittedIndex()))); + Assertions.assertTrue(result.contains( + String.format(LogInfoProtoFormat, "lastSnapshot", + leader.getStateMachine().getLatestSnapshot().getIndex()))); + Assertions.assertTrue(result.contains( + String.format(LogInfoProtoFormat, "lastEntry", + leader.getRaftLog().getLastCommittedIndex()))); + } + } From 07d9bdc260ad2eeba0ce2679824656481a8a56c4 Mon Sep 17 00:00:00 2001 From: Symious <14933944+symious@users.noreply.github.com> Date: Fri, 15 Mar 2024 09:22:03 +0800 Subject: [PATCH 010/397] RATIS-2043. Change toString() of RaftConfigurationImpl (#1051) --- .../org/apache/ratis/server/impl/RaftConfigurationImpl.java | 2 +- .../main/java/org/apache/ratis/server/impl/RaftServerImpl.java | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/ratis-server/src/main/java/org/apache/ratis/server/impl/RaftConfigurationImpl.java b/ratis-server/src/main/java/org/apache/ratis/server/impl/RaftConfigurationImpl.java index d609264af5..98f41020e0 100644 --- a/ratis-server/src/main/java/org/apache/ratis/server/impl/RaftConfigurationImpl.java +++ b/ratis-server/src/main/java/org/apache/ratis/server/impl/RaftConfigurationImpl.java @@ -298,7 +298,7 @@ boolean isSingleton() { @Override public String toString() { - return logEntryIndex + ": " + conf + ", old=" + oldConf; + return "conf: {index: " + logEntryIndex + ", cur=" + conf + ", old=" + oldConf + "}"; } boolean hasNoChange(Collection newMembers, Collection newListeners) { diff --git a/ratis-server/src/main/java/org/apache/ratis/server/impl/RaftServerImpl.java b/ratis-server/src/main/java/org/apache/ratis/server/impl/RaftServerImpl.java index 6ad9e181bb..7cc6ae2eac 100644 --- a/ratis-server/src/main/java/org/apache/ratis/server/impl/RaftServerImpl.java +++ b/ratis-server/src/main/java/org/apache/ratis/server/impl/RaftServerImpl.java @@ -1199,7 +1199,7 @@ CompletableFuture transferLeadershipAsync(TransferLeadershipReq if (!conf.isHighestPriority(request.getNewLeader())) { String msg = getMemberId() + " refused to transfer leadership to peer " + request.getNewLeader() + - " as it does not has highest priority " + conf; + " as it does not has highest priority in " + conf; return logAndReturnTransferLeadershipFail(request, msg); } From 2ece48ee2f90d855cfb5c4b10d0778b3632f7277 Mon Sep 17 00:00:00 2001 From: William Song <48054931+SzyWilliam@users.noreply.github.com> Date: Mon, 18 Mar 2024 23:15:14 +0800 Subject: [PATCH 011/397] RATIS-2044. Fix ReadIndex loss caused by data race in AppendEntriesListeners (#1052) --- .../ratis/server/impl/ReadIndexHeartbeats.java | 12 ++++++++++++ 1 file changed, 12 insertions(+) diff --git a/ratis-server/src/main/java/org/apache/ratis/server/impl/ReadIndexHeartbeats.java b/ratis-server/src/main/java/org/apache/ratis/server/impl/ReadIndexHeartbeats.java index d08a1ea406..4ff1460d7d 100644 --- a/ratis-server/src/main/java/org/apache/ratis/server/impl/ReadIndexHeartbeats.java +++ b/ratis-server/src/main/java/org/apache/ratis/server/impl/ReadIndexHeartbeats.java @@ -23,6 +23,7 @@ import org.apache.ratis.server.raftlog.RaftLog; import org.apache.ratis.server.raftlog.RaftLogIndex; import org.apache.ratis.util.JavaUtils; +import org.apache.ratis.util.Preconditions; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -123,8 +124,15 @@ boolean isAcknowledged(RaftPeerId id) { class AppendEntriesListeners { private final NavigableMap sorted = new TreeMap<>(); + private Exception exception = null; synchronized AppendEntriesListener add(long commitIndex, Function constructor) { + if (exception != null) { + Preconditions.assertTrue(sorted.isEmpty()); + final AppendEntriesListener listener = constructor.apply(commitIndex); + listener.getFuture().completeExceptionally(exception); + return listener; + } return sorted.computeIfAbsent(commitIndex, constructor); } @@ -152,6 +160,10 @@ synchronized void onAppendEntriesReply(LogAppender appender, AppendEntriesReplyP } synchronized void failAll(Exception e) { + if (exception != null) { + return; + } + exception = e; sorted.forEach((index, listener) -> listener.getFuture().completeExceptionally(e)); sorted.clear(); } From 9a7bccefdb4ec05d79f0dadd367722124d33e88a Mon Sep 17 00:00:00 2001 From: Hemant Kumar Date: Wed, 20 Mar 2024 16:26:43 -0700 Subject: [PATCH 012/397] RATIS-2045. SnapshotInstallationHandler doesn't notify follower when snapshotIndex is -1 and firstAvailableLogIndex is 0 (#1053) --- .../apache/ratis/server/impl/SnapshotInstallationHandler.java | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/ratis-server/src/main/java/org/apache/ratis/server/impl/SnapshotInstallationHandler.java b/ratis-server/src/main/java/org/apache/ratis/server/impl/SnapshotInstallationHandler.java index 3e5ac2b671..f03e2d883d 100644 --- a/ratis-server/src/main/java/org/apache/ratis/server/impl/SnapshotInstallationHandler.java +++ b/ratis-server/src/main/java/org/apache/ratis/server/impl/SnapshotInstallationHandler.java @@ -221,7 +221,8 @@ private InstallSnapshotReplyProto notifyStateMachineToInstallSnapshot( // Check if snapshot index is already at par or ahead of the first // available log index of the Leader. final long snapshotIndex = state.getLog().getSnapshotIndex(); - if (snapshotIndex + 1 >= firstAvailableLogIndex && firstAvailableLogIndex > INVALID_LOG_INDEX) { + if (snapshotIndex != INVALID_LOG_INDEX && snapshotIndex + 1 >= firstAvailableLogIndex && + firstAvailableLogIndex > INVALID_LOG_INDEX) { // State Machine has already installed the snapshot. Return the // latest snapshot index to the Leader. From 0d91262573637e66f81da304c802592ee5ea719c Mon Sep 17 00:00:00 2001 From: Potato Date: Sat, 23 Mar 2024 00:53:30 +0800 Subject: [PATCH 013/397] RATIS-2047. Avoid unnecessary warn log when creating raft group (#1054) --- .../ratis/server/storage/RaftStorageImpl.java | 17 +++++++++-------- 1 file changed, 9 insertions(+), 8 deletions(-) diff --git a/ratis-server/src/main/java/org/apache/ratis/server/storage/RaftStorageImpl.java b/ratis-server/src/main/java/org/apache/ratis/server/storage/RaftStorageImpl.java index fbb7bf7d46..ce809cad8f 100644 --- a/ratis-server/src/main/java/org/apache/ratis/server/storage/RaftStorageImpl.java +++ b/ratis-server/src/main/java/org/apache/ratis/server/storage/RaftStorageImpl.java @@ -19,7 +19,6 @@ import java.io.InputStream; import java.io.OutputStream; -import java.nio.file.NoSuchFileException; import java.util.concurrent.atomic.AtomicReference; import org.apache.ratis.proto.RaftProtos.LogEntryProto; import org.apache.ratis.server.RaftConfiguration; @@ -153,14 +152,16 @@ public void writeRaftConfiguration(LogEntryProto conf) { public RaftConfiguration readRaftConfiguration() { File confFile = storageDir.getMetaConfFile(); - try (InputStream fio = FileUtils.newInputStream(confFile)) { - LogEntryProto confProto = LogEntryProto.newBuilder().mergeFrom(fio).build(); - return LogProtoUtils.toRaftConfiguration(confProto); - } catch (FileNotFoundException | NoSuchFileException e) { - return null; - } catch (Exception e) { - LOG.error("Failed reading configuration from file:" + confFile, e); + if (!confFile.exists()) { return null; + } else { + try (InputStream fio = FileUtils.newInputStream(confFile)) { + LogEntryProto confProto = LogEntryProto.newBuilder().mergeFrom(fio).build(); + return LogProtoUtils.toRaftConfiguration(confProto); + } catch (Exception e) { + LOG.error("Failed reading configuration from file:" + confFile, e); + return null; + } } } From 9452a86384dab24aab9ad12271da945521b6a31d Mon Sep 17 00:00:00 2001 From: Potato Date: Sat, 23 Mar 2024 00:58:31 +0800 Subject: [PATCH 014/397] RATIS-2048. Fix SimpleStateMachineStorage potential NPE exception (#1056) --- .../ratis/statemachine/impl/SimpleStateMachineStorage.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/ratis-server/src/main/java/org/apache/ratis/statemachine/impl/SimpleStateMachineStorage.java b/ratis-server/src/main/java/org/apache/ratis/statemachine/impl/SimpleStateMachineStorage.java index 37e94a8a0a..88cc57dab5 100644 --- a/ratis-server/src/main/java/org/apache/ratis/statemachine/impl/SimpleStateMachineStorage.java +++ b/ratis-server/src/main/java/org/apache/ratis/statemachine/impl/SimpleStateMachineStorage.java @@ -116,7 +116,7 @@ public void cleanupOldSnapshots(SnapshotRetentionPolicy snapshotRetentionPolicy) final List allSnapshotFiles = getSingleFileSnapshotInfos(stateMachineDir.toPath()); - if (allSnapshotFiles.size() > snapshotRetentionPolicy.getNumSnapshotsRetained()) { + if (allSnapshotFiles.size() > numSnapshotsRetained) { allSnapshotFiles.sort(Comparator.comparing(SingleFileSnapshotInfo::getIndex).reversed()); allSnapshotFiles.subList(numSnapshotsRetained, allSnapshotFiles.size()) .stream() From 0f221166e259588fd9cd15881dd3d677a61662ec Mon Sep 17 00:00:00 2001 From: Potato Date: Sun, 24 Mar 2024 02:11:39 +0800 Subject: [PATCH 015/397] RATIS-2036. Avoid trigger snapshot when removing raftGroup (#1055) --- .../ratis/server/RaftServerConfigKeys.java | 12 ++++++++ .../ratis/server/impl/RaftServerImpl.java | 1 + .../server/impl/StateMachineUpdater.java | 28 ++++++++++++++++++- 3 files changed, 40 insertions(+), 1 deletion(-) diff --git a/ratis-server-api/src/main/java/org/apache/ratis/server/RaftServerConfigKeys.java b/ratis-server-api/src/main/java/org/apache/ratis/server/RaftServerConfigKeys.java index 565e881269..7419ca095f 100644 --- a/ratis-server-api/src/main/java/org/apache/ratis/server/RaftServerConfigKeys.java +++ b/ratis-server-api/src/main/java/org/apache/ratis/server/RaftServerConfigKeys.java @@ -647,6 +647,18 @@ static void setTriggerWhenStopEnabled(RaftProperties properties, boolean trigger setBoolean(properties::setBoolean, TRIGGER_WHEN_STOP_ENABLED_KEY, triggerWhenStopEnabled); } + /** whether trigger snapshot when remove raft server */ + String TRIGGER_WHEN_REMOVE_ENABLED_KEY = PREFIX + ".trigger-when-remove.enabled"; + /** by default let the state machine to trigger snapshot when remove */ + boolean TRIGGER_WHEN_REMOVE_ENABLED_DEFAULT = true; + static boolean triggerWhenRemoveEnabled(RaftProperties properties) { + return getBoolean(properties::getBoolean, + TRIGGER_WHEN_REMOVE_ENABLED_KEY, TRIGGER_WHEN_REMOVE_ENABLED_DEFAULT, getDefaultLog()); + } + static void setTriggerWhenRemoveEnabled(RaftProperties properties, boolean triggerWhenRemoveEnabled) { + setBoolean(properties::setBoolean, TRIGGER_WHEN_REMOVE_ENABLED_KEY, triggerWhenRemoveEnabled); + } + /** The log index gap between to two snapshot creations. */ String CREATION_GAP_KEY = PREFIX + ".creation.gap"; long CREATION_GAP_DEFAULT = 1024; diff --git a/ratis-server/src/main/java/org/apache/ratis/server/impl/RaftServerImpl.java b/ratis-server/src/main/java/org/apache/ratis/server/impl/RaftServerImpl.java index 7cc6ae2eac..67c90f20bb 100644 --- a/ratis-server/src/main/java/org/apache/ratis/server/impl/RaftServerImpl.java +++ b/ratis-server/src/main/java/org/apache/ratis/server/impl/RaftServerImpl.java @@ -461,6 +461,7 @@ void groupRemove(boolean deleteDirectory, boolean renameDirectory) { final RaftStorageDirectory dir = state.getStorage().getStorageDir(); /* Shutdown is triggered here inorder to avoid any locked files. */ + state.getStateMachineUpdater().setRemoving(); close(); getStateMachine().event().notifyGroupRemove(); if (deleteDirectory) { diff --git a/ratis-server/src/main/java/org/apache/ratis/server/impl/StateMachineUpdater.java b/ratis-server/src/main/java/org/apache/ratis/server/impl/StateMachineUpdater.java index 5f6e972e2c..7c314ef441 100644 --- a/ratis-server/src/main/java/org/apache/ratis/server/impl/StateMachineUpdater.java +++ b/ratis-server/src/main/java/org/apache/ratis/server/impl/StateMachineUpdater.java @@ -74,6 +74,8 @@ enum State { private final boolean triggerSnapshotWhenStopEnabled; + private final boolean triggerSnapshotWhenRemoveEnabled; + private final Long autoSnapshotThreshold; private final boolean purgeUptoSnapshotIndex; @@ -91,6 +93,8 @@ enum State { private final Consumer appliedIndexConsumer; + private volatile boolean isRemoving; + StateMachineUpdater(StateMachine stateMachine, RaftServerImpl server, ServerState serverState, long lastAppliedIndex, RaftProperties properties, Consumer appliedIndexConsumer) { this.name = serverState.getMemberId() + "-" + JavaUtils.getClassSimpleName(getClass()); @@ -106,6 +110,7 @@ enum State { this.snapshotIndex = new RaftLogIndex("snapshotIndex", lastAppliedIndex); this.triggerSnapshotWhenStopEnabled = RaftServerConfigKeys.Snapshot.triggerWhenStopEnabled(properties); + this.triggerSnapshotWhenRemoveEnabled = RaftServerConfigKeys.Snapshot.triggerWhenRemoveEnabled(properties); final boolean autoSnapshot = RaftServerConfigKeys.Snapshot.autoTriggerEnabled(properties); this.autoSnapshotThreshold = autoSnapshot? RaftServerConfigKeys.Snapshot.autoTriggerThreshold(properties): null; final int numSnapshotFilesRetained = RaftServerConfigKeys.Snapshot.retentionFileNum(properties); @@ -322,12 +327,33 @@ private boolean shouldTakeSnapshot() { if (autoSnapshotThreshold == null) { return false; } else if (shouldStop()) { - return triggerSnapshotWhenStopEnabled && getLastAppliedIndex() - snapshotIndex.get() > 0; + return shouldTakeSnapshotAtStop() && getLastAppliedIndex() - snapshotIndex.get() > 0; } return state == State.RUNNING && getStateMachineLastAppliedIndex() - snapshotIndex.get() >= autoSnapshotThreshold; } + /** + * In view of the three variables triggerSnapshotWhenStopEnabled, triggerSnapshotWhenRemoveEnabled and isRemoving, + * we can draw the following 8 combination: + * true true true => true + * true true false => true + * true false true => false + * true false false => true + * false true true => true + * false true false => false + * false false true => false + * false false false => false + * @return result + */ + private boolean shouldTakeSnapshotAtStop() { + return isRemoving ? triggerSnapshotWhenRemoveEnabled : triggerSnapshotWhenStopEnabled; + } + + void setRemoving() { + this.isRemoving = true; + } + private long getLastAppliedIndex() { return appliedIndex.get(); } From e6b06a590526e890a04e4eb8526a198eb8b1244f Mon Sep 17 00:00:00 2001 From: Potato Date: Mon, 25 Mar 2024 23:12:57 +0800 Subject: [PATCH 016/397] RATIS-2049. Suppress Sonar warning for volatile objects (#1057) --- .../java/org/apache/ratis/client/impl/RaftClientImpl.java | 1 + .../java/org/apache/ratis/util/MemoizedCheckedSupplier.java | 2 ++ .../src/main/java/org/apache/ratis/util/MemoizedSupplier.java | 1 + .../java/org/apache/ratis/examples/debug/server/Server.java | 1 + .../java/org/apache/ratis/examples/filestore/cli/Client.java | 3 +++ .../org/apache/ratis/examples/membership/server/CServer.java | 4 +--- .../java/org/apache/ratis/grpc/server/GrpcLogAppender.java | 4 ++-- .../main/java/org/apache/ratis/server/leader/LogAppender.java | 3 ++- .../java/org/apache/ratis/server/storage/TestRaftStorage.java | 1 + 9 files changed, 14 insertions(+), 6 deletions(-) diff --git a/ratis-client/src/main/java/org/apache/ratis/client/impl/RaftClientImpl.java b/ratis-client/src/main/java/org/apache/ratis/client/impl/RaftClientImpl.java index 1b82709daf..db789aef2f 100644 --- a/ratis-client/src/main/java/org/apache/ratis/client/impl/RaftClientImpl.java +++ b/ratis-client/src/main/java/org/apache/ratis/client/impl/RaftClientImpl.java @@ -172,6 +172,7 @@ private synchronized Set getAndReset() { private final RaftGroupId groupId; private final RetryPolicy retryPolicy; + @SuppressWarnings({"squid:S3077"}) // Suppress volatile for generic type private volatile RaftPeerId leaderId; /** The callIds of the replied requests. */ private final RepliedCallIds repliedCallIds; diff --git a/ratis-common/src/main/java/org/apache/ratis/util/MemoizedCheckedSupplier.java b/ratis-common/src/main/java/org/apache/ratis/util/MemoizedCheckedSupplier.java index cf2d060239..0e9ae44fa5 100644 --- a/ratis-common/src/main/java/org/apache/ratis/util/MemoizedCheckedSupplier.java +++ b/ratis-common/src/main/java/org/apache/ratis/util/MemoizedCheckedSupplier.java @@ -44,6 +44,8 @@ public static MemoizedCheckedSupplier initializer; + + @SuppressWarnings({"squid:S3077"}) // Suppress volatile for generic type private volatile RETURN value = null; /** diff --git a/ratis-common/src/main/java/org/apache/ratis/util/MemoizedSupplier.java b/ratis-common/src/main/java/org/apache/ratis/util/MemoizedSupplier.java index f179d2dcd0..5c2754a307 100644 --- a/ratis-common/src/main/java/org/apache/ratis/util/MemoizedSupplier.java +++ b/ratis-common/src/main/java/org/apache/ratis/util/MemoizedSupplier.java @@ -40,6 +40,7 @@ public static MemoizedSupplier valueOf(Supplier supplier) { } private final Supplier initializer; + @SuppressWarnings({"squid:S3077"}) // Suppress volatile for generic type private volatile T value = null; /** diff --git a/ratis-examples/src/main/java/org/apache/ratis/examples/debug/server/Server.java b/ratis-examples/src/main/java/org/apache/ratis/examples/debug/server/Server.java index 4377a1420f..07f7b20f44 100644 --- a/ratis-examples/src/main/java/org/apache/ratis/examples/debug/server/Server.java +++ b/ratis-examples/src/main/java/org/apache/ratis/examples/debug/server/Server.java @@ -34,6 +34,7 @@ public final class Server { private Server(){ } + @SuppressWarnings({"squid:S2095"}) // Suppress closeable warning public static void main(String[] args) throws IOException { if (args.length < 1) { System.err.println("The arguments should be "); diff --git a/ratis-examples/src/main/java/org/apache/ratis/examples/filestore/cli/Client.java b/ratis-examples/src/main/java/org/apache/ratis/examples/filestore/cli/Client.java index 1856fc9b5d..caf2aa59b2 100644 --- a/ratis-examples/src/main/java/org/apache/ratis/examples/filestore/cli/Client.java +++ b/ratis-examples/src/main/java/org/apache/ratis/examples/filestore/cli/Client.java @@ -159,6 +159,9 @@ protected void dropCache() { Process pro = Runtime.getRuntime().exec(cmds); pro.waitFor(); } catch (Throwable t) { + if (t instanceof InterruptedException) { + Thread.currentThread().interrupt(); + } System.err.println("Failed to run command:" + Arrays.toString(cmds) + ":" + t.getMessage()); } } diff --git a/ratis-examples/src/main/java/org/apache/ratis/examples/membership/server/CServer.java b/ratis-examples/src/main/java/org/apache/ratis/examples/membership/server/CServer.java index a846cd1e77..2145412e04 100644 --- a/ratis-examples/src/main/java/org/apache/ratis/examples/membership/server/CServer.java +++ b/ratis-examples/src/main/java/org/apache/ratis/examples/membership/server/CServer.java @@ -32,7 +32,6 @@ import org.apache.ratis.thirdparty.com.google.common.base.MoreObjects; import org.apache.ratis.util.FileUtils; -import java.io.Closeable; import java.io.File; import java.io.IOException; import java.util.Collections; @@ -40,7 +39,7 @@ /** * A simple raft server using {@link CounterStateMachine}. */ -public class CServer implements Closeable { +public class CServer { public static final RaftGroupId GROUP_ID = RaftGroupId.randomId(); public static final String LOCAL_ADDR = "0.0.0.0"; @@ -78,7 +77,6 @@ public RaftPeer getPeer() { return server.getPeer(); } - @Override public void close() throws IOException { server.close(); FileUtils.deleteFully(storageDir); diff --git a/ratis-grpc/src/main/java/org/apache/ratis/grpc/server/GrpcLogAppender.java b/ratis-grpc/src/main/java/org/apache/ratis/grpc/server/GrpcLogAppender.java index ec44d8c485..5f9c94eb24 100644 --- a/ratis-grpc/src/main/java/org/apache/ratis/grpc/server/GrpcLogAppender.java +++ b/ratis-grpc/src/main/java/org/apache/ratis/grpc/server/GrpcLogAppender.java @@ -156,7 +156,6 @@ synchronized int process(Event event) { private final TimeDuration logMessageBatchDuration; private final int maxOutstandingInstallSnapshots; private final TimeoutExecutor scheduler = TimeoutExecutor.getInstance(); - private volatile StreamObservers appendLogRequestObserver; private final boolean useSeparateHBChannel; @@ -863,6 +862,7 @@ private TermIndex shouldNotifyToInstallSnapshot() { static class AppendEntriesRequest { private final Timekeeper timer; + @SuppressWarnings({"squid:S3077"}) // Suppress volatile for generic type private volatile Timekeeper.Context timerContext; private final long callId; @@ -871,7 +871,7 @@ static class AppendEntriesRequest { private final TermIndex firstEntry; private final TermIndex lastEntry; - + @SuppressWarnings({"squid:S3077"}) // Suppress volatile for generic type private volatile Timestamp sendTime; AppendEntriesRequest(AppendEntriesRequestProto proto, RaftPeerId followerId, GrpcServerMetrics grpcServerMetrics) { diff --git a/ratis-server-api/src/main/java/org/apache/ratis/server/leader/LogAppender.java b/ratis-server-api/src/main/java/org/apache/ratis/server/leader/LogAppender.java index 020a352c05..36331e3abb 100644 --- a/ratis-server-api/src/main/java/org/apache/ratis/server/leader/LogAppender.java +++ b/ratis-server-api/src/main/java/org/apache/ratis/server/leader/LogAppender.java @@ -87,10 +87,11 @@ default void stop() { default CompletableFuture stopAsync() { stop(); return CompletableFuture.supplyAsync(() -> { - for (; isRunning(); ) { + while (isRunning()) { try { Thread.sleep(10); } catch (InterruptedException e) { + Thread.currentThread().interrupt(); throw new CompletionException("stopAsync interrupted", e); } } diff --git a/ratis-test/src/test/java/org/apache/ratis/server/storage/TestRaftStorage.java b/ratis-test/src/test/java/org/apache/ratis/server/storage/TestRaftStorage.java index eb65f342e3..12cd771315 100644 --- a/ratis-test/src/test/java/org/apache/ratis/server/storage/TestRaftStorage.java +++ b/ratis-test/src/test/java/org/apache/ratis/server/storage/TestRaftStorage.java @@ -87,6 +87,7 @@ static RaftStorageImpl formatRaftStorage(File dir) throws IOException { return impl; } + @SuppressWarnings({"squid:S5783"}) // Suppress same exception warning @Test public void testNotExistent() throws IOException { FileUtils.deleteFully(storageDir); From 4470c2ede00363749770a7778109bf2841371c13 Mon Sep 17 00:00:00 2001 From: William Song <48054931+SzyWilliam@users.noreply.github.com> Date: Tue, 26 Mar 2024 22:52:06 +0800 Subject: [PATCH 017/397] RATIS-2050. Add creationGap param to snapshot management API (#1058) --- .../client/api/SnapshotManagementApi.java | 22 +++++++++++++++++-- .../ratis/client/impl/ClientProtoUtils.java | 5 +++-- .../client/impl/SnapshotManagementImpl.java | 5 +++-- .../protocol/SnapshotManagementRequest.java | 18 +++++++++++++-- ratis-proto/src/main/proto/Raft.proto | 2 +- .../ratis/server/impl/RaftServerImpl.java | 5 +++-- 6 files changed, 46 insertions(+), 11 deletions(-) diff --git a/ratis-client/src/main/java/org/apache/ratis/client/api/SnapshotManagementApi.java b/ratis-client/src/main/java/org/apache/ratis/client/api/SnapshotManagementApi.java index edd0475442..f83d976040 100644 --- a/ratis-client/src/main/java/org/apache/ratis/client/api/SnapshotManagementApi.java +++ b/ratis-client/src/main/java/org/apache/ratis/client/api/SnapshotManagementApi.java @@ -27,6 +27,24 @@ */ public interface SnapshotManagementApi { - /** trigger create snapshot file. */ - RaftClientReply create(long timeoutMs) throws IOException; + /** The same as create(0, timeoutMs). */ + default RaftClientReply create(long timeoutMs) throws IOException { + return create(0, timeoutMs); + } + + /** The same as create(force? 1 : 0, timeoutMs). */ + default RaftClientReply create(boolean force, long timeoutMs) throws IOException { + return create(force? 1 : 0, timeoutMs); + } + + /** + * Trigger to create a snapshot. + * + * @param creationGap When (creationGap > 0) and (astAppliedIndex - lastSnapshotIndex < creationGap), + * return lastSnapshotIndex; otherwise, take a new snapshot and then return its index. + * When creationGap == 0, use the server configured value as the creationGap. + * @return a reply. When {@link RaftClientReply#isSuccess()} is true, + * {@link RaftClientReply#getLogIndex()} is the snapshot index fulfilling the operation. + */ + RaftClientReply create(long creationGap, long timeoutMs) throws IOException; } diff --git a/ratis-client/src/main/java/org/apache/ratis/client/impl/ClientProtoUtils.java b/ratis-client/src/main/java/org/apache/ratis/client/impl/ClientProtoUtils.java index 003f202bd9..cab9606a0e 100644 --- a/ratis-client/src/main/java/org/apache/ratis/client/impl/ClientProtoUtils.java +++ b/ratis-client/src/main/java/org/apache/ratis/client/impl/ClientProtoUtils.java @@ -659,7 +659,8 @@ static SnapshotManagementRequest toSnapshotManagementRequest(SnapshotManagementR switch(p.getOpCase()) { case CREATE: return SnapshotManagementRequest.newCreate(clientId, serverId, - ProtoUtils.toRaftGroupId(m.getRaftGroupId()), m.getCallId(), m.getTimeoutMs()); + ProtoUtils.toRaftGroupId(m.getRaftGroupId()), m.getCallId(), m.getTimeoutMs(), + p.getCreate().getCreationGap()); default: throw new IllegalArgumentException("Unexpected op " + p.getOpCase() + " in " + p); } @@ -671,7 +672,7 @@ static SnapshotManagementRequestProto toSnapshotManagementRequestProto( .setRpcRequest(toRaftRpcRequestProtoBuilder(request)); final SnapshotManagementRequest.Create create = request.getCreate(); if (create != null) { - b.setCreate(SnapshotCreateRequestProto.newBuilder().build()); + b.setCreate(SnapshotCreateRequestProto.newBuilder().setCreationGap(create.getCreationGap()).build()); } return b.build(); } diff --git a/ratis-client/src/main/java/org/apache/ratis/client/impl/SnapshotManagementImpl.java b/ratis-client/src/main/java/org/apache/ratis/client/impl/SnapshotManagementImpl.java index 1762dc0e49..65c54d0f21 100644 --- a/ratis-client/src/main/java/org/apache/ratis/client/impl/SnapshotManagementImpl.java +++ b/ratis-client/src/main/java/org/apache/ratis/client/impl/SnapshotManagementImpl.java @@ -37,9 +37,10 @@ class SnapshotManagementImpl implements SnapshotManagementApi { } @Override - public RaftClientReply create(long timeoutMs) throws IOException { + public RaftClientReply create(long creationGap, long timeoutMs) throws IOException { final long callId = CallId.getAndIncrement(); return client.io().sendRequestWithRetry(() -> SnapshotManagementRequest.newCreate(client.getId(), - Optional.ofNullable(server).orElseGet(client::getLeaderId), client.getGroupId(), callId, timeoutMs)); + Optional.ofNullable(server).orElseGet(client::getLeaderId), + client.getGroupId(), callId, timeoutMs, creationGap)); } } diff --git a/ratis-common/src/main/java/org/apache/ratis/protocol/SnapshotManagementRequest.java b/ratis-common/src/main/java/org/apache/ratis/protocol/SnapshotManagementRequest.java index 2ea2059b51..269fdfc591 100644 --- a/ratis-common/src/main/java/org/apache/ratis/protocol/SnapshotManagementRequest.java +++ b/ratis-common/src/main/java/org/apache/ratis/protocol/SnapshotManagementRequest.java @@ -24,7 +24,16 @@ public final class SnapshotManagementRequest extends RaftClientRequest { public abstract static class Op { } - public static class Create extends Op { + + public static final class Create extends Op { + private final long creationGap; + private Create(long creationGap) { + this.creationGap = creationGap; + } + + public long getCreationGap() { + return creationGap; + } @Override public String toString() { @@ -35,8 +44,13 @@ public String toString() { public static SnapshotManagementRequest newCreate(ClientId clientId, RaftPeerId serverId, RaftGroupId groupId, long callId, long timeoutMs) { + return newCreate(clientId, serverId, groupId, callId, timeoutMs, 0); + } + + public static SnapshotManagementRequest newCreate(ClientId clientId, + RaftPeerId serverId, RaftGroupId groupId, long callId, long timeoutMs, long creationGap) { return new SnapshotManagementRequest(clientId, - serverId, groupId, callId, timeoutMs,new SnapshotManagementRequest.Create()); + serverId, groupId, callId, timeoutMs, new SnapshotManagementRequest.Create(creationGap)); } private final Op op; diff --git a/ratis-proto/src/main/proto/Raft.proto b/ratis-proto/src/main/proto/Raft.proto index edc57ec65e..b2e96e283e 100644 --- a/ratis-proto/src/main/proto/Raft.proto +++ b/ratis-proto/src/main/proto/Raft.proto @@ -470,7 +470,7 @@ message SnapshotManagementRequestProto { } message SnapshotCreateRequestProto { - + uint64 creationGap = 1; } message StartLeaderElectionRequestProto { diff --git a/ratis-server/src/main/java/org/apache/ratis/server/impl/RaftServerImpl.java b/ratis-server/src/main/java/org/apache/ratis/server/impl/RaftServerImpl.java index 67c90f20bb..7ec0444407 100644 --- a/ratis-server/src/main/java/org/apache/ratis/server/impl/RaftServerImpl.java +++ b/ratis-server/src/main/java/org/apache/ratis/server/impl/RaftServerImpl.java @@ -1212,9 +1212,10 @@ CompletableFuture takeSnapshotAsync(SnapshotManagementRequest r LOG.info("{}: takeSnapshotAsync {}", getMemberId(), request); assertLifeCycleState(LifeCycle.States.RUNNING); assertGroup(getMemberId(), request); + Preconditions.assertNotNull(request.getCreate(), "create"); - //TODO(liuyaolong): get the gap value from shell command - long minGapValue = RaftServerConfigKeys.Snapshot.creationGap(proxy.getProperties()); + final long creationGap = request.getCreate().getCreationGap(); + long minGapValue = creationGap > 0? creationGap : RaftServerConfigKeys.Snapshot.creationGap(proxy.getProperties()); final long lastSnapshotIndex = Optional.ofNullable(stateMachine.getLatestSnapshot()) .map(SnapshotInfo::getIndex) .orElse(0L); From 9b339620874fd2d58841ac08ea0d486b58dc369c Mon Sep 17 00:00:00 2001 From: Potato Date: Sat, 30 Mar 2024 02:16:24 +0800 Subject: [PATCH 018/397] RATIS-2052. Add docs for triggering snapshot when removing raftGroup parameter. (#1061) --- ratis-docs/src/site/markdown/configurations.md | 5 +++++ 1 file changed, 5 insertions(+) diff --git a/ratis-docs/src/site/markdown/configurations.md b/ratis-docs/src/site/markdown/configurations.md index 7c8fb001fb..95505fd549 100644 --- a/ratis-docs/src/site/markdown/configurations.md +++ b/ratis-docs/src/site/markdown/configurations.md @@ -494,6 +494,11 @@ further wait for 5sec for max times ((5sec*980)/2 times ~= 40min) | **Type** | boolean | | **Default** | true | +| **Property** | `raft.server.snapshot.trigger-when-remove.enabled` | +|:----------------|:--------------------------------------------------------| +| **Description** | whether to trigger snapshot when raft server is removed | +| **Type** | boolean | +| **Default** | true | | **Property** | `raft.server.snapshot.creation.gap` | |:----------------|:-----------------------------------------------------| From b7a79f342a6c08468d152e425941530d99f7f906 Mon Sep 17 00:00:00 2001 From: Potato Date: Thu, 4 Apr 2024 00:16:16 +0800 Subject: [PATCH 019/397] RATIS-2051. Fix Sonar bugs for Ratis. (#1059) --- .../src/main/java/org/apache/ratis/util/PeerProxyMap.java | 1 + .../src/main/java/org/apache/ratis/util/Preconditions.java | 5 +++++ .../java/org/apache/ratis/grpc/server/GrpcLogAppender.java | 1 + .../org/apache/ratis/server/impl/ConfigurationManager.java | 2 ++ .../java/org/apache/ratis/server/impl/FollowerState.java | 1 + .../main/java/org/apache/ratis/server/impl/LeaderLease.java | 2 +- .../java/org/apache/ratis/server/impl/LeaderStateImpl.java | 3 ++- .../org/apache/ratis/server/impl/RaftConfigurationImpl.java | 1 + .../main/java/org/apache/ratis/server/impl/ServerState.java | 2 ++ .../org/apache/ratis/server/impl/StateMachineUpdater.java | 1 + .../apache/ratis/server/metrics/LeaderElectionMetrics.java | 1 + .../apache/ratis/server/raftlog/segmented/LogSegment.java | 4 ++-- .../ratis/server/raftlog/segmented/SegmentedRaftLog.java | 1 + .../server/raftlog/segmented/SegmentedRaftLogCache.java | 1 + .../server/raftlog/segmented/SegmentedRaftLogWorker.java | 1 + .../ratis/server/storage/RaftStorageDirectoryImpl.java | 1 + .../org/apache/ratis/server/storage/SnapshotManager.java | 1 + .../org/apache/ratis/statemachine/impl/BaseStateMachine.java | 1 + .../ratis/statemachine/impl/TransactionContextImpl.java | 4 ++++ 19 files changed, 30 insertions(+), 4 deletions(-) diff --git a/ratis-common/src/main/java/org/apache/ratis/util/PeerProxyMap.java b/ratis-common/src/main/java/org/apache/ratis/util/PeerProxyMap.java index 0ce0595fa9..eda41a0096 100644 --- a/ratis-common/src/main/java/org/apache/ratis/util/PeerProxyMap.java +++ b/ratis-common/src/main/java/org/apache/ratis/util/PeerProxyMap.java @@ -45,6 +45,7 @@ public class PeerProxyMap implements RaftPeer.Add, Clos /** Peer and its proxy. */ private class PeerAndProxy { private final RaftPeer peer; + @SuppressWarnings({"squid:S3077"}) // Suppress volatile for generic type private volatile PROXY proxy = null; private final LifeCycle lifeCycle; diff --git a/ratis-common/src/main/java/org/apache/ratis/util/Preconditions.java b/ratis-common/src/main/java/org/apache/ratis/util/Preconditions.java index c757de2990..f37b250f53 100644 --- a/ratis-common/src/main/java/org/apache/ratis/util/Preconditions.java +++ b/ratis-common/src/main/java/org/apache/ratis/util/Preconditions.java @@ -78,6 +78,11 @@ static void assertTrue(boolean value, Supplier message) { } } + static void assertSame(int expected, int computed, String name) { + assertTrue(expected == computed, + () -> name + ": expected == " + expected + " but computed == " + computed); + } + static void assertSame(long expected, long computed, String name) { assertTrue(expected == computed, () -> name + ": expected == " + expected + " but computed == " + computed); diff --git a/ratis-grpc/src/main/java/org/apache/ratis/grpc/server/GrpcLogAppender.java b/ratis-grpc/src/main/java/org/apache/ratis/grpc/server/GrpcLogAppender.java index 5f9c94eb24..88e28dc48b 100644 --- a/ratis-grpc/src/main/java/org/apache/ratis/grpc/server/GrpcLogAppender.java +++ b/ratis-grpc/src/main/java/org/apache/ratis/grpc/server/GrpcLogAppender.java @@ -156,6 +156,7 @@ synchronized int process(Event event) { private final TimeDuration logMessageBatchDuration; private final int maxOutstandingInstallSnapshots; private final TimeoutExecutor scheduler = TimeoutExecutor.getInstance(); + @SuppressWarnings({"squid:S3077"}) // Suppress volatile for generic type private volatile StreamObservers appendLogRequestObserver; private final boolean useSeparateHBChannel; diff --git a/ratis-server/src/main/java/org/apache/ratis/server/impl/ConfigurationManager.java b/ratis-server/src/main/java/org/apache/ratis/server/impl/ConfigurationManager.java index 0e020b7e36..10c59c8b19 100644 --- a/ratis-server/src/main/java/org/apache/ratis/server/impl/ConfigurationManager.java +++ b/ratis-server/src/main/java/org/apache/ratis/server/impl/ConfigurationManager.java @@ -41,8 +41,10 @@ public class ConfigurationManager { * The current raft configuration. If configurations is not empty, should be * the last entry of the map. Otherwise is initialConf. */ + @SuppressWarnings({"squid:S3077"}) // Suppress volatile for generic type private volatile RaftConfigurationImpl currentConf; /** Cache the peer corresponding to {@link #id}. */ + @SuppressWarnings({"squid:S3077"}) // Suppress volatile for generic type private volatile RaftPeer currentPeer; ConfigurationManager(RaftPeerId id, RaftConfigurationImpl initialConf) { diff --git a/ratis-server/src/main/java/org/apache/ratis/server/impl/FollowerState.java b/ratis-server/src/main/java/org/apache/ratis/server/impl/FollowerState.java index 3911e39a5c..e980daede5 100644 --- a/ratis-server/src/main/java/org/apache/ratis/server/impl/FollowerState.java +++ b/ratis-server/src/main/java/org/apache/ratis/server/impl/FollowerState.java @@ -59,6 +59,7 @@ int update(AtomicInteger outstanding) { private final RaftServerImpl server; private final Timestamp creationTime = Timestamp.currentTime(); + @SuppressWarnings({"squid:S3077"}) // Suppress volatile for generic type private volatile Timestamp lastRpcTime = creationTime; private volatile boolean isRunning = true; private final AtomicInteger outstandingOp = new AtomicInteger(); diff --git a/ratis-server/src/main/java/org/apache/ratis/server/impl/LeaderLease.java b/ratis-server/src/main/java/org/apache/ratis/server/impl/LeaderLease.java index 315cc9f143..3b8f53da14 100644 --- a/ratis-server/src/main/java/org/apache/ratis/server/impl/LeaderLease.java +++ b/ratis-server/src/main/java/org/apache/ratis/server/impl/LeaderLease.java @@ -92,7 +92,7 @@ private Timestamp getMaxTimestampWithMajorityAck(List followers) { return Timestamp.currentTime(); } - final int mid = followers.size() / 2; + final long mid = followers.size() / 2; return followers.stream() .map(FollowerInfo::getLastRespondedAppendEntriesSendTime) .sorted() diff --git a/ratis-server/src/main/java/org/apache/ratis/server/impl/LeaderStateImpl.java b/ratis-server/src/main/java/org/apache/ratis/server/impl/LeaderStateImpl.java index ea25c8a315..dd2e564f74 100644 --- a/ratis-server/src/main/java/org/apache/ratis/server/impl/LeaderStateImpl.java +++ b/ratis-server/src/main/java/org/apache/ratis/server/impl/LeaderStateImpl.java @@ -268,7 +268,7 @@ static boolean isSameConf(CurrentOldFollowerInfos cached, RaftConfigurationImpl static class FollowerInfoMap { private final Map map = new ConcurrentHashMap<>(); - + @SuppressWarnings({"squid:S3077"}) // Suppress volatile for generic type private volatile CurrentOldFollowerInfos followerInfos; void put(RaftPeerId id, FollowerInfo info) { @@ -332,6 +332,7 @@ boolean isApplied() { private final RaftServerImpl server; private final RaftLog raftLog; private final long currentTerm; + @SuppressWarnings({"squid:S3077"}) // Suppress volatile for generic type private volatile ConfigurationStagingState stagingState; private final FollowerInfoMap followerInfoMap = new FollowerInfoMap(); diff --git a/ratis-server/src/main/java/org/apache/ratis/server/impl/RaftConfigurationImpl.java b/ratis-server/src/main/java/org/apache/ratis/server/impl/RaftConfigurationImpl.java index 98f41020e0..15a114cdb4 100644 --- a/ratis-server/src/main/java/org/apache/ratis/server/impl/RaftConfigurationImpl.java +++ b/ratis-server/src/main/java/org/apache/ratis/server/impl/RaftConfigurationImpl.java @@ -148,6 +148,7 @@ boolean isStable() { return oldConf == null; } + @SuppressWarnings({"squid:S6466"}) // Suppress ArrayIndexOutOfBoundsException warning boolean containsInConf(RaftPeerId peerId, RaftPeerRole... roles) { if (roles == null || roles.length == 0) { return conf.contains(peerId); diff --git a/ratis-server/src/main/java/org/apache/ratis/server/impl/ServerState.java b/ratis-server/src/main/java/org/apache/ratis/server/impl/ServerState.java index 3099b5b840..be74567fc0 100644 --- a/ratis-server/src/main/java/org/apache/ratis/server/impl/ServerState.java +++ b/ratis-server/src/main/java/org/apache/ratis/server/impl/ServerState.java @@ -87,6 +87,7 @@ class ServerState { /** * Candidate that this peer granted vote for in current term (or null if none) */ + @SuppressWarnings({"squid:S3077"}) // Suppress volatile for generic type private volatile RaftPeerId votedFor; /** @@ -171,6 +172,7 @@ private RaftLog initRaftLog(LongSupplier getSnapshotIndexFromStateMachine, RaftP } } + @SuppressWarnings({"squid:S2095"}) // Suppress closeable warning private static RaftLog initRaftLog(RaftGroupMemberId memberId, RaftServerImpl server, RaftStorage storage, Consumer logConsumer, LongSupplier getSnapshotIndexFromStateMachine, RaftProperties prop) throws IOException { diff --git a/ratis-server/src/main/java/org/apache/ratis/server/impl/StateMachineUpdater.java b/ratis-server/src/main/java/org/apache/ratis/server/impl/StateMachineUpdater.java index 7c314ef441..c1db1fd3cc 100644 --- a/ratis-server/src/main/java/org/apache/ratis/server/impl/StateMachineUpdater.java +++ b/ratis-server/src/main/java/org/apache/ratis/server/impl/StateMachineUpdater.java @@ -199,6 +199,7 @@ public void run() { } } catch (Throwable t) { if (t instanceof InterruptedException && state == State.STOP) { + Thread.currentThread().interrupt(); LOG.info("{} was interrupted. Exiting ...", this); } else { state = State.EXCEPTION; diff --git a/ratis-server/src/main/java/org/apache/ratis/server/metrics/LeaderElectionMetrics.java b/ratis-server/src/main/java/org/apache/ratis/server/metrics/LeaderElectionMetrics.java index 7447498d42..1d044bb6fe 100644 --- a/ratis-server/src/main/java/org/apache/ratis/server/metrics/LeaderElectionMetrics.java +++ b/ratis-server/src/main/java/org/apache/ratis/server/metrics/LeaderElectionMetrics.java @@ -51,6 +51,7 @@ public final class LeaderElectionMetrics extends RatisMetrics { private final Timekeeper electionTime = getRegistry().timer(LEADER_ELECTION_TIME_TAKEN); + @SuppressWarnings({"squid:S3077"}) // Suppress volatile for generic type private volatile Timestamp lastElectionTime; private LeaderElectionMetrics(RaftGroupMemberId serverId, LongSupplier getLastLeaderElapsedTimeMs) { diff --git a/ratis-server/src/main/java/org/apache/ratis/server/raftlog/segmented/LogSegment.java b/ratis-server/src/main/java/org/apache/ratis/server/raftlog/segmented/LogSegment.java index 0750d2cc8c..12e7c4f1d1 100644 --- a/ratis-server/src/main/java/org/apache/ratis/server/raftlog/segmented/LogSegment.java +++ b/ratis-server/src/main/java/org/apache/ratis/server/raftlog/segmented/LogSegment.java @@ -294,10 +294,10 @@ File getFile() { private volatile boolean isOpen; private long totalFileSize = SegmentedRaftLogFormat.getHeaderLength(); /** Segment start index, inclusive. */ - private long startIndex; + private final long startIndex; /** Segment end index, inclusive. */ private volatile long endIndex; - private RaftStorage storage; + private final RaftStorage storage; private final SizeInBytes maxOpSize; private final LogEntryLoader cacheLoader; /** later replace it with a metric */ diff --git a/ratis-server/src/main/java/org/apache/ratis/server/raftlog/segmented/SegmentedRaftLog.java b/ratis-server/src/main/java/org/apache/ratis/server/raftlog/segmented/SegmentedRaftLog.java index 7010189671..975e00a81b 100644 --- a/ratis-server/src/main/java/org/apache/ratis/server/raftlog/segmented/SegmentedRaftLog.java +++ b/ratis-server/src/main/java/org/apache/ratis/server/raftlog/segmented/SegmentedRaftLog.java @@ -205,6 +205,7 @@ public TransactionContext getTransactionContext(LogEntryProto entry, boolean cre private final boolean stateMachineCachingEnabled; private final SegmentedRaftLogMetrics metrics; + @SuppressWarnings({"squid:S2095"}) // Suppress closeable warning private SegmentedRaftLog(Builder b) { super(b.memberId, b.snapshotIndexSupplier, b.properties); this.metrics = new SegmentedRaftLogMetrics(b.memberId); diff --git a/ratis-server/src/main/java/org/apache/ratis/server/raftlog/segmented/SegmentedRaftLogCache.java b/ratis-server/src/main/java/org/apache/ratis/server/raftlog/segmented/SegmentedRaftLogCache.java index 1d08316fda..0b05b14e5c 100644 --- a/ratis-server/src/main/java/org/apache/ratis/server/raftlog/segmented/SegmentedRaftLogCache.java +++ b/ratis-server/src/main/java/org/apache/ratis/server/raftlog/segmented/SegmentedRaftLogCache.java @@ -394,6 +394,7 @@ public String toString() { } private final String name; + @SuppressWarnings({"squid:S3077"}) // Suppress volatile for generic type private volatile LogSegment openSegment; private final LogSegmentList closedSegments; private final RaftStorage storage; diff --git a/ratis-server/src/main/java/org/apache/ratis/server/raftlog/segmented/SegmentedRaftLogWorker.java b/ratis-server/src/main/java/org/apache/ratis/server/raftlog/segmented/SegmentedRaftLogWorker.java index 0d1ea763b6..502e279e58 100644 --- a/ratis-server/src/main/java/org/apache/ratis/server/raftlog/segmented/SegmentedRaftLogWorker.java +++ b/ratis-server/src/main/java/org/apache/ratis/server/raftlog/segmented/SegmentedRaftLogWorker.java @@ -143,6 +143,7 @@ synchronized void updateIndex(long i) { private volatile boolean running = true; private final ExecutorService workerThreadExecutor; private final RaftStorage storage; + @SuppressWarnings({"squid:S3077"}) // Suppress volatile for generic type private volatile SegmentedRaftLogOutputStream out; private final Runnable submitUpdateCommitEvent; private final StateMachine stateMachine; diff --git a/ratis-server/src/main/java/org/apache/ratis/server/storage/RaftStorageDirectoryImpl.java b/ratis-server/src/main/java/org/apache/ratis/server/storage/RaftStorageDirectoryImpl.java index e7f69d1e24..119f7922db 100644 --- a/ratis-server/src/main/java/org/apache/ratis/server/storage/RaftStorageDirectoryImpl.java +++ b/ratis-server/src/main/java/org/apache/ratis/server/storage/RaftStorageDirectoryImpl.java @@ -210,6 +210,7 @@ void lock() throws IOException { * null if storage is already locked. * @throws IOException if locking fails. */ + @SuppressWarnings({"squid:S2095"}) // Suppress closeable warning private FileLock tryLock(File lockF) throws IOException { boolean deletionHookAdded = false; if (!lockF.exists()) { diff --git a/ratis-server/src/main/java/org/apache/ratis/server/storage/SnapshotManager.java b/ratis-server/src/main/java/org/apache/ratis/server/storage/SnapshotManager.java index c49a86ec59..794604d66b 100644 --- a/ratis-server/src/main/java/org/apache/ratis/server/storage/SnapshotManager.java +++ b/ratis-server/src/main/java/org/apache/ratis/server/storage/SnapshotManager.java @@ -77,6 +77,7 @@ public class SnapshotManager { new File(dir.get().getRoot(), c.getFilename()).toPath()).toString(); } + @SuppressWarnings({"squid:S2095"}) // Suppress closeable warning private FileChannel open(FileChunkProto chunk, File tmpSnapshotFile) throws IOException { final FileChannel out; final boolean exists = tmpSnapshotFile.exists(); diff --git a/ratis-server/src/main/java/org/apache/ratis/statemachine/impl/BaseStateMachine.java b/ratis-server/src/main/java/org/apache/ratis/statemachine/impl/BaseStateMachine.java index c987c53ddb..657c6a0fd3 100644 --- a/ratis-server/src/main/java/org/apache/ratis/statemachine/impl/BaseStateMachine.java +++ b/ratis-server/src/main/java/org/apache/ratis/statemachine/impl/BaseStateMachine.java @@ -50,6 +50,7 @@ public class BaseStateMachine implements StateMachine, StateMachine.DataApi, StateMachine.EventApi, StateMachine.LeaderEventApi, StateMachine.FollowerEventApi { private final CompletableFuture server = new CompletableFuture<>(); + @SuppressWarnings({"squid:S3077"}) // Suppress volatile for generic type private volatile RaftGroupId groupId; private final LifeCycle lifeCycle = new LifeCycle(JavaUtils.getClassSimpleName(getClass())); diff --git a/ratis-server/src/main/java/org/apache/ratis/statemachine/impl/TransactionContextImpl.java b/ratis-server/src/main/java/org/apache/ratis/statemachine/impl/TransactionContextImpl.java index 7c4f1782de..6006257166 100644 --- a/ratis-server/src/main/java/org/apache/ratis/statemachine/impl/TransactionContextImpl.java +++ b/ratis-server/src/main/java/org/apache/ratis/statemachine/impl/TransactionContextImpl.java @@ -47,6 +47,7 @@ public class TransactionContextImpl implements TransactionContext { private final RaftClientRequest clientRequest; /** Exception from the {@link StateMachine} or from the log */ + @SuppressWarnings({"squid:S3077"}) // Suppress volatile for generic type private volatile Exception exception; /** Data from the {@link StateMachine} */ @@ -58,6 +59,7 @@ public class TransactionContextImpl implements TransactionContext { * {@link StateMachine#startTransaction(RaftClientRequest)} and * {@link StateMachine#applyTransaction(TransactionContext)}. */ + @SuppressWarnings({"squid:S3077"}) // Suppress volatile for generic type private volatile Object stateMachineContext; /** @@ -68,8 +70,10 @@ public class TransactionContextImpl implements TransactionContext { private boolean shouldCommit = true; /** Committed LogEntry. */ + @SuppressWarnings({"squid:S3077"}) // Suppress volatile for generic type private volatile LogEntryProto logEntry; /** For wrapping {@link #logEntry} in order to release the underlying buffer. */ + @SuppressWarnings({"squid:S3077"}) // Suppress volatile for generic type private volatile ReferenceCountedObject delegatedRef; private final CompletableFuture logIndexFuture = new CompletableFuture<>(); From eed4e749ade5f98eb9214aab55e738caf02967f6 Mon Sep 17 00:00:00 2001 From: hao guo Date: Tue, 9 Apr 2024 02:10:22 +0800 Subject: [PATCH 020/397] RATIS-1519. When DataStreamManagement#read an exception occurs, remove DataStream (#596) --- .../netty/server/DataStreamManagement.java | 42 +++++++++++-------- .../ratis/datastream/DataStreamTestUtils.java | 7 +++- .../TestNettyDataStreamWithMock.java | 10 ++++- 3 files changed, 38 insertions(+), 21 deletions(-) diff --git a/ratis-netty/src/main/java/org/apache/ratis/netty/server/DataStreamManagement.java b/ratis-netty/src/main/java/org/apache/ratis/netty/server/DataStreamManagement.java index a6e9b815ee..e265d8b924 100644 --- a/ratis-netty/src/main/java/org/apache/ratis/netty/server/DataStreamManagement.java +++ b/ratis-netty/src/main/java/org/apache/ratis/netty/server/DataStreamManagement.java @@ -140,19 +140,19 @@ static class StreamInfo { private final boolean primary; private final LocalStream local; private final Set remotes; - private final RaftServer server; + private final Division division; private final AtomicReference> previous = new AtomicReference<>(CompletableFuture.completedFuture(null)); - StreamInfo(RaftClientRequest request, boolean primary, CompletableFuture stream, RaftServer server, + StreamInfo(RaftClientRequest request, boolean primary, CompletableFuture stream, Division division, CheckedBiFunction, Set, IOException> getStreams, Function metricsConstructor) throws IOException { this.request = request; this.primary = primary; this.local = new LocalStream(stream, metricsConstructor.apply(RequestType.LOCAL_WRITE)); - this.server = server; - final Set successors = getSuccessors(server.getId()); + this.division = division; + final Set successors = getSuccessors(division.getId()); final Set outs = getStreams.apply(request, successors); this.remotes = outs.stream() .map(o -> new RemoteStream(o, metricsConstructor.apply(RequestType.REMOTE_WRITE))) @@ -167,16 +167,12 @@ RaftClientRequest getRequest() { return request; } - Division getDivision() throws IOException { - return server.getDivision(request.getRaftGroupId()); + Division getDivision() { + return division; } Collection getCommitInfos() { - try { - return getDivision().getCommitInfos(); - } catch (IOException e) { - throw new IllegalStateException(e); - } + return getDivision().getCommitInfos(); } boolean isPrimary() { @@ -196,7 +192,7 @@ public String toString() { return JavaUtils.getClassSimpleName(getClass()) + ":" + request; } - private Set getSuccessors(RaftPeerId peerId) throws IOException { + private Set getSuccessors(RaftPeerId peerId) { final RaftConfiguration conf = getDivision().getRaftConf(); final RoutingTable routingTable = request.getRoutingTable(); @@ -208,7 +204,7 @@ private Set getSuccessors(RaftPeerId peerId) throws IOException { // Default start topology // get the other peers from the current configuration return conf.getCurrentPeers().stream() - .filter(p -> !p.getId().equals(server.getId())) + .filter(p -> !p.getId().equals(division.getId())) .collect(Collectors.toSet()); } @@ -276,7 +272,8 @@ private StreamInfo newStreamInfo(ByteBuf buf, final RaftClientRequest request = ClientProtoUtils.toRaftClientRequest( RaftClientRequestProto.parseFrom(buf.nioBuffer())); final boolean isPrimary = server.getId().equals(request.getServerId()); - return new StreamInfo(request, isPrimary, computeDataStreamIfAbsent(request), server, getStreams, + final Division division = server.getDivision(request.getRaftGroupId()); + return new StreamInfo(request, isPrimary, computeDataStreamIfAbsent(request), division, getStreams, getMetrics()::newRequestMetrics); } catch (Throwable e) { throw new CompletionException(e); @@ -411,6 +408,18 @@ void read(DataStreamRequestByteBuf request, ChannelHandlerContext ctx, readImpl(request, ctx, getStreams); } catch (Throwable t) { replyDataStreamException(t, request, ctx); + removeDataStream(ClientInvocationId.valueOf(request.getClientId(), request.getStreamId()), null); + } + } + + private void removeDataStream(ClientInvocationId invocationId, StreamInfo info) { + final StreamInfo removed = streams.remove(invocationId); + if (info == null) { + info = removed; + } + if (info != null) { + info.getDivision().getDataStreamMap().remove(invocationId); + info.getLocal().cleanUp(); } } @@ -429,8 +438,6 @@ private void readImpl(DataStreamRequestByteBuf request, ChannelHandlerContext ct () -> newStreamInfo(request.slice(), getStreams)); info = streams.computeIfAbsent(key, id -> supplier.get()); if (!supplier.isInitialized()) { - final StreamInfo removed = streams.remove(key); - removed.getLocal().cleanUp(); throw new IllegalStateException("Failed to create a new stream for " + request + " since a stream already exists Key: " + key + " StreamInfo:" + info); } @@ -468,9 +475,8 @@ private void readImpl(DataStreamRequestByteBuf request, ChannelHandlerContext ct }, requestExecutor)).whenComplete((v, exception) -> { try { if (exception != null) { - final StreamInfo removed = streams.remove(key); replyDataStreamException(server, exception, info.getRequest(), request, ctx); - removed.getLocal().cleanUp(); + removeDataStream(key, info); } } finally { request.release(); diff --git a/ratis-test/src/test/java/org/apache/ratis/datastream/DataStreamTestUtils.java b/ratis-test/src/test/java/org/apache/ratis/datastream/DataStreamTestUtils.java index e4a930f1d1..40c005c08d 100644 --- a/ratis-test/src/test/java/org/apache/ratis/datastream/DataStreamTestUtils.java +++ b/ratis-test/src/test/java/org/apache/ratis/datastream/DataStreamTestUtils.java @@ -149,6 +149,7 @@ class MultiDataStreamStateMachine extends BaseStateMachine { @Override public CompletableFuture stream(RaftClientRequest request) { final SingleDataStream s = new SingleDataStream(request); + LOG.info("XXX {} put {}, {}", this, ClientInvocationId.valueOf(request), s); streams.put(ClientInvocationId.valueOf(request), s); return CompletableFuture.completedFuture(s); } @@ -177,7 +178,9 @@ SingleDataStream getSingleDataStream(RaftClientRequest request) { } SingleDataStream getSingleDataStream(ClientInvocationId invocationId) { - return streams.get(invocationId); + final SingleDataStream s = streams.get(invocationId); + LOG.info("XXX {}: get {} return {}", this, invocationId, s); + return s; } Collection getStreams() { @@ -327,6 +330,8 @@ static CompletableFuture writeAndCloseAndAssertReplies( static void assertHeader(RaftServer server, RaftClientRequest header, int dataSize, boolean stepDownLeader) throws Exception { + LOG.info("XXX {}: dataSize={}, stepDownLeader={}, header={}", + server.getId(), dataSize, stepDownLeader, header); // check header Assertions.assertEquals(RaftClientRequest.dataStreamRequestType(), header.getType()); diff --git a/ratis-test/src/test/java/org/apache/ratis/datastream/TestNettyDataStreamWithMock.java b/ratis-test/src/test/java/org/apache/ratis/datastream/TestNettyDataStreamWithMock.java index 503f8cf66e..1d8c67a43d 100644 --- a/ratis-test/src/test/java/org/apache/ratis/datastream/TestNettyDataStreamWithMock.java +++ b/ratis-test/src/test/java/org/apache/ratis/datastream/TestNettyDataStreamWithMock.java @@ -59,12 +59,18 @@ public void setup() { RaftConfigKeys.DataStream.setType(properties, SupportedDataStreamType.NETTY); } - RaftServer.Division mockDivision(RaftServer server) { + + RaftServer.Division mockDivision(RaftServer server, RaftGroupId groupId) { final RaftServer.Division division = mock(RaftServer.Division.class); when(division.getRaftServer()).thenReturn(server); when(division.getRaftConf()).thenAnswer(i -> getRaftConf()); final MultiDataStreamStateMachine stateMachine = new MultiDataStreamStateMachine(); + try { + stateMachine.initialize(server, groupId, null); + } catch (IOException e) { + throw new IllegalStateException(e); + } when(division.getStateMachine()).thenReturn(stateMachine); final DataStreamMap streamMap = RaftServerTestUtil.newDataStreamMap(server.getId()); @@ -95,7 +101,7 @@ private void testMockCluster(int numServers, RaftException leaderException, when(raftServer.getId()).thenReturn(peerId); when(raftServer.getPeer()).thenReturn(RaftPeer.newBuilder().setId(peerId).build()); if (getStateMachineException == null) { - final RaftServer.Division myDivision = mockDivision(raftServer); + final RaftServer.Division myDivision = mockDivision(raftServer, groupId); when(raftServer.getDivision(Mockito.any(RaftGroupId.class))).thenReturn(myDivision); } else { when(raftServer.getDivision(Mockito.any(RaftGroupId.class))).thenThrow(getStateMachineException); From 6facdc455fa715d4e58c8352b5347656540d5ee3 Mon Sep 17 00:00:00 2001 From: DaveTeng0 <109315747+DaveTeng0@users.noreply.github.com> Date: Wed, 10 Apr 2024 16:38:57 -0700 Subject: [PATCH 021/397] Ratis-2040. Fix RaftPeerId generated by command of "raftMetaConf" to use real PeerId (#1060) --- ratis-docs/src/site/markdown/cli.md | 2 +- .../cli/sh/local/RaftMetaConfCommand.java | 51 ++++++- .../cli/sh/LocalCommandIntegrationTest.java | 142 ++++++++++++++++++ 3 files changed, 190 insertions(+), 5 deletions(-) create mode 100644 ratis-test/src/test/java/org/apache/ratis/shell/cli/sh/LocalCommandIntegrationTest.java diff --git a/ratis-docs/src/site/markdown/cli.md b/ratis-docs/src/site/markdown/cli.md index 60958fc7ed..ab9f89982f 100644 --- a/ratis-docs/src/site/markdown/cli.md +++ b/ratis-docs/src/site/markdown/cli.md @@ -182,5 +182,5 @@ It has the following subcommands: ### local raftMetaConf Generate a new raft-meta.conf file based on original raft-meta.conf and new peers, which is used to move a raft node to a new node. ``` -$ ratis sh local raftMetaConf -peers -path +$ ratis sh local raftMetaConf -peers <[P0_ID|]P0_HOST:P0_PORT,[P1_ID|]P1_HOST:P1_PORT,[P2_ID|]P2_HOST:P2_PORT> -path ``` diff --git a/ratis-shell/src/main/java/org/apache/ratis/shell/cli/sh/local/RaftMetaConfCommand.java b/ratis-shell/src/main/java/org/apache/ratis/shell/cli/sh/local/RaftMetaConfCommand.java index 231c643ac3..9f0558c5ea 100644 --- a/ratis-shell/src/main/java/org/apache/ratis/shell/cli/sh/local/RaftMetaConfCommand.java +++ b/ratis-shell/src/main/java/org/apache/ratis/shell/cli/sh/local/RaftMetaConfCommand.java @@ -24,6 +24,7 @@ import org.apache.ratis.proto.RaftProtos.RaftConfigurationProto; import org.apache.ratis.proto.RaftProtos.RaftPeerProto; import org.apache.ratis.proto.RaftProtos.RaftPeerRole; +import org.apache.ratis.protocol.RaftPeerId; import org.apache.ratis.shell.cli.RaftUtils; import org.apache.ratis.shell.cli.sh.command.AbstractCommand; import org.apache.ratis.shell.cli.sh.command.Context; @@ -32,11 +33,14 @@ import java.io.IOException; import java.io.InputStream; import java.io.OutputStream; +import java.net.InetSocketAddress; import java.nio.charset.StandardCharsets; import java.nio.file.Files; import java.nio.file.Paths; import java.util.ArrayList; +import java.util.HashSet; import java.util.List; +import java.util.Set; /** * Command for generate a new raft-meta.conf file based on original raft-meta.conf and new peers, @@ -49,6 +53,7 @@ public class RaftMetaConfCommand extends AbstractCommand { private static final String RAFT_META_CONF = "raft-meta.conf"; private static final String NEW_RAFT_META_CONF = "new-raft-meta.conf"; + private static final String SEPARATOR = "\\|"; /** * @param context command context */ @@ -69,11 +74,49 @@ public int run(CommandLine cl) throws IOException { printf("peers or path can't be empty."); return -1; } + Set addresses = new HashSet<>(); + Set ids = new HashSet<>(); List raftPeerProtos = new ArrayList<>(); - for (String address : peersStr.split(",")) { - String peerId = RaftUtils.getPeerId(parseInetSocketAddress(address)).toString(); + for (String idWithAddress : peersStr.split(",")) { + String[] peerIdWithAddressArray = idWithAddress.split(SEPARATOR); + + if (peerIdWithAddressArray.length < 1 || peerIdWithAddressArray.length > 2) { + String message = + "Failed to parse peer's ID and address for: %s, " + + "from option: -peers %s. \n" + + "Please make sure to provide list of peers" + + " in format <[P0_ID|]P0_HOST:P0_PORT,[P1_ID|]P1_HOST:P1_PORT,[P2_ID|]P2_HOST:P2_PORT>"; + printf(message, idWithAddress, peersStr); + return -1; + } + InetSocketAddress inetSocketAddress = parseInetSocketAddress( + peerIdWithAddressArray[peerIdWithAddressArray.length - 1]); + String addressString = inetSocketAddress.toString(); + if (addresses.contains(addressString)) { + printf("Found duplicated address: %s. Please make sure the address of peer have no duplicated value.", + addressString); + return -1; + } + addresses.add(addressString); + + String peerId; + if (peerIdWithAddressArray.length == 2) { + // Peer ID is provided + peerId = RaftPeerId.getRaftPeerId(peerIdWithAddressArray[0]).toString(); + + if (ids.contains(peerId)) { + printf("Found duplicated ID: %s. Please make sure the ID of peer have no duplicated value.", peerId); + return -1; + } + ids.add(peerId); + } else { + // If peer ID is not provided, use host address as peerId value + peerId = RaftUtils.getPeerId(inetSocketAddress).toString(); + } + raftPeerProtos.add(RaftPeerProto.newBuilder() - .setId(ByteString.copyFrom(peerId.getBytes(StandardCharsets.UTF_8))).setAddress(address) + .setId(ByteString.copyFrom(peerId.getBytes(StandardCharsets.UTF_8))) + .setAddress(addressString) .setStartupRole(RaftPeerRole.FOLLOWER).build()); } try (InputStream in = Files.newInputStream(Paths.get(path, RAFT_META_CONF)); @@ -93,7 +136,7 @@ public int run(CommandLine cl) throws IOException { @Override public String getUsage() { return String.format("%s" - + " -%s " + + " -%s <[P0_ID|]P0_HOST:P0_PORT,[P1_ID|]P1_HOST:P1_PORT,[P2_ID|]P2_HOST:P2_PORT>" + " -%s ", getCommandName(), PEER_OPTION_NAME, PATH_OPTION_NAME); } diff --git a/ratis-test/src/test/java/org/apache/ratis/shell/cli/sh/LocalCommandIntegrationTest.java b/ratis-test/src/test/java/org/apache/ratis/shell/cli/sh/LocalCommandIntegrationTest.java new file mode 100644 index 0000000000..4a07e372ca --- /dev/null +++ b/ratis-test/src/test/java/org/apache/ratis/shell/cli/sh/LocalCommandIntegrationTest.java @@ -0,0 +1,142 @@ +/* + * 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.ratis.shell.cli.sh; + +import org.apache.ratis.proto.RaftProtos.RaftPeerRole; +import org.apache.ratis.proto.RaftProtos.LogEntryProto; +import org.apache.ratis.proto.RaftProtos.RaftConfigurationProto; +import org.apache.ratis.proto.RaftProtos.RaftPeerProto; +import org.apache.ratis.thirdparty.com.google.protobuf.ByteString; +import org.junit.jupiter.api.Assertions; +import org.junit.jupiter.api.Test; +import org.junit.jupiter.api.io.TempDir; + +import java.io.IOException; +import java.io.InputStream; +import java.io.OutputStream; +import java.nio.charset.StandardCharsets; +import java.nio.file.Files; +import java.nio.file.Path; +import java.util.ArrayList; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import java.util.regex.Pattern; +import java.util.stream.Collectors; + +public class LocalCommandIntegrationTest { + + private static final String RAFT_META_CONF = "raft-meta.conf"; + private static final String NEW_RAFT_META_CONF = "new-raft-meta.conf"; + private static Pattern p = Pattern.compile("(?:\\w+\\|\\w+:\\d+,?)+"); + + + @Test + public void testDuplicatedPeerAddresses() throws Exception { + String[] duplicatedAddressesList = {"peer1_ID1|host1:9872,peer2_ID|host2:9872,peer1_ID2|host1:9872", + "host1:9872,host2:9872,host1:9872"}; + + testDuplicatedPeers(duplicatedAddressesList, "address", "host1:9872"); + } + + @Test + public void testDuplicatedPeerIds() throws Exception { + String[] duplicatedIdsList = {"peer1_ID1|host1:9872,peer2_ID|host2:9872,peer1_ID1|host3:9872"}; + + testDuplicatedPeers(duplicatedIdsList, "ID", "peer1_ID1"); + } + + private void testDuplicatedPeers(String[] peersList, String expectedErrorMessagePart, String expectedDuplicatedValue) throws Exception { + for (String peersStr : peersList) { + StringPrintStream out = new StringPrintStream(); + RatisShell shell = new RatisShell(out.getPrintStream()); + int ret = shell.run("local", "raftMetaConf", "-peers", peersStr, "-path", "test"); + Assertions.assertEquals(-1, ret); + String message = out.toString().trim(); + Assertions.assertEquals(String.format("Found duplicated %s: %s. Please make sure the %s of peer have no duplicated value.", + expectedErrorMessagePart, expectedDuplicatedValue, expectedErrorMessagePart), message); + } + } + + @Test + public void testRunMethod(@TempDir Path tempDir) throws Exception { + int index = 1; + generateRaftConf(tempDir.resolve(RAFT_META_CONF), index); + + String[] testPeersListArray = {"peer1_ID|host1:9872,peer2_ID|host2:9872,peer3_ID|host3:9872", + "host1:9872,host2:9872,host3:9872"}; + + for (String peersListStr : testPeersListArray) { + generateRaftConf(tempDir, index); + StringPrintStream out = new StringPrintStream(); + RatisShell shell = new RatisShell(out.getPrintStream()); + int ret = shell.run("local", "raftMetaConf", "-peers", peersListStr, "-path", tempDir.toString()); + Assertions.assertEquals(0, ret); + + // read & verify the contents of the new-raft-meta.conf file + long indexFromNewConf; + List peers; + try (InputStream in = Files.newInputStream(tempDir.resolve(NEW_RAFT_META_CONF))) { + LogEntryProto logEntry = LogEntryProto.newBuilder().mergeFrom(in).build(); + indexFromNewConf = logEntry.getIndex(); + peers = logEntry.getConfigurationEntry().getPeersList(); + } + + Assertions.assertEquals(index + 1, indexFromNewConf); + + String peersListStrFromNewMetaConf; + if (containsPeerId(peersListStr)) { + peersListStrFromNewMetaConf = peers.stream() + .map(peer -> peer.getId().toStringUtf8() + "|" + peer.getAddress()) + .collect(Collectors.joining(",")); + } else { + peersListStrFromNewMetaConf = peers.stream().map(RaftPeerProto::getAddress) + .collect(Collectors.joining(",")); + } + + Assertions.assertEquals(peersListStr, peersListStrFromNewMetaConf); + } + } + + + private void generateRaftConf(Path path, int index) throws IOException { + Map map = new HashMap<>(); + map.put("peer1_ID", "host1:9872"); + map.put("peer2_ID", "host2:9872"); + map.put("peer3_ID", "host3:9872"); + map.put("peer4_ID", "host4:9872"); + List raftPeerProtos = new ArrayList<>(); + for (Map.Entry en : map.entrySet()) { + raftPeerProtos.add(RaftPeerProto.newBuilder() + .setId(ByteString.copyFrom(en.getKey().getBytes(StandardCharsets.UTF_8))).setAddress(en.getValue()) + .setStartupRole(RaftPeerRole.FOLLOWER).build()); + } + + LogEntryProto generateLogEntryProto = LogEntryProto.newBuilder() + .setConfigurationEntry(RaftConfigurationProto.newBuilder().addAllPeers(raftPeerProtos).build()) + .setIndex(index).build(); + try (OutputStream out = Files.newOutputStream(path)) { + generateLogEntryProto.writeTo(out); + } + } + + private boolean containsPeerId(String str) { + return p.matcher(str).find(); + } + +} From a72d14da082dae989cf00a7a0fdee9aa4cf299e9 Mon Sep 17 00:00:00 2001 From: Tsz-Wo Nicholas Sze Date: Fri, 12 Apr 2024 00:58:23 -0700 Subject: [PATCH 022/397] RATIS-1504. Add timeout handling to DataStreamManagement#checkSuccessRemoteWrite. (#1064) --- .../netty/client/NettyClientReplies.java | 29 ++++++++++--------- .../netty/client/NettyClientStreamRpc.java | 9 ++---- .../netty/server/DataStreamManagement.java | 14 +++++++-- 3 files changed, 30 insertions(+), 22 deletions(-) diff --git a/ratis-netty/src/main/java/org/apache/ratis/netty/client/NettyClientReplies.java b/ratis-netty/src/main/java/org/apache/ratis/netty/client/NettyClientReplies.java index fc97b6fe34..4c49b1d160 100644 --- a/ratis-netty/src/main/java/org/apache/ratis/netty/client/NettyClientReplies.java +++ b/ratis-netty/src/main/java/org/apache/ratis/netty/client/NettyClientReplies.java @@ -30,11 +30,10 @@ import java.util.Map; import java.util.Objects; -import java.util.Optional; import java.util.concurrent.CompletableFuture; import java.util.concurrent.ConcurrentHashMap; import java.util.concurrent.ConcurrentMap; -import java.util.concurrent.atomic.AtomicReference; +import java.util.function.Supplier; public class NettyClientReplies { public static final Logger LOG = LoggerFactory.getLogger(NettyClientReplies.class); @@ -56,8 +55,8 @@ class ReplyMap { ReplyEntry submitRequest(RequestEntry requestEntry, boolean isClose, CompletableFuture f) { LOG.debug("put {} to the map for {}", requestEntry, clientInvocationId); - final MemoizedSupplier replySupplier = MemoizedSupplier.valueOf(() -> new ReplyEntry(isClose, f)); - return map.computeIfAbsent(requestEntry, r -> replySupplier.get()); + // ConcurrentHashMap.computeIfAbsent javadoc: the function is applied at most once per key. + return map.computeIfAbsent(requestEntry, r -> new ReplyEntry(isClose, f)); } void receiveReply(DataStreamReply reply) { @@ -147,7 +146,7 @@ public String toString() { static class ReplyEntry { private final boolean isClosed; private final CompletableFuture replyFuture; - private final AtomicReference> timeoutFuture = new AtomicReference<>(); + private ScheduledFuture timeoutFuture; // for reply timeout ReplyEntry(boolean isClosed, CompletableFuture replyFuture) { this.isClosed = isClosed; @@ -158,22 +157,26 @@ boolean isClosed() { return isClosed; } - void complete(DataStreamReply reply) { - cancelTimeoutFuture(); + synchronized void complete(DataStreamReply reply) { + cancel(timeoutFuture); replyFuture.complete(reply); } - void completeExceptionally(Throwable t) { - cancelTimeoutFuture(); + synchronized void completeExceptionally(Throwable t) { + cancel(timeoutFuture); replyFuture.completeExceptionally(t); } - private void cancelTimeoutFuture() { - Optional.ofNullable(timeoutFuture.get()).ifPresent(f -> f.cancel(false)); + static void cancel(ScheduledFuture future) { + if (future != null) { + future.cancel(true); + } } - void setTimeoutFuture(ScheduledFuture timeoutFuture) { - this.timeoutFuture.compareAndSet(null, timeoutFuture); + synchronized void scheduleTimeout(Supplier> scheduleMethod) { + if (!replyFuture.isDone()) { + timeoutFuture = scheduleMethod.get(); + } } } } diff --git a/ratis-netty/src/main/java/org/apache/ratis/netty/client/NettyClientStreamRpc.java b/ratis-netty/src/main/java/org/apache/ratis/netty/client/NettyClientStreamRpc.java index b2dc3812f2..534fcc5818 100644 --- a/ratis-netty/src/main/java/org/apache/ratis/netty/client/NettyClientStreamRpc.java +++ b/ratis-netty/src/main/java/org/apache/ratis/netty/client/NettyClientStreamRpc.java @@ -53,7 +53,6 @@ import org.apache.ratis.thirdparty.io.netty.handler.codec.ByteToMessageDecoder; import org.apache.ratis.thirdparty.io.netty.handler.codec.MessageToMessageEncoder; import org.apache.ratis.thirdparty.io.netty.handler.ssl.SslContext; -import org.apache.ratis.thirdparty.io.netty.util.concurrent.ScheduledFuture; import org.apache.ratis.util.JavaUtils; import org.apache.ratis.util.MemoizedSupplier; import org.apache.ratis.util.NetUtils; @@ -466,15 +465,13 @@ public CompletableFuture streamAsync(DataStreamRequest request) LOG.debug("{}: write after {}", this, request); final TimeDuration timeout = isClose ? closeTimeout : requestTimeout; - // if reply success cancel this future - final ScheduledFuture timeoutFuture = channel.eventLoop().schedule(() -> { + replyEntry.scheduleTimeout(() -> channel.eventLoop().schedule(() -> { if (!f.isDone()) { f.completeExceptionally(new TimeoutIOException( - "Timeout " + timeout + ": Failed to send " + request + " channel: " + channel)); + "Timeout " + timeout + ": Failed to send " + request + " via channel " + channel)); replyMap.fail(requestEntry); } - }, timeout.toLong(timeout.getUnit()), timeout.getUnit()); - replyEntry.setTimeoutFuture(timeoutFuture); + }, timeout.getDuration(), timeout.getUnit())); } }); return f; diff --git a/ratis-netty/src/main/java/org/apache/ratis/netty/server/DataStreamManagement.java b/ratis-netty/src/main/java/org/apache/ratis/netty/server/DataStreamManagement.java index e265d8b924..74d5cd7fde 100644 --- a/ratis-netty/src/main/java/org/apache/ratis/netty/server/DataStreamManagement.java +++ b/ratis-netty/src/main/java/org/apache/ratis/netty/server/DataStreamManagement.java @@ -18,6 +18,7 @@ package org.apache.ratis.netty.server; +import org.apache.ratis.client.RaftClientConfigKeys; import org.apache.ratis.client.impl.ClientProtoUtils; import org.apache.ratis.client.impl.DataStreamClientImpl.DataStreamOutputImpl; import org.apache.ratis.conf.RaftProperties; @@ -219,6 +220,7 @@ private Set getSuccessors(RaftPeerId peerId) { private final ChannelMap channels; private final ExecutorService requestExecutor; private final ExecutorService writeExecutor; + private final TimeDuration requestTimeout; private final NettyServerStreamRpcMetrics nettyServerStreamRpcMetrics; @@ -235,6 +237,7 @@ private Set getSuccessors(RaftPeerId peerId) { this.writeExecutor = ConcurrentUtils.newThreadPoolWithMax(useCachedThreadPool, RaftServerConfigKeys.DataStream.asyncWriteThreadPoolSize(properties), name + "-write-"); + this.requestTimeout = RaftClientConfigKeys.DataStream.requestTimeout(server.getProperties()); this.nettyServerStreamRpcMetrics = metrics; } @@ -339,7 +342,7 @@ static DataStreamReplyByteBuffer newDataStreamReplyByteBuffer(DataStreamRequestB .build(); } - static void sendReply(List> remoteWrites, + private void sendReply(List> remoteWrites, DataStreamRequestByteBuf request, long bytesWritten, Collection commitInfos, ChannelHandlerContext ctx) { final boolean success = checkSuccessRemoteWrite(remoteWrites, bytesWritten, request); @@ -493,10 +496,15 @@ static void assertReplyCorrespondingToRequest( Preconditions.assertTrue(request.getStreamOffset() == reply.getStreamOffset()); } - static boolean checkSuccessRemoteWrite(List> replyFutures, long bytesWritten, + private boolean checkSuccessRemoteWrite(List> replyFutures, long bytesWritten, final DataStreamRequestByteBuf request) { for (CompletableFuture replyFuture : replyFutures) { - final DataStreamReply reply = replyFuture.join(); + final DataStreamReply reply; + try { + reply = replyFuture.get(requestTimeout.getDuration(), requestTimeout.getUnit()); + } catch (Exception e) { + throw new CompletionException("Failed to get reply for bytesWritten=" + bytesWritten + ", " + request, e); + } assertReplyCorrespondingToRequest(request, reply); if (!reply.isSuccess()) { LOG.warn("reply is not success, request: {}", request); From 202a56753c6e19f47403ec328dfb9bf8b4cf54b6 Mon Sep 17 00:00:00 2001 From: William Song <48054931+SzyWilliam@users.noreply.github.com> Date: Sat, 13 Apr 2024 02:05:11 +0800 Subject: [PATCH 023/397] RATIS-2056. Update purge index when snapshot installed. (#1065) --- .../org/apache/ratis/server/raftlog/RaftLogBase.java | 12 +++++++++--- .../server/raftlog/segmented/SegmentedRaftLog.java | 4 ++-- 2 files changed, 11 insertions(+), 5 deletions(-) diff --git a/ratis-server/src/main/java/org/apache/ratis/server/raftlog/RaftLogBase.java b/ratis-server/src/main/java/org/apache/ratis/server/raftlog/RaftLogBase.java index be2ebcb45f..cfcf1ccd1b 100644 --- a/ratis-server/src/main/java/org/apache/ratis/server/raftlog/RaftLogBase.java +++ b/ratis-server/src/main/java/org/apache/ratis/server/raftlog/RaftLogBase.java @@ -139,6 +139,14 @@ public boolean updateCommitIndex(long majorityIndex, long currentTerm, boolean i return false; } + protected void updatePurgeIndex(Long purged) { + try (AutoCloseableLock writeLock = writeLock()) { + if (purged != null) { + purgeIndex.updateToMax(purged, infoIndexChange); + } + } + } + protected void updateSnapshotIndexFromStateMachine() { updateSnapshotIndex(getSnapshotIndexFromStateMachine.getAsLong()); } @@ -330,9 +338,7 @@ public final CompletableFuture purge(long suggestedIndex) { LOG.info("{}: purge {}", getName(), suggestedIndex); final long finalSuggestedIndex = suggestedIndex; return purgeImpl(suggestedIndex).whenComplete((purged, e) -> { - if (purged != null) { - purgeIndex.updateToMax(purged, infoIndexChange); - } + updatePurgeIndex(purged); if (e != null) { LOG.warn(getName() + ": Failed to purge " + finalSuggestedIndex, e); } diff --git a/ratis-server/src/main/java/org/apache/ratis/server/raftlog/segmented/SegmentedRaftLog.java b/ratis-server/src/main/java/org/apache/ratis/server/raftlog/segmented/SegmentedRaftLog.java index 975e00a81b..d83c7dcc00 100644 --- a/ratis-server/src/main/java/org/apache/ratis/server/raftlog/segmented/SegmentedRaftLog.java +++ b/ratis-server/src/main/java/org/apache/ratis/server/raftlog/segmented/SegmentedRaftLog.java @@ -266,7 +266,7 @@ private void loadLogSegments(long lastIndexInSnapshot, if (!cache.isEmpty() && cache.getEndIndex() < lastIndexInSnapshot) { LOG.warn("End log index {} is smaller than last index in snapshot {}", cache.getEndIndex(), lastIndexInSnapshot); - purgeImpl(lastIndexInSnapshot); + purgeImpl(lastIndexInSnapshot).whenComplete((purged, e) -> updatePurgeIndex(purged)); } } } @@ -522,7 +522,7 @@ public CompletableFuture onSnapshotInstalled(long lastSnapshotIndex) { cacheEviction.signal(); } } - return purgeImpl(lastSnapshotIndex); + return purgeImpl(lastSnapshotIndex).whenComplete((purged, e) -> updatePurgeIndex(purged)); } @Override From 644af85a875c4955492aa187263c6adab962821c Mon Sep 17 00:00:00 2001 From: Tsz-Wo Nicholas Sze Date: Mon, 15 Apr 2024 23:28:53 -0700 Subject: [PATCH 024/397] RATIS-2057. Add back JavaUtils.attemptUntilTrue(..). (#1066) --- .../main/java/org/apache/ratis/util/JavaUtils.java | 13 +++++++++++++ 1 file changed, 13 insertions(+) diff --git a/ratis-common/src/main/java/org/apache/ratis/util/JavaUtils.java b/ratis-common/src/main/java/org/apache/ratis/util/JavaUtils.java index f689006db2..7d1d75309a 100644 --- a/ratis-common/src/main/java/org/apache/ratis/util/JavaUtils.java +++ b/ratis-common/src/main/java/org/apache/ratis/util/JavaUtils.java @@ -41,6 +41,7 @@ import java.util.concurrent.ConcurrentMap; import java.util.concurrent.TimeUnit; import java.util.function.BiConsumer; +import java.util.function.BooleanSupplier; import java.util.function.Consumer; import java.util.function.Function; import java.util.function.Supplier; @@ -244,6 +245,18 @@ static void attempt( attemptRepeatedly(CheckedRunnable.asCheckedSupplier(runnable), numAttempts, sleepTime, name, log); } + /** Attempt to wait the given condition to return true multiple times. */ + static void attemptUntilTrue( + BooleanSupplier condition, int numAttempts, TimeDuration sleepTime, String name, Logger log) + throws InterruptedException { + Objects.requireNonNull(condition, "condition == null"); + attempt(() -> { + if (!condition.getAsBoolean()) { + throw new IllegalStateException("Condition " + name + " is false."); + } + }, numAttempts, sleepTime, name, log); + } + static Timer runRepeatedly(Runnable runnable, long delay, long period, TimeUnit unit) { final Timer timer = new Timer(true); timer.schedule(new TimerTask() { From 555a7707454773e904273a1e99905fc08f7caf27 Mon Sep 17 00:00:00 2001 From: Symious <14933944+symious@users.noreply.github.com> Date: Fri, 19 Apr 2024 23:20:16 +0800 Subject: [PATCH 025/397] RATIS-2055. Move notifyTermIndexUpdated after leader.checkReady (#1068) --- .../ratis/server/impl/RaftServerImpl.java | 26 +++++++++++++------ 1 file changed, 18 insertions(+), 8 deletions(-) diff --git a/ratis-server/src/main/java/org/apache/ratis/server/impl/RaftServerImpl.java b/ratis-server/src/main/java/org/apache/ratis/server/impl/RaftServerImpl.java index 7ec0444407..4ab4a51ad0 100644 --- a/ratis-server/src/main/java/org/apache/ratis/server/impl/RaftServerImpl.java +++ b/ratis-server/src/main/java/org/apache/ratis/server/impl/RaftServerImpl.java @@ -1787,17 +1787,18 @@ TransactionContext getTransactionContext(LogEntryProto entry, Boolean createNew) } CompletableFuture applyLogToStateMachine(LogEntryProto next) throws RaftLogIOException { - if (!next.hasStateMachineLogEntry()) { - stateMachine.event().notifyTermIndexUpdated(next.getTerm(), next.getIndex()); - } + CompletableFuture messageFuture = null; - if (next.hasConfigurationEntry()) { + switch (next.getLogEntryBodyCase()) { + case CONFIGURATIONENTRY: // the reply should have already been set. only need to record // the new conf in the metadata file and notify the StateMachine. state.writeRaftConfiguration(next); - stateMachine.event().notifyConfigurationChanged(next.getTerm(), next.getIndex(), next.getConfigurationEntry()); + stateMachine.event().notifyConfigurationChanged(next.getTerm(), next.getIndex(), + next.getConfigurationEntry()); role.getLeaderState().ifPresent(leader -> leader.checkReady(next)); - } else if (next.hasStateMachineLogEntry()) { + break; + case STATEMACHINELOGENTRY: TransactionContext trx = getTransactionContext(next, true); final ClientInvocationId invocationId = ClientInvocationId.valueOf(next.getStateMachineLogEntry()); writeIndexCache.add(invocationId.getClientId(), ((TransactionContextImpl) trx).getLogIndexFuture()); @@ -1807,12 +1808,21 @@ CompletableFuture applyLogToStateMachine(LogEntryProto next) throws Raf trx = stateMachine.applyTransactionSerial(trx); final CompletableFuture stateMachineFuture = stateMachine.applyTransaction(trx); - return replyPendingRequest(invocationId, TermIndex.valueOf(next), stateMachineFuture); + messageFuture = replyPendingRequest(invocationId, TermIndex.valueOf(next), stateMachineFuture); } catch (Exception e) { throw new RaftLogIOException(e); } + break; + case METADATAENTRY: + break; + default: + throw new IllegalStateException("Unexpected LogEntryBodyCase " + next.getLogEntryBodyCase() + ", next=" + next); } - return null; + + if (next.getLogEntryBodyCase() != LogEntryProto.LogEntryBodyCase.STATEMACHINELOGENTRY) { + stateMachine.event().notifyTermIndexUpdated(next.getTerm(), next.getIndex()); + } + return messageFuture; } /** From bcbfff938886de4d442c7791c7af3259bb3b6b60 Mon Sep 17 00:00:00 2001 From: Potato Date: Thu, 25 Apr 2024 19:44:55 +0800 Subject: [PATCH 026/397] RATIS-2060 Avoid unexpected shutdown of RaftServerProxy during membership change (#1069) --- .../main/java/org/apache/ratis/server/impl/LeaderElection.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/ratis-server/src/main/java/org/apache/ratis/server/impl/LeaderElection.java b/ratis-server/src/main/java/org/apache/ratis/server/impl/LeaderElection.java index 53e83d866e..d738c87578 100644 --- a/ratis-server/src/main/java/org/apache/ratis/server/impl/LeaderElection.java +++ b/ratis-server/src/main/java/org/apache/ratis/server/impl/LeaderElection.java @@ -333,7 +333,7 @@ private boolean askForVotes(Phase phase, int round) throws InterruptedException, return true; case NOT_IN_CONF: case SHUTDOWN: - server.getRaftServer().close(); + server.close(); server.getStateMachine().event().notifyServerShutdown(server.getRoleInfoProto()); return false; case TIMEOUT: From 7fc25331d57e15626ecde830bbc2c58ebe235190 Mon Sep 17 00:00:00 2001 From: Hongbing Wang <284734261@qq.com> Date: Fri, 26 Apr 2024 23:14:07 +0800 Subject: [PATCH 027/397] RATIS-2061. Fix setCloseThreshold parameter in RaftServerConfigKeys (#1070) --- .../apache/ratis/server/RaftServerConfigKeys.java | 14 ++++++++++++-- 1 file changed, 12 insertions(+), 2 deletions(-) diff --git a/ratis-server-api/src/main/java/org/apache/ratis/server/RaftServerConfigKeys.java b/ratis-server-api/src/main/java/org/apache/ratis/server/RaftServerConfigKeys.java index 7419ca095f..f6dcbaa66b 100644 --- a/ratis-server-api/src/main/java/org/apache/ratis/server/RaftServerConfigKeys.java +++ b/ratis-server-api/src/main/java/org/apache/ratis/server/RaftServerConfigKeys.java @@ -77,9 +77,14 @@ static TimeDuration sleepDeviationThreshold(RaftProperties properties) { return getTimeDuration(properties.getTimeDuration(SLEEP_DEVIATION_THRESHOLD_DEFAULT.getUnit()), SLEEP_DEVIATION_THRESHOLD_KEY, SLEEP_DEVIATION_THRESHOLD_DEFAULT, getDefaultLog()); } + /** @deprecated use {@link #setSleepDeviationThreshold(RaftProperties, TimeDuration)}. */ + @Deprecated static void setSleepDeviationThreshold(RaftProperties properties, int thresholdMs) { setInt(properties::setInt, SLEEP_DEVIATION_THRESHOLD_KEY, thresholdMs); } + static void setSleepDeviationThreshold(RaftProperties properties, TimeDuration threshold) { + setTimeDuration(properties::setTimeDuration, SLEEP_DEVIATION_THRESHOLD_KEY, threshold); + } String CLOSE_THRESHOLD_KEY = PREFIX + ".close.threshold"; TimeDuration CLOSE_THRESHOLD_DEFAULT = TimeDuration.valueOf(60, TimeUnit.SECONDS); @@ -87,8 +92,13 @@ static TimeDuration closeThreshold(RaftProperties properties) { return getTimeDuration(properties.getTimeDuration(CLOSE_THRESHOLD_DEFAULT.getUnit()), CLOSE_THRESHOLD_KEY, CLOSE_THRESHOLD_DEFAULT, getDefaultLog()); } - static void setCloseThreshold(RaftProperties properties, int thresholdMs) { - setInt(properties::setInt, CLOSE_THRESHOLD_KEY, thresholdMs); + /** @deprecated use {@link #setCloseThreshold(RaftProperties, TimeDuration)}. */ + @Deprecated + static void setCloseThreshold(RaftProperties properties, int thresholdSec) { + setInt(properties::setInt, CLOSE_THRESHOLD_KEY, thresholdSec); + } + static void setCloseThreshold(RaftProperties properties, TimeDuration threshold) { + setTimeDuration(properties::setTimeDuration, CLOSE_THRESHOLD_KEY, threshold); } /** From 8b01c4139a8c0e50f28f179603ca6d2bda3ba135 Mon Sep 17 00:00:00 2001 From: Potato Date: Tue, 30 Apr 2024 05:24:26 +0800 Subject: [PATCH 028/397] RATIS-2065. Avoid the out-of-heap memory OOM phenomenon of frequent creation and deletion of Raft group scenarios. (#1071) --- .../ratis/server/raftlog/segmented/SegmentedRaftLogWorker.java | 2 ++ 1 file changed, 2 insertions(+) diff --git a/ratis-server/src/main/java/org/apache/ratis/server/raftlog/segmented/SegmentedRaftLogWorker.java b/ratis-server/src/main/java/org/apache/ratis/server/raftlog/segmented/SegmentedRaftLogWorker.java index 502e279e58..bfeca7f87f 100644 --- a/ratis-server/src/main/java/org/apache/ratis/server/raftlog/segmented/SegmentedRaftLogWorker.java +++ b/ratis-server/src/main/java/org/apache/ratis/server/raftlog/segmented/SegmentedRaftLogWorker.java @@ -38,6 +38,7 @@ import org.apache.ratis.statemachine.StateMachine; import org.apache.ratis.statemachine.StateMachine.DataStream; import org.apache.ratis.statemachine.TransactionContext; +import org.apache.ratis.thirdparty.io.netty.util.internal.PlatformDependent; import org.apache.ratis.util.*; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -242,6 +243,7 @@ void close() { Optional.ofNullable(flushExecutor).ifPresent(ExecutorService::shutdown); ConcurrentUtils.shutdownAndWait(TimeDuration.ONE_SECOND.multiply(3), workerThreadExecutor, timeout -> LOG.warn("{}: shutdown timeout in " + timeout, name)); + PlatformDependent.freeDirectBuffer(writeBuffer); IOUtils.cleanup(LOG, out); LOG.info("{} close()", name); } From 0c35451aac33aaf1eebe817e91d7559de9fb1ce4 Mon Sep 17 00:00:00 2001 From: Sadanand Shenoy Date: Sat, 4 May 2024 00:10:53 +0530 Subject: [PATCH 029/397] RATIS-2069. RaftMetaConf command sets incorrect peerAddress pattern. (#1075) --- .../ratis/shell/cli/sh/local/RaftMetaConfCommand.java | 2 +- .../ratis/shell/cli/sh/LocalCommandIntegrationTest.java | 9 +++++++-- 2 files changed, 8 insertions(+), 3 deletions(-) diff --git a/ratis-shell/src/main/java/org/apache/ratis/shell/cli/sh/local/RaftMetaConfCommand.java b/ratis-shell/src/main/java/org/apache/ratis/shell/cli/sh/local/RaftMetaConfCommand.java index 9f0558c5ea..e258d863b8 100644 --- a/ratis-shell/src/main/java/org/apache/ratis/shell/cli/sh/local/RaftMetaConfCommand.java +++ b/ratis-shell/src/main/java/org/apache/ratis/shell/cli/sh/local/RaftMetaConfCommand.java @@ -91,7 +91,7 @@ public int run(CommandLine cl) throws IOException { } InetSocketAddress inetSocketAddress = parseInetSocketAddress( peerIdWithAddressArray[peerIdWithAddressArray.length - 1]); - String addressString = inetSocketAddress.toString(); + String addressString = inetSocketAddress.getHostString() + ":" + inetSocketAddress.getPort(); if (addresses.contains(addressString)) { printf("Found duplicated address: %s. Please make sure the address of peer have no duplicated value.", addressString); diff --git a/ratis-test/src/test/java/org/apache/ratis/shell/cli/sh/LocalCommandIntegrationTest.java b/ratis-test/src/test/java/org/apache/ratis/shell/cli/sh/LocalCommandIntegrationTest.java index 4a07e372ca..a85c247d66 100644 --- a/ratis-test/src/test/java/org/apache/ratis/shell/cli/sh/LocalCommandIntegrationTest.java +++ b/ratis-test/src/test/java/org/apache/ratis/shell/cli/sh/LocalCommandIntegrationTest.java @@ -78,11 +78,11 @@ public void testRunMethod(@TempDir Path tempDir) throws Exception { int index = 1; generateRaftConf(tempDir.resolve(RAFT_META_CONF), index); - String[] testPeersListArray = {"peer1_ID|host1:9872,peer2_ID|host2:9872,peer3_ID|host3:9872", + String[] testPeersListArray = {"peer1_ID|localhost:9872,peer2_ID|host2:9872,peer3_ID|host3:9872", "host1:9872,host2:9872,host3:9872"}; for (String peersListStr : testPeersListArray) { - generateRaftConf(tempDir, index); + generateRaftConf(tempDir.resolve(RAFT_META_CONF), index); StringPrintStream out = new StringPrintStream(); RatisShell shell = new RatisShell(out.getPrintStream()); int ret = shell.run("local", "raftMetaConf", "-peers", peersListStr, "-path", tempDir.toString()); @@ -99,6 +99,11 @@ public void testRunMethod(@TempDir Path tempDir) throws Exception { Assertions.assertEquals(index + 1, indexFromNewConf); + String addressRegex = "^[a-zA-Z0-9.-]+:\\d+$"; + Pattern pattern = Pattern.compile(addressRegex); + peers.forEach(p -> Assertions.assertTrue( + pattern.matcher(p.getAddress()).matches())); + String peersListStrFromNewMetaConf; if (containsPeerId(peersListStr)) { peersListStrFromNewMetaConf = peers.stream() From 6d9827dae370563f3ef6a33690967c8d7c1eb182 Mon Sep 17 00:00:00 2001 From: Duong Nguyen Date: Fri, 3 May 2024 11:48:52 -0700 Subject: [PATCH 030/397] RATIS-2068. Avoid logging raw StateMachine data body. (#1074) --- .../server/GrpcServerProtocolService.java | 2 +- .../ratis/server/impl/RaftServerImpl.java | 5 +++-- .../ratis/server/raftlog/LogProtoUtils.java | 19 ++++++++++++------- .../ratis/server/raftlog/RaftLogBase.java | 4 ++-- .../raftlog/segmented/SegmentedRaftLog.java | 8 ++++---- .../ratis/server/util/ServerStringUtils.java | 8 ++++++-- 6 files changed, 28 insertions(+), 18 deletions(-) diff --git a/ratis-grpc/src/main/java/org/apache/ratis/grpc/server/GrpcServerProtocolService.java b/ratis-grpc/src/main/java/org/apache/ratis/grpc/server/GrpcServerProtocolService.java index 766e14321a..77d79e64f3 100644 --- a/ratis-grpc/src/main/java/org/apache/ratis/grpc/server/GrpcServerProtocolService.java +++ b/ratis-grpc/src/main/java/org/apache/ratis/grpc/server/GrpcServerProtocolService.java @@ -236,7 +236,7 @@ long getCallId(AppendEntriesRequestProto request) { @Override String requestToString(AppendEntriesRequestProto request) { - return ServerStringUtils.toAppendEntriesRequestString(request); + return ServerStringUtils.toAppendEntriesRequestString(request, null); } @Override diff --git a/ratis-server/src/main/java/org/apache/ratis/server/impl/RaftServerImpl.java b/ratis-server/src/main/java/org/apache/ratis/server/impl/RaftServerImpl.java index 4ab4a51ad0..f6d8dceb03 100644 --- a/ratis-server/src/main/java/org/apache/ratis/server/impl/RaftServerImpl.java +++ b/ratis-server/src/main/java/org/apache/ratis/server/impl/RaftServerImpl.java @@ -1459,7 +1459,8 @@ public CompletableFuture appendEntriesAsync(AppendEntri return appendEntriesAsync(leaderId, request.getCallId(), previous, r); } catch(Exception t) { - LOG.error("{}: Failed appendEntries* {}", getMemberId(), toAppendEntriesRequestString(r), t); + LOG.error("{}: Failed appendEntries* {}", getMemberId(), + toAppendEntriesRequestString(r, stateMachine::toStateMachineLogEntryString), t); throw IOUtils.asIOException(t); } } @@ -1515,7 +1516,7 @@ private CompletableFuture appendEntriesAsync(RaftPeerId final List entries = proto.getEntriesList(); final boolean isHeartbeat = entries.isEmpty(); logAppendEntries(isHeartbeat, () -> getMemberId() + ": appendEntries* " - + toAppendEntriesRequestString(proto)); + + toAppendEntriesRequestString(proto, stateMachine::toStateMachineLogEntryString)); final long leaderTerm = proto.getLeaderTerm(); final long currentTerm; diff --git a/ratis-server/src/main/java/org/apache/ratis/server/raftlog/LogProtoUtils.java b/ratis-server/src/main/java/org/apache/ratis/server/raftlog/LogProtoUtils.java index de06faf63e..3b6000e741 100644 --- a/ratis-server/src/main/java/org/apache/ratis/server/raftlog/LogProtoUtils.java +++ b/ratis-server/src/main/java/org/apache/ratis/server/raftlog/LogProtoUtils.java @@ -19,7 +19,6 @@ import org.apache.ratis.proto.RaftProtos.*; import org.apache.ratis.protocol.ClientId; -import org.apache.ratis.protocol.ClientInvocationId; import org.apache.ratis.protocol.RaftClientRequest; import org.apache.ratis.protocol.RaftPeer; import org.apache.ratis.server.RaftConfiguration; @@ -45,9 +44,10 @@ public static String toLogEntryString(LogEntryProto entry, Function proto -> "" + ClientInvocationId.valueOf(proto)) - .apply(entry.getStateMachineLogEntry()); + if (function == null) { + function = LogProtoUtils::stateMachineLogEntryProtoToString; + } + s = ", " + function.apply(entry.getStateMachineLogEntry()); } else if (entry.hasMetadataEntry()) { final MetadataProto metadata = entry.getMetadataEntry(); s = "(c:" + metadata.getCommitIndex() + ")"; @@ -69,7 +69,11 @@ static String peersToString(List peers) { } static String stateMachineLogEntryProtoToString(StateMachineLogEntryProto p) { - return "logData:" + p.getLogData() + ", stateMachineEntry:" + p.getType() + ":" + p.getStateMachineEntry(); + final StateMachineEntryProto stateMachineEntry = p.getStateMachineEntry(); + return p.getType() + + ": logData.size=" + p.getLogData().size() + + ", stateMachineData.size=" + stateMachineEntry.getStateMachineData().size() + + ", logEntryProtoSerializedSize=" + stateMachineEntry.getLogEntryProtoSerializedSize(); } public static String toLogEntryString(LogEntryProto entry) { @@ -81,10 +85,11 @@ public static String toLogEntriesString(List entries) { : entries.stream().map(LogProtoUtils::toLogEntryString).collect(Collectors.toList()).toString(); } - public static String toLogEntriesShortString(List entries) { + public static String toLogEntriesShortString(List entries, + Function stateMachineToString) { return entries == null ? null : entries.isEmpty()? "" - : "size=" + entries.size() + ", first=" + LogProtoUtils.toLogEntryString(entries.get(0)); + : "size=" + entries.size() + ", first=" + toLogEntryString(entries.get(0), stateMachineToString); } public static LogEntryProto toLogEntryProto(RaftConfiguration conf, Long term, long index) { diff --git a/ratis-server/src/main/java/org/apache/ratis/server/raftlog/RaftLogBase.java b/ratis-server/src/main/java/org/apache/ratis/server/raftlog/RaftLogBase.java index cfcf1ccd1b..1ad108aefe 100644 --- a/ratis-server/src/main/java/org/apache/ratis/server/raftlog/RaftLogBase.java +++ b/ratis-server/src/main/java/org/apache/ratis/server/raftlog/RaftLogBase.java @@ -195,10 +195,10 @@ private long appendImpl(long term, TransactionContext operation) throws StateMac } appendEntry(e, operation).whenComplete((returned, t) -> { if (t != null) { - LOG.error(name + ": Failed to write log entry " + LogProtoUtils.toLogEntryString(e), t); + LOG.error(name + ": Failed to write log entry " + toLogEntryString(e), t); } else if (returned != nextIndex) { LOG.error("{}: Indices mismatched: returned index={} but nextIndex={} for log entry {}", - name, returned, nextIndex, LogProtoUtils.toLogEntryString(e)); + name, returned, nextIndex, toLogEntryString(e)); } else { return; // no error } diff --git a/ratis-server/src/main/java/org/apache/ratis/server/raftlog/segmented/SegmentedRaftLog.java b/ratis-server/src/main/java/org/apache/ratis/server/raftlog/segmented/SegmentedRaftLog.java index d83c7dcc00..def472a607 100644 --- a/ratis-server/src/main/java/org/apache/ratis/server/raftlog/segmented/SegmentedRaftLog.java +++ b/ratis-server/src/main/java/org/apache/ratis/server/raftlog/segmented/SegmentedRaftLog.java @@ -318,8 +318,7 @@ public EntryWithData getEntryWithData(long index) throws RaftLogIOException { } return newEntryWithData(entry, future); } catch (Exception e) { - final String err = getName() + ": Failed readStateMachineData for " + - LogProtoUtils.toLogEntryString(entry); + final String err = getName() + ": Failed readStateMachineData for " + toLogEntryString(entry); LOG.error(err, e); throw new RaftLogIOException(err, JavaUtils.unwrapCompletionException(e)); } @@ -439,7 +438,7 @@ protected CompletableFuture appendEntryImpl(LogEntryProto entry, Transacti } return write.getFuture().whenComplete((clientReply, exception) -> appendEntryTimerContext.stop()); } catch (Exception e) { - LOG.error("{}: Failed to append {}", getName(), LogProtoUtils.toLogEntryString(entry), e); + LOG.error("{}: Failed to append {}", getName(), toLogEntryString(entry), e); throw e; } } @@ -543,7 +542,8 @@ SegmentedRaftLogCache getRaftLogCache() { @Override public String toLogEntryString(LogEntryProto logEntry) { - return LogProtoUtils.toLogEntryString(logEntry, stateMachine::toStateMachineLogEntryString); + return LogProtoUtils.toLogEntryString(logEntry, stateMachine != null ? + stateMachine::toStateMachineLogEntryString : null); } public static Builder newBuilder() { diff --git a/ratis-server/src/main/java/org/apache/ratis/server/util/ServerStringUtils.java b/ratis-server/src/main/java/org/apache/ratis/server/util/ServerStringUtils.java index 284664d012..6e0fce4d66 100644 --- a/ratis-server/src/main/java/org/apache/ratis/server/util/ServerStringUtils.java +++ b/ratis-server/src/main/java/org/apache/ratis/server/util/ServerStringUtils.java @@ -23,11 +23,13 @@ import org.apache.ratis.proto.RaftProtos.InstallSnapshotRequestProto; import org.apache.ratis.proto.RaftProtos.LogEntryProto; import org.apache.ratis.proto.RaftProtos.RequestVoteReplyProto; +import org.apache.ratis.proto.RaftProtos.StateMachineLogEntryProto; import org.apache.ratis.server.protocol.TermIndex; import org.apache.ratis.server.raftlog.LogProtoUtils; import org.apache.ratis.util.ProtoUtils; import java.util.List; +import java.util.function.Function; /** * This class provides convenient utilities for converting Protocol Buffers messages to strings. @@ -41,7 +43,8 @@ public final class ServerStringUtils { private ServerStringUtils() {} - public static String toAppendEntriesRequestString(AppendEntriesRequestProto request) { + public static String toAppendEntriesRequestString(AppendEntriesRequestProto request, + Function stateMachineToString) { if (request == null) { return null; } @@ -51,7 +54,8 @@ public static String toAppendEntriesRequestString(AppendEntriesRequestProto requ + ",previous=" + TermIndex.valueOf(request.getPreviousLog()) + ",leaderCommit=" + request.getLeaderCommit() + ",initializing? " + request.getInitializing() - + "," + (entries.isEmpty()? "HEARTBEAT" : "entries: " + LogProtoUtils.toLogEntriesShortString(entries)); + + "," + (entries.isEmpty()? "HEARTBEAT" : "entries: " + + LogProtoUtils.toLogEntriesShortString(entries, stateMachineToString)); } public static String toAppendEntriesReplyString(AppendEntriesReplyProto reply) { From 5828e9682521ec67890426de253738265a29b8d7 Mon Sep 17 00:00:00 2001 From: "Doroszlai, Attila" <6454655+adoroszlai@users.noreply.github.com> Date: Sun, 5 May 2024 08:04:09 +0200 Subject: [PATCH 031/397] RATIS-2070. Skip push build for dependabot (#1077) --- .github/workflows/post-commit.yml | 8 ++++++-- 1 file changed, 6 insertions(+), 2 deletions(-) diff --git a/.github/workflows/post-commit.yml b/.github/workflows/post-commit.yml index 2d3258ccc5..22a6df756e 100644 --- a/.github/workflows/post-commit.yml +++ b/.github/workflows/post-commit.yml @@ -14,8 +14,12 @@ # limitations under the License. name: build-branch on: - - push - - pull_request + push: + branches-ignore: + - 'dependabot/**' + tags: + - '**' + pull_request: env: WITH_COVERAGE: true jobs: From 126a02dceb3919086fa28c01616c95beed592448 Mon Sep 17 00:00:00 2001 From: "dependabot[bot]" <49699333+dependabot[bot]@users.noreply.github.com> Date: Sun, 5 May 2024 13:28:02 +0200 Subject: [PATCH 032/397] RATIS-2071. Bump Bouncy Castle to 1.78.1 (#1076) --- ratis-test/pom.xml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/ratis-test/pom.xml b/ratis-test/pom.xml index 5990ae057f..01d54ba15b 100644 --- a/ratis-test/pom.xml +++ b/ratis-test/pom.xml @@ -24,7 +24,7 @@ Apache Ratis Test - 1.76 + 1.78.1 From 9b766f5af86a13827b69753ffcba5f892cb52789 Mon Sep 17 00:00:00 2001 From: "Doroszlai, Attila" <6454655+adoroszlai@users.noreply.github.com> Date: Mon, 6 May 2024 17:44:26 +0200 Subject: [PATCH 033/397] RATIS-2072. Limit timeout of CI workflow jobs (#1078) --- .github/workflows/post-commit.yml | 8 ++++++++ 1 file changed, 8 insertions(+) diff --git a/.github/workflows/post-commit.yml b/.github/workflows/post-commit.yml index 22a6df756e..13a9214445 100644 --- a/.github/workflows/post-commit.yml +++ b/.github/workflows/post-commit.yml @@ -25,6 +25,7 @@ env: jobs: build: runs-on: ubuntu-20.04 + timeout-minutes: 30 steps: - name: Checkout project uses: actions/checkout@v4 @@ -62,6 +63,7 @@ jobs: needs: - build runs-on: ubuntu-20.04 + timeout-minutes: 30 strategy: matrix: java: [ 11 ] @@ -93,6 +95,7 @@ jobs: rat: name: rat runs-on: ubuntu-20.04 + timeout-minutes: 15 steps: - name: Checkout project uses: actions/checkout@v4 @@ -116,6 +119,7 @@ jobs: author: name: author runs-on: ubuntu-20.04 + timeout-minutes: 15 steps: - name: Checkout project uses: actions/checkout@v4 @@ -130,6 +134,7 @@ jobs: unit: name: unit runs-on: ubuntu-20.04 + timeout-minutes: 60 strategy: matrix: profile: @@ -173,6 +178,7 @@ jobs: checkstyle: name: checkstyle runs-on: ubuntu-20.04 + timeout-minutes: 15 steps: - name: Checkout project uses: actions/checkout@v4 @@ -196,6 +202,7 @@ jobs: findbugs: name: findbugs runs-on: ubuntu-20.04 + timeout-minutes: 30 steps: - name: Setup java uses: actions/setup-java@v4 @@ -226,6 +233,7 @@ jobs: - build - unit runs-on: ubuntu-20.04 + timeout-minutes: 30 if: (github.repository == 'apache/ratis' || github.repository == 'apache/incubator-ratis') && github.event_name != 'pull_request' steps: - name: Checkout project From baa74d42be994383e1c26236cd1971c31258b60b Mon Sep 17 00:00:00 2001 From: "Doroszlai, Attila" <6454655+adoroszlai@users.noreply.github.com> Date: Mon, 6 May 2024 20:48:45 +0200 Subject: [PATCH 034/397] RATIS-2073. Enable Surefire process checkers (#1079) --- pom.xml | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/pom.xml b/pom.xml index 0b3eedee12..0130258b18 100644 --- a/pom.xml +++ b/pom.xml @@ -176,7 +176,7 @@ 3.3.0 4.0.6 1.6.1 - 3.0.0 + 3.0.0-M4 3.5.3 @@ -634,6 +634,7 @@ false false false + all 600 -Xmx2048m -XX:+HeapDumpOnOutOfMemoryError @{argLine} From 269bded5315d90b3359f3a9bb06a98d161d43e6d Mon Sep 17 00:00:00 2001 From: Potato Date: Wed, 8 May 2024 00:28:46 +0800 Subject: [PATCH 035/397] RATIS-2067. Enhance content for concept in ratis-docs. (#1073) --- ratis-docs/src/site/markdown/concept/index.md | 66 +++++++++++++++++-- ratis-docs/src/site/markdown/snapshot.md | 2 + 2 files changed, 61 insertions(+), 7 deletions(-) diff --git a/ratis-docs/src/site/markdown/concept/index.md b/ratis-docs/src/site/markdown/concept/index.md index 279338a5fd..6ff77b2203 100644 --- a/ratis-docs/src/site/markdown/concept/index.md +++ b/ratis-docs/src/site/markdown/concept/index.md @@ -15,24 +15,76 @@ limitations under the License. --> # Concepts -TODO: complete this section + ## RaftServer +The `RaftServer` is a core component of Apache Ratis, +responsible for handling all operations related to the RAFT protocol. +A `RaftServer` manages all the `RaftGroup`(s) within the current process. +For each group, a corresponding `RaftServer.Division` handles the core functions +such as replication of log entries, voting, and leader election within that group. +Each division can independently act as a Leader, Candidate, Follower or Listener, +with the specific role depending on the outcome of elections and the state of the protocol. + ## RaftPeer and RaftPeerId +`RaftPeer` represents a participant node in the cluster, +including the node's unique identifier, IP address, and port number. +The unique identifier is represented by the `RaftPeerId` class, +which is crucial for distinguishing different nodes within a cluster. + ## RaftGroup -## Transport (gRPC,Netty ...) +A `RaftGroup` represents a collection of `RaftPeer`(s) in a Raft protocol cluster. +Each group has a unique identifier represented by the `RaftGroupId` class. +Multiple groups can operate independently within a physical network, +while each group managing its own consistency and state replication. -## Raft Log +## Transport (gRPC, Netty, etc.) -## StateMachine -A StateMachine is the abstraction point for user-code +Ratis supports various network transport protocols for node communication, +including gRPC (default) and Netty. +These transport layers in Ratis are used for data serialization and deserialization, +as well as ensuring safe and efficient data transmission between different nodes. + +## RaftLog + +The `RaftLog` is a core component of the Raft algorithm, +used to record all state change transactions. +Once a log entry has been acknowledged by a majority of peers, +the entry becomes committed. +The Raft log is key to achieving distributed data consistency. ## Snapshot +A `Snapshot` is a point-in-time copy of the current state of the `StateMachine`. +It can be used for quick recovery of the state after system restarts, +and for transferring the state to newly joined nodes. +When a snapshot has been taken, +the log entries earlier than the snapshot can be purged +in order to free up the storage space. + ## TermIndex -## Transaction (as in applyTransaction or startTransaction in StateMachine) +`TermIndex` is an order pair of `long` integers (term, index) as defined in the Raft protocol. +Term is the logical clock in Raft. +A newly elected leader starts a new term and remains the leader for the rest of the term. +Index is the position of log entries in the Raft log. + +## StateMachine + +In Ratis, `StateMachine` is the abstraction point for user-defined code. +Developers implement specific business logic or data storage operations at this layer. +The transactions committed through the Raft protocol will be applied to it. + +### The `applyTransaction` method + +In Ratis, transaction processing is implemented by the `StateMachine` +through the `applyTransaction` method. +A transaction usually changes the state of the `StateMachine`. + +### StateMachineStorage -## StateMachineStorage +`StateMachineStorage` is a component for storing data related to the `StateMachine`. +It is for persisting the Raft log and the snapshots +such that the state can be fully recovered even after system failures. diff --git a/ratis-docs/src/site/markdown/snapshot.md b/ratis-docs/src/site/markdown/snapshot.md index f20dc19d71..a9f06511fc 100644 --- a/ratis-docs/src/site/markdown/snapshot.md +++ b/ratis-docs/src/site/markdown/snapshot.md @@ -75,6 +75,8 @@ Note that Ratis imposes a minimal creation gap between two subsequent snapshot c ```java // SnapshotManagementApi RaftClientReply create(long timeoutMs) throws IOException; + RaftClientReply create(boolean force, long timeoutMs) throws IOException; + RaftClientReply create(long creationGap, long timeoutMs) throws IOException; ``` ```java From ed1ecfd32d03a8105d275d7e033289e748e3e96e Mon Sep 17 00:00:00 2001 From: Tsz-Wo Nicholas Sze Date: Tue, 7 May 2024 11:27:36 -0700 Subject: [PATCH 036/397] RATIS-2074. Intermittent fork timeout in TestRaftWithNetty#testBasicAppendEntriesKillLeader. (#1080) --- .../org/apache/ratis/netty/NettyClient.java | 21 ++++++++++++------- .../org/apache/ratis/netty/NettyRpcProxy.java | 4 ++-- .../org/apache/ratis/netty/NettyUtils.java | 20 ++++++++++++++++++ .../java/org/apache/ratis/RaftBasicTests.java | 2 ++ 4 files changed, 38 insertions(+), 9 deletions(-) diff --git a/ratis-netty/src/main/java/org/apache/ratis/netty/NettyClient.java b/ratis-netty/src/main/java/org/apache/ratis/netty/NettyClient.java index 0cf4bd3833..efea5fd0f8 100644 --- a/ratis-netty/src/main/java/org/apache/ratis/netty/NettyClient.java +++ b/ratis-netty/src/main/java/org/apache/ratis/netty/NettyClient.java @@ -33,13 +33,17 @@ import java.net.InetSocketAddress; public class NettyClient implements Closeable { - private final LifeCycle lifeCycle = new LifeCycle(JavaUtils.getClassSimpleName(getClass())); - + private final LifeCycle lifeCycle; + private final String serverAddress; private Channel channel; + NettyClient(String serverAddress) { + this.lifeCycle = new LifeCycle(JavaUtils.getClassSimpleName(getClass()) + "-" + serverAddress); + this.serverAddress = serverAddress; + } + /** Connects to the given server address. */ - public void connect(String serverAddress, EventLoopGroup group, - ChannelInitializer initializer) + public void connect(EventLoopGroup group, ChannelInitializer initializer) throws InterruptedException { final InetSocketAddress address = NetUtils.createSocketAddr(serverAddress); @@ -57,13 +61,16 @@ public void connect(String serverAddress, EventLoopGroup group, @Override public void close() { - lifeCycle.checkStateAndClose(() -> { - channel.close().syncUninterruptibly(); - }); + lifeCycle.checkStateAndClose(() -> NettyUtils.closeChannel(channel, serverAddress)); } public ChannelFuture writeAndFlush(Object msg) { lifeCycle.assertCurrentState(LifeCycle.States.RUNNING); return channel.writeAndFlush(msg); } + + @Override + public String toString() { + return lifeCycle.toString(); + } } diff --git a/ratis-netty/src/main/java/org/apache/ratis/netty/NettyRpcProxy.java b/ratis-netty/src/main/java/org/apache/ratis/netty/NettyRpcProxy.java index b7a04b050b..b9788a8bb4 100644 --- a/ratis-netty/src/main/java/org/apache/ratis/netty/NettyRpcProxy.java +++ b/ratis-netty/src/main/java/org/apache/ratis/netty/NettyRpcProxy.java @@ -99,7 +99,7 @@ public static long getCallId(RaftNettyServerReplyProto proto) { class Connection implements Closeable { - private final NettyClient client = new NettyClient(); + private final NettyClient client = new NettyClient(peer.getAddress()); private final Queue> replies = new LinkedList<>(); @@ -137,7 +137,7 @@ protected void initChannel(SocketChannel ch) throws Exception { } }; - client.connect(peer.getAddress(), group, initializer); + client.connect(group, initializer); } synchronized ChannelFuture offer(RaftNettyServerRequestProto request, diff --git a/ratis-netty/src/main/java/org/apache/ratis/netty/NettyUtils.java b/ratis-netty/src/main/java/org/apache/ratis/netty/NettyUtils.java index 8cce291af9..37666bf189 100644 --- a/ratis-netty/src/main/java/org/apache/ratis/netty/NettyUtils.java +++ b/ratis-netty/src/main/java/org/apache/ratis/netty/NettyUtils.java @@ -22,6 +22,8 @@ import org.apache.ratis.security.TlsConf.KeyManagerConf; import org.apache.ratis.security.TlsConf.PrivateKeyConf; import org.apache.ratis.security.TlsConf.TrustManagerConf; +import org.apache.ratis.thirdparty.io.netty.channel.Channel; +import org.apache.ratis.thirdparty.io.netty.channel.ChannelFuture; import org.apache.ratis.thirdparty.io.netty.channel.EventLoopGroup; import org.apache.ratis.thirdparty.io.netty.channel.ServerChannel; import org.apache.ratis.thirdparty.io.netty.channel.epoll.Epoll; @@ -35,16 +37,19 @@ import org.apache.ratis.thirdparty.io.netty.handler.ssl.SslContext; import org.apache.ratis.thirdparty.io.netty.handler.ssl.SslContextBuilder; import org.apache.ratis.util.ConcurrentUtils; +import org.apache.ratis.util.TimeDuration; import org.slf4j.Logger; import org.slf4j.LoggerFactory; import javax.net.ssl.KeyManager; import javax.net.ssl.TrustManager; +import java.util.concurrent.TimeUnit; import java.util.concurrent.atomic.AtomicBoolean; import java.util.function.Function; public interface NettyUtils { Logger LOG = LoggerFactory.getLogger(NettyUtils.class); + TimeDuration CLOSE_TIMEOUT = TimeDuration.valueOf(5, TimeUnit.SECONDS); class Print { private static final AtomicBoolean PRINTED_EPOLL_UNAVAILABILITY_CAUSE = new AtomicBoolean(); @@ -176,4 +181,19 @@ static Class getServerChannelClass(EventLoopGroup event return eventLoopGroup instanceof EpollEventLoopGroup ? EpollServerSocketChannel.class : NioServerSocketChannel.class; } + + static void closeChannel(Channel channel, String name) { + final ChannelFuture f = channel.close(); + final boolean completed; + try { + completed = f.await(CLOSE_TIMEOUT.getDuration(), CLOSE_TIMEOUT.getUnit()); + } catch (InterruptedException e) { + Thread.currentThread().interrupt(); + LOG.info("Interrupted closeChannel {} ", name, e); + return; + } + if (!completed) { + LOG.warn("closeChannel {} is not yet completed in {}", name, CLOSE_TIMEOUT); + } + } } \ No newline at end of file diff --git a/ratis-server/src/test/java/org/apache/ratis/RaftBasicTests.java b/ratis-server/src/test/java/org/apache/ratis/RaftBasicTests.java index 13ee08c83b..9f360cb51e 100644 --- a/ratis-server/src/test/java/org/apache/ratis/RaftBasicTests.java +++ b/ratis-server/src/test/java/org/apache/ratis/RaftBasicTests.java @@ -45,6 +45,7 @@ import org.junit.jupiter.api.Assertions; import org.junit.jupiter.api.Assumptions; import org.junit.jupiter.api.Test; +import org.junit.jupiter.api.Timeout; import org.slf4j.Logger; import org.slf4j.event.Level; @@ -338,6 +339,7 @@ public String toString() { } @Test + @Timeout(value = 300) public void testWithLoad() throws Exception { runWithNewCluster(NUM_SERVERS, cluster -> testWithLoad(10, 300, false, cluster, LOG)); } From 2f29571296944e93c2f20b6c3891f9cef0f5dc60 Mon Sep 17 00:00:00 2001 From: Symious <14933944+symious@users.noreply.github.com> Date: Thu, 9 May 2024 01:24:50 +0800 Subject: [PATCH 037/397] RATIS-2078. Correct timeout unit for tests (#1083) --- .../apache/ratis/conf/TestRaftProperties.java | 2 +- .../ratis/grpc/TestRetryCacheWithGrpc.java | 2 +- .../ratis/server/impl/TestRaftServerJmx.java | 4 ++-- .../ratis/util/TestDataBlockingQueue.java | 8 ++++---- .../org/apache/ratis/util/TestDataQueue.java | 8 ++++---- .../org/apache/ratis/util/TestExitUtils.java | 4 ++-- .../org/apache/ratis/util/TestLifeCycle.java | 2 +- .../java/org/apache/ratis/util/TestMinMax.java | 2 +- .../apache/ratis/util/TestPeerProxyMap.java | 4 ++-- .../apache/ratis/util/TestPreconditions.java | 4 ++-- .../ratis/util/TestReferenceCountedObject.java | 4 ++-- .../ratis/util/TestResourceSemaphore.java | 2 +- .../apache/ratis/util/TestTimeDuration.java | 18 +++++++++--------- .../ratis/util/TestTimeoutScheduler.java | 10 +++++----- .../util/TestTraditionalBinaryPrefix.java | 4 ++-- 15 files changed, 39 insertions(+), 39 deletions(-) diff --git a/ratis-test/src/test/java/org/apache/ratis/conf/TestRaftProperties.java b/ratis-test/src/test/java/org/apache/ratis/conf/TestRaftProperties.java index 29bfc321df..d17d6fb6f6 100644 --- a/ratis-test/src/test/java/org/apache/ratis/conf/TestRaftProperties.java +++ b/ratis-test/src/test/java/org/apache/ratis/conf/TestRaftProperties.java @@ -41,7 +41,7 @@ static void setNonUnderscoreValue(RaftProperties p, String valueWithoutUnderscor } @Test - @Timeout(value = 1000) + @Timeout(value = 1) public void testUnderscore() { final RaftProperties p = new RaftProperties(); diff --git a/ratis-test/src/test/java/org/apache/ratis/grpc/TestRetryCacheWithGrpc.java b/ratis-test/src/test/java/org/apache/ratis/grpc/TestRetryCacheWithGrpc.java index 879b9eabdb..b244691437 100644 --- a/ratis-test/src/test/java/org/apache/ratis/grpc/TestRetryCacheWithGrpc.java +++ b/ratis-test/src/test/java/org/apache/ratis/grpc/TestRetryCacheWithGrpc.java @@ -172,7 +172,7 @@ void run() throws Exception { } @Test - @Timeout(value = 10000) + @Timeout(value = 10) public void testRetryOnResourceUnavailableException() throws InterruptedException, IOException { RaftProperties properties = new RaftProperties(); diff --git a/ratis-test/src/test/java/org/apache/ratis/server/impl/TestRaftServerJmx.java b/ratis-test/src/test/java/org/apache/ratis/server/impl/TestRaftServerJmx.java index 39a1f8855b..6302c7e8b1 100644 --- a/ratis-test/src/test/java/org/apache/ratis/server/impl/TestRaftServerJmx.java +++ b/ratis-test/src/test/java/org/apache/ratis/server/impl/TestRaftServerJmx.java @@ -40,7 +40,7 @@ public class TestRaftServerJmx extends BaseTest { @Test - @Timeout(value = 30000) + @Timeout(value = 30) public void testJmxBeans() throws Exception { final int NUM_SERVERS = 3; final MiniRaftClusterWithSimulatedRpc cluster @@ -60,7 +60,7 @@ public void testJmxBeans() throws Exception { } @Test - @Timeout(value = 30000) + @Timeout(value = 30) public void testRegister() throws JMException { { final JmxRegister jmx = new JmxRegister(); diff --git a/ratis-test/src/test/java/org/apache/ratis/util/TestDataBlockingQueue.java b/ratis-test/src/test/java/org/apache/ratis/util/TestDataBlockingQueue.java index e9dcf61d09..3ad40d4ca7 100644 --- a/ratis-test/src/test/java/org/apache/ratis/util/TestDataBlockingQueue.java +++ b/ratis-test/src/test/java/org/apache/ratis/util/TestDataBlockingQueue.java @@ -38,25 +38,25 @@ public class TestDataBlockingQueue { final TimeDuration fast = TimeDuration.valueOf(10, TimeUnit.MILLISECONDS); @Test - @Timeout(value = 1000) + @Timeout(value = 1) public void testElementLimit() { TestDataQueue.runTestElementLimit(q); } @Test - @Timeout(value = 1000) + @Timeout(value = 1) public void testByteLimit() { TestDataQueue.runTestByteLimit(q); } @Test - @Timeout(value = 1000) + @Timeout(value = 10) public void testSlowOfferFastPoll() throws Exception { runTestBlockingCalls(slow, fast, q); } @Test - @Timeout(value = 1000) + @Timeout(value = 10) public void testFastOfferSlowPoll() throws Exception { runTestBlockingCalls(fast, slow, q); } diff --git a/ratis-test/src/test/java/org/apache/ratis/util/TestDataQueue.java b/ratis-test/src/test/java/org/apache/ratis/util/TestDataQueue.java index 2dafa7e91c..42a55d90ff 100644 --- a/ratis-test/src/test/java/org/apache/ratis/util/TestDataQueue.java +++ b/ratis-test/src/test/java/org/apache/ratis/util/TestDataQueue.java @@ -47,7 +47,7 @@ static void assertSizes(long expectedNumElements, long expectedNumBytes, DataQue final DataQueue q = new DataQueue<>(null, byteLimit, elementLimit, Long::longValue); @Test - @Timeout(value = 1000) + @Timeout(value = 1) public void testElementLimit() { runTestElementLimit(q); } @@ -82,7 +82,7 @@ static void runTestElementLimit(DataQueue q) { } @Test - @Timeout(value = 1000) + @Timeout(value = 1) public void testByteLimit() { runTestByteLimit(q); } @@ -140,7 +140,7 @@ static void runTestByteLimit(DataQueue q) { } @Test - @Timeout(value = 1000) + @Timeout(value = 1) public void testIteratorAndRemove() { runTestIteratorAndRemove(q); } @@ -185,7 +185,7 @@ static void runTestIteratorAndRemove(DataQueue q) { } @Test - @Timeout(value = 1000) + @Timeout(value = 1) public void testTimeout() { assertSizes(0, 0, q); diff --git a/ratis-test/src/test/java/org/apache/ratis/util/TestExitUtils.java b/ratis-test/src/test/java/org/apache/ratis/util/TestExitUtils.java index 1dd5b270bb..3c7cc025c9 100644 --- a/ratis-test/src/test/java/org/apache/ratis/util/TestExitUtils.java +++ b/ratis-test/src/test/java/org/apache/ratis/util/TestExitUtils.java @@ -26,7 +26,7 @@ public class TestExitUtils extends BaseTest { /** Test if {@link BaseTest} can handle uncaught exception. */ @Test - @Timeout(value = 1000) + @Timeout(value = 1) public void testUncaughtException() throws Exception { Assertions.assertFalse(ExitUtils.isTerminated()); Assertions.assertFalse(ExitUtils.clear()); @@ -43,7 +43,7 @@ public void testUncaughtException() throws Exception { /** Test if {@link BaseTest} can handle ExitUtils.terminate(..). */ @Test - @Timeout(value = 1000) + @Timeout(value = 1) public void testExitStatus() { Assertions.assertFalse(ExitUtils.isTerminated()); Assertions.assertFalse(ExitUtils.clear()); diff --git a/ratis-test/src/test/java/org/apache/ratis/util/TestLifeCycle.java b/ratis-test/src/test/java/org/apache/ratis/util/TestLifeCycle.java index f817449995..f08ceb3461 100644 --- a/ratis-test/src/test/java/org/apache/ratis/util/TestLifeCycle.java +++ b/ratis-test/src/test/java/org/apache/ratis/util/TestLifeCycle.java @@ -35,7 +35,7 @@ public class TestLifeCycle { * while this test uses successors. */ @Test - @Timeout(value = 1000) + @Timeout(value = 1) public void testIsValid() { final Map> successors = new EnumMap<>(LifeCycle.State.class); diff --git a/ratis-test/src/test/java/org/apache/ratis/util/TestMinMax.java b/ratis-test/src/test/java/org/apache/ratis/util/TestMinMax.java index 050d9973aa..07a9882ad2 100644 --- a/ratis-test/src/test/java/org/apache/ratis/util/TestMinMax.java +++ b/ratis-test/src/test/java/org/apache/ratis/util/TestMinMax.java @@ -29,7 +29,7 @@ public class TestMinMax { @Test - @Timeout(value = 1000) + @Timeout(value = 1) public void testMinMax() { runTestMinMax(LongStream.empty()); runTestMinMax(LongStream.iterate(0, n -> n).limit(10)); diff --git a/ratis-test/src/test/java/org/apache/ratis/util/TestPeerProxyMap.java b/ratis-test/src/test/java/org/apache/ratis/util/TestPeerProxyMap.java index 118830bda8..a2255041f8 100644 --- a/ratis-test/src/test/java/org/apache/ratis/util/TestPeerProxyMap.java +++ b/ratis-test/src/test/java/org/apache/ratis/util/TestPeerProxyMap.java @@ -53,7 +53,7 @@ public String toString() { } @Test - @Timeout(value = 10_000) + @Timeout(value = 10) public void testCloseDeadLock() throws Exception { final PeerProxyMap map = new PeerProxyMap<>("test", DummyProxy::new); final RaftPeerId id = RaftPeerId.valueOf("s0"); @@ -118,7 +118,7 @@ public String toString() { } @Test - @Timeout(value = 1000) + @Timeout(value = 1) public void testStackTrace() { final RaftPeerId id = RaftPeerId.valueOf("s0"); final RaftPeer peer = RaftPeer.newBuilder().setId(id).build(); diff --git a/ratis-test/src/test/java/org/apache/ratis/util/TestPreconditions.java b/ratis-test/src/test/java/org/apache/ratis/util/TestPreconditions.java index e99c11fdde..73cf4c1676 100644 --- a/ratis-test/src/test/java/org/apache/ratis/util/TestPreconditions.java +++ b/ratis-test/src/test/java/org/apache/ratis/util/TestPreconditions.java @@ -30,7 +30,7 @@ public class TestPreconditions extends BaseTest { @Test - @Timeout(value = 1000) + @Timeout(value = 1) public void testAssertUnique() { final Set empty = Collections.emptySet(); Preconditions.assertUnique(empty); @@ -56,7 +56,7 @@ public void testAssertUnique() { } @Test - @Timeout(value = 1000) + @Timeout(value = 1) public void testAssertNull() { final Map map = new HashMap<>(); final String key = "abc1234"; diff --git a/ratis-test/src/test/java/org/apache/ratis/util/TestReferenceCountedObject.java b/ratis-test/src/test/java/org/apache/ratis/util/TestReferenceCountedObject.java index 502261ec23..2436310141 100644 --- a/ratis-test/src/test/java/org/apache/ratis/util/TestReferenceCountedObject.java +++ b/ratis-test/src/test/java/org/apache/ratis/util/TestReferenceCountedObject.java @@ -41,7 +41,7 @@ static void assertRelease(ReferenceCountedObject ref, } @Test - @Timeout(value = 1000) + @Timeout(value = 1) public void testWrap() { final String value = "testWrap"; final AtomicInteger retained = new AtomicInteger(); @@ -116,7 +116,7 @@ public void testWrap() { } @Test - @Timeout(value = 1000) + @Timeout(value = 1) public void testReleaseWithoutRetaining() { final ReferenceCountedObject ref = ReferenceCountedObject.wrap(""); diff --git a/ratis-test/src/test/java/org/apache/ratis/util/TestResourceSemaphore.java b/ratis-test/src/test/java/org/apache/ratis/util/TestResourceSemaphore.java index 0f3612dce4..72c811d877 100644 --- a/ratis-test/src/test/java/org/apache/ratis/util/TestResourceSemaphore.java +++ b/ratis-test/src/test/java/org/apache/ratis/util/TestResourceSemaphore.java @@ -29,7 +29,7 @@ public class TestResourceSemaphore extends BaseTest { @Test - @Timeout(value = 5000) + @Timeout(value = 5) public void testGroup() throws InterruptedException, TimeoutException { final int FAILED_IN_ELEMENT_LIMIT = 0; final int FAILED_IN_BYTE_SIZE_LIMIT = 1; diff --git a/ratis-test/src/test/java/org/apache/ratis/util/TestTimeDuration.java b/ratis-test/src/test/java/org/apache/ratis/util/TestTimeDuration.java index f2d58f48db..9ba5e57235 100644 --- a/ratis-test/src/test/java/org/apache/ratis/util/TestTimeDuration.java +++ b/ratis-test/src/test/java/org/apache/ratis/util/TestTimeDuration.java @@ -40,7 +40,7 @@ public class TestTimeDuration { } @Test - @Timeout(value = 1000) + @Timeout(value = 1) public void testAbbreviation() { Arrays.asList(TimeUnit.values()) .forEach(a -> assertNotNull(Abbreviation.valueOf(a.name()))); @@ -57,7 +57,7 @@ public void testAbbreviation() { } @Test - @Timeout(value = 1000) + @Timeout(value = 1) public void testParse() { assertEquals(1L, parse("1_000_000 ns", TimeUnit.MILLISECONDS)); assertEquals(10L, parse("10_000_000 nanos", TimeUnit.MILLISECONDS)); @@ -97,7 +97,7 @@ public void testParse() { } @Test - @Timeout(value = 1000) + @Timeout(value = 1) public void testRoundUp() { final long nanosPerSecond = 1_000_000_000L; final TimeDuration oneSecond = TimeDuration.valueOf(1, TimeUnit.SECONDS); @@ -113,7 +113,7 @@ public void testRoundUp() { } @Test - @Timeout(value = 1000) + @Timeout(value = 1) public void testTo() { final TimeDuration oneSecond = TimeDuration.valueOf(1, TimeUnit.SECONDS); assertTo(1000, "1000ms", oneSecond, TimeUnit.MILLISECONDS); @@ -137,7 +137,7 @@ static TimeDuration assertTo(long expected, String expectedString, TimeDuration } @Test - @Timeout(value = 1000) + @Timeout(value = 1) public void testAddAndSubtract() { final TimeDuration oneSecond = TimeDuration.valueOf(1, TimeUnit.SECONDS); final TimeDuration tenSecond = TimeDuration.valueOf(10, TimeUnit.SECONDS); @@ -191,7 +191,7 @@ public void testAddAndSubtract() { } @Test - @Timeout(value = 1000) + @Timeout(value = 1) public void testNegate() { assertNegate(0); assertNegate(1); @@ -213,7 +213,7 @@ private static void assertNegate(long n) { } @Test - @Timeout(value = 1000) + @Timeout(value = 1) public void testMultiply() { assertMultiply(0, TimeDuration.ONE_SECOND, TimeDuration.valueOf(0, TimeUnit.SECONDS)); assertMultiply(0.001, TimeDuration.ONE_SECOND, TimeDuration.ONE_MILLISECOND); @@ -244,7 +244,7 @@ private static void assertMultiply(TimeDuration t, double multiplier, TimeDurati } @Test - @Timeout(value = 1000) + @Timeout(value = 1) public void testHigherLower() { final TimeUnit[] units = {TimeUnit.NANOSECONDS, TimeUnit.MICROSECONDS, TimeUnit.MILLISECONDS, TimeUnit.SECONDS, TimeUnit.MINUTES, TimeUnit.HOURS, TimeUnit.DAYS}; @@ -257,7 +257,7 @@ public void testHigherLower() { } @Test - @Timeout(value = 1000) + @Timeout(value = 1) public void testCompareTo() { assertTimeDurationCompareTo(TimeDuration.ONE_MINUTE, TimeDuration.ONE_SECOND); diff --git a/ratis-test/src/test/java/org/apache/ratis/util/TestTimeoutScheduler.java b/ratis-test/src/test/java/org/apache/ratis/util/TestTimeoutScheduler.java index 79c97a6ede..e8e9d6eb0c 100644 --- a/ratis-test/src/test/java/org/apache/ratis/util/TestTimeoutScheduler.java +++ b/ratis-test/src/test/java/org/apache/ratis/util/TestTimeoutScheduler.java @@ -47,7 +47,7 @@ void assertNoError() { } @Test - @Timeout(value = 1000) + @Timeout(value = 1) public void testSingleTask() throws Exception { final TimeoutScheduler scheduler = TimeoutScheduler.newInstance(); final TimeDuration grace = TimeDuration.valueOf(100, TimeUnit.MILLISECONDS); @@ -84,7 +84,7 @@ public void testSingleTask() throws Exception { } @Test - @Timeout(value = 1000) + @Timeout(value = 1) public void testMultipleTasks() throws Exception { final TimeoutScheduler scheduler = TimeoutScheduler.newInstance(); final TimeDuration grace = TimeDuration.valueOf(100, TimeUnit.MILLISECONDS); @@ -131,7 +131,7 @@ public void testMultipleTasks() throws Exception { } @Test - @Timeout(value = 1000) + @Timeout(value = 1) public void testExtendingGracePeriod() throws Exception { final TimeoutScheduler scheduler = TimeoutScheduler.newInstance(); final TimeDuration grace = TimeDuration.valueOf(100, TimeUnit.MILLISECONDS); @@ -182,7 +182,7 @@ public void testExtendingGracePeriod() throws Exception { } @Test - @Timeout(value = 1000) + @Timeout(value = 1) public void testRestartingScheduler() throws Exception { final TimeoutScheduler scheduler = TimeoutScheduler.newInstance(); final TimeDuration grace = TimeDuration.valueOf(100, TimeUnit.MILLISECONDS); @@ -216,7 +216,7 @@ public void testRestartingScheduler() throws Exception { } @Test - @Timeout(value = 10_000) + @Timeout(value = 10) public void testShutdown() throws Exception { final TimeoutScheduler scheduler = TimeoutScheduler.newInstance(); Assertions.assertEquals(TimeoutScheduler.DEFAULT_GRACE_PERIOD, scheduler.getGracePeriod()); diff --git a/ratis-test/src/test/java/org/apache/ratis/util/TestTraditionalBinaryPrefix.java b/ratis-test/src/test/java/org/apache/ratis/util/TestTraditionalBinaryPrefix.java index 988cc5c710..a699b91c81 100644 --- a/ratis-test/src/test/java/org/apache/ratis/util/TestTraditionalBinaryPrefix.java +++ b/ratis-test/src/test/java/org/apache/ratis/util/TestTraditionalBinaryPrefix.java @@ -28,7 +28,7 @@ public class TestTraditionalBinaryPrefix { @Test - @Timeout(value = 1000) + @Timeout(value = 1) public void testTraditionalBinaryPrefix() { //test string2long(..) String[] symbol = {"k", "m", "g", "t", "p", "e"}; @@ -147,7 +147,7 @@ private static String byteDescription(long len) { } @Test - @Timeout(value = 1000) + @Timeout(value = 1) public void testUnderscore() { final SizeInBytes value = SizeInBytes.valueOf("1_000_000_000_000_000"); assertEquals(1_000_000_000_000_000L, value.getSize()); From 19877416ac751dca9c7be961e1bf169afdfeece4 Mon Sep 17 00:00:00 2001 From: "Doroszlai, Attila" <6454655+adoroszlai@users.noreply.github.com> Date: Fri, 10 May 2024 17:59:36 +0200 Subject: [PATCH 038/397] RATIS-2087. Remove workaround for GitHub reverse DNS issue (#1089) --- .github/workflows/post-commit.yml | 5 ----- 1 file changed, 5 deletions(-) diff --git a/.github/workflows/post-commit.yml b/.github/workflows/post-commit.yml index 13a9214445..9408743e85 100644 --- a/.github/workflows/post-commit.yml +++ b/.github/workflows/post-commit.yml @@ -143,11 +143,6 @@ jobs: - misc fail-fast: false steps: - # TEMPORARY WHILE GITHUB FIXES https://github.com/actions/virtual-environments/issues/3185 - - name: Add the current IP address, long hostname and short hostname record to /etc/hosts file - run: | - echo -e "$(ip addr show eth0 | grep "inet\b" | awk '{print $2}' | cut -d/ -f1)\t$(hostname -f) $(hostname -s)" | sudo tee -a /etc/hosts - # REMOVE CODE ABOVE WHEN ISSUE IS ADDRESSED! - name: Checkout project uses: actions/checkout@v4 - name: Cache for maven dependencies From 9806a31a9beb148194c576499ae8e6055b9f620a Mon Sep 17 00:00:00 2001 From: "Doroszlai, Attila" <6454655+adoroszlai@users.noreply.github.com> Date: Fri, 10 May 2024 18:00:38 +0200 Subject: [PATCH 039/397] RATIS-2086. Autolink Ozone issues in PRs (#1088) --- .asf.yaml | 4 +++- 1 file changed, 3 insertions(+), 1 deletion(-) diff --git a/.asf.yaml b/.asf.yaml index 16e358f62f..23fccc71e8 100644 --- a/.asf.yaml +++ b/.asf.yaml @@ -24,7 +24,9 @@ github: squash: true merge: false rebase: false - autolink_jira: RATIS + autolink_jira: + - HDDS + - RATIS notifications: commits: commits@ratis.apache.org From f887d74e367430a6dd9959a9f3435b6088049012 Mon Sep 17 00:00:00 2001 From: Tsz-Wo Nicholas Sze Date: Sat, 11 May 2024 09:53:17 -0700 Subject: [PATCH 040/397] RATIS-2076. Intermittent timeout in TestRaftWithNetty#testWithLoad. (#1090) --- .../main/java/org/apache/ratis/server/impl/LeaderElection.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/ratis-server/src/main/java/org/apache/ratis/server/impl/LeaderElection.java b/ratis-server/src/main/java/org/apache/ratis/server/impl/LeaderElection.java index d738c87578..b31fbaba62 100644 --- a/ratis-server/src/main/java/org/apache/ratis/server/impl/LeaderElection.java +++ b/ratis-server/src/main/java/org/apache/ratis/server/impl/LeaderElection.java @@ -143,7 +143,7 @@ static class Executor { } void shutdown() { - executor.shutdown(); + executor.shutdownNow(); } void submit(Callable task) { From 7be38ca65e90f4e4e2e8ca164ab66745a448be09 Mon Sep 17 00:00:00 2001 From: "Doroszlai, Attila" <6454655+adoroszlai@users.noreply.github.com> Date: Sun, 12 May 2024 20:51:44 +0200 Subject: [PATCH 041/397] RATIS-2085. Compile Ratis with JDK 17/21 in CI (#1087) --- .github/workflows/post-commit.yml | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/.github/workflows/post-commit.yml b/.github/workflows/post-commit.yml index 9408743e85..7c286b29b2 100644 --- a/.github/workflows/post-commit.yml +++ b/.github/workflows/post-commit.yml @@ -66,7 +66,7 @@ jobs: timeout-minutes: 30 strategy: matrix: - java: [ 11 ] + java: [ 11, 17, 21 ] fail-fast: false steps: - name: Download source tarball @@ -91,7 +91,7 @@ jobs: distribution: 'temurin' java-version: ${{ matrix.java }} - name: Run a full build - run: ./dev-support/checks/build.sh + run: ./dev-support/checks/build.sh -Djavac.version=${{ matrix.java }} rat: name: rat runs-on: ubuntu-20.04 From 0e794bf69c046e98a9e6071cdc7e9e549d27e968 Mon Sep 17 00:00:00 2001 From: Sammi Chen Date: Mon, 13 May 2024 23:20:20 +0800 Subject: [PATCH 042/397] RATIS-2083. Timer thread leak when log level is DEBUG (#1086) --- .../src/main/java/org/apache/ratis/util/SlidingWindow.java | 4 ---- 1 file changed, 4 deletions(-) diff --git a/ratis-common/src/main/java/org/apache/ratis/util/SlidingWindow.java b/ratis-common/src/main/java/org/apache/ratis/util/SlidingWindow.java index 732e3d890e..7e37d81322 100644 --- a/ratis-common/src/main/java/org/apache/ratis/util/SlidingWindow.java +++ b/ratis-common/src/main/java/org/apache/ratis/util/SlidingWindow.java @@ -28,7 +28,6 @@ import java.util.SortedMap; import java.util.TreeMap; import java.util.concurrent.ConcurrentSkipListMap; -import java.util.concurrent.TimeUnit; import java.util.function.Consumer; import java.util.function.LongFunction; @@ -70,9 +69,6 @@ class RequestMap, REPLY> implements Iterable Date: Tue, 14 May 2024 09:15:50 +0800 Subject: [PATCH 043/397] RATIS-2088. Fix name and imports related check in test-directory (#1092) --- .../test/java/org/apache/ratis/BaseTest.java | 4 +-- .../apache/ratis/util/TestRefCountingMap.java | 1 - .../ratis/examples/ParameterizedBaseTest.java | 8 +++--- .../arithmetic/TestArithmeticLogDump.java | 8 +++--- .../ratis/examples/common/TestSubCommand.java | 3 -- .../ratis/grpc/MiniRaftClusterWithGrpc.java | 4 +-- .../ratis/netty/MiniRaftClusterWithNetty.java | 4 +-- .../InstallSnapshotNotificationTests.java | 24 ++++++++-------- .../apache/ratis/OutputStreamBaseTest.java | 2 -- .../apache/ratis/RaftAsyncExceptionTests.java | 2 +- .../java/org/apache/ratis/RaftAsyncTests.java | 6 ++-- .../java/org/apache/ratis/RaftTestUtil.java | 6 ++-- .../org/apache/ratis/WatchRequestTests.java | 2 -- .../server/impl/GroupManagementBaseTest.java | 10 +++---- .../ratis/server/impl/MiniRaftCluster.java | 8 +++--- .../impl/RaftReconfigurationBaseTest.java | 12 ++++---- .../server/simulation/RaftServerReply.java | 1 - .../ratis/TestRaftServerNoLeaderTimeout.java | 8 +++--- .../TestRaftServerSlownessDetection.java | 9 +++--- .../grpc/TestLeaderElectionWithGrpc.java | 2 +- .../ratis/grpc/util/TestGrpcZeroCopy.java | 20 ++++++------- .../netty/TestLeaderElectionWithNetty.java | 2 +- .../ratis/netty/TestTlsConfWithNetty.java | 3 +- .../server/TestRaftServerConfigKeys.java | 8 +++--- .../ratis/server/impl/TestRaftServerJmx.java | 4 +-- .../raftlog/segmented/TestCacheEviction.java | 28 +++++++++---------- .../segmented/TestSegmentedRaftLog.java | 24 ++++++++-------- .../segmented/TestSegmentedRaftLogCache.java | 4 +-- .../server/storage/TestStorageImplUtils.java | 12 ++++---- .../cli/sh/GroupCommandIntegrationTest.java | 10 +++---- 30 files changed, 116 insertions(+), 123 deletions(-) diff --git a/ratis-common/src/test/java/org/apache/ratis/BaseTest.java b/ratis-common/src/test/java/org/apache/ratis/BaseTest.java index fb34d64f1e..e289784030 100644 --- a/ratis-common/src/test/java/org/apache/ratis/BaseTest.java +++ b/ratis-common/src/test/java/org/apache/ratis/BaseTest.java @@ -124,7 +124,7 @@ public int getGlobalTimeoutSeconds() { return 100; } - private static final Supplier rootTestDir = JavaUtils.memoize( + private static final Supplier ROOT_TEST_DIR = JavaUtils.memoize( () -> JavaUtils.callAsUnchecked(() -> { final File dir = new File(System.getProperty("test.build.data", "target/test/data"), Integer.toHexString(ThreadLocalRandom.current().nextInt())); @@ -138,7 +138,7 @@ public int getGlobalTimeoutSeconds() { public static File getRootTestDir() { - return rootTestDir.get(); + return ROOT_TEST_DIR.get(); } public File getClassTestDir() { diff --git a/ratis-common/src/test/java/org/apache/ratis/util/TestRefCountingMap.java b/ratis-common/src/test/java/org/apache/ratis/util/TestRefCountingMap.java index db6e9239ae..6c438e247f 100644 --- a/ratis-common/src/test/java/org/apache/ratis/util/TestRefCountingMap.java +++ b/ratis-common/src/test/java/org/apache/ratis/util/TestRefCountingMap.java @@ -26,7 +26,6 @@ import java.util.Set; import org.apache.ratis.thirdparty.com.google.common.collect.Lists; -import org.apache.ratis.util.RefCountingMap; import org.junit.jupiter.api.BeforeEach; import org.junit.jupiter.api.Test; diff --git a/ratis-examples/src/test/java/org/apache/ratis/examples/ParameterizedBaseTest.java b/ratis-examples/src/test/java/org/apache/ratis/examples/ParameterizedBaseTest.java index df2fce13ee..fc3c4c8be8 100644 --- a/ratis-examples/src/test/java/org/apache/ratis/examples/ParameterizedBaseTest.java +++ b/ratis-examples/src/test/java/org/apache/ratis/examples/ParameterizedBaseTest.java @@ -48,11 +48,11 @@ public static Collection data() { } /** For {@link ParameterizedTest} so that a cluster can be shared by multiple {@link Test} */ - private static final AtomicReference currentCluster = new AtomicReference<>(); + private static final AtomicReference CURRENT_CLUSTER = new AtomicReference<>(); - /** Set {@link #currentCluster} to the given cluster and start it if {@link #currentCluster} is changed. */ + /** Set {@link #CURRENT_CLUSTER} to the given cluster and start it if {@link #CURRENT_CLUSTER} is changed. */ public static void setAndStart(MiniRaftCluster cluster) throws InterruptedException, IOException { - final MiniRaftCluster previous = currentCluster.getAndSet(cluster); + final MiniRaftCluster previous = CURRENT_CLUSTER.getAndSet(cluster); if (previous != cluster) { if (previous != null) { previous.shutdown(); @@ -65,7 +65,7 @@ public static void setAndStart(MiniRaftCluster cluster) throws InterruptedExcept @AfterAll public static void shutdownCurrentCluster() { - final MiniRaftCluster cluster = currentCluster.getAndSet(null); + final MiniRaftCluster cluster = CURRENT_CLUSTER.getAndSet(null); if (cluster != null) { cluster.shutdown(); } diff --git a/ratis-examples/src/test/java/org/apache/ratis/examples/arithmetic/TestArithmeticLogDump.java b/ratis-examples/src/test/java/org/apache/ratis/examples/arithmetic/TestArithmeticLogDump.java index c39335308f..f17ebe2604 100644 --- a/ratis-examples/src/test/java/org/apache/ratis/examples/arithmetic/TestArithmeticLogDump.java +++ b/ratis-examples/src/test/java/org/apache/ratis/examples/arithmetic/TestArithmeticLogDump.java @@ -47,17 +47,17 @@ public class TestArithmeticLogDump extends BaseTest { public static final int NUM_SERVERS = 1; - protected static final RaftProperties properties = new RaftProperties(); + protected static final RaftProperties PROPERTIES = new RaftProperties(); private final MiniRaftClusterWithSimulatedRpc cluster = MiniRaftClusterWithSimulatedRpc .FACTORY.newCluster(NUM_SERVERS, getProperties()); public RaftProperties getProperties() { RaftServerConfigKeys.Rpc - .setSlownessTimeout(properties, TimeDuration.valueOf(1, TimeUnit.SECONDS)); - properties.setClass(MiniRaftCluster.STATEMACHINE_CLASS_KEY, + .setSlownessTimeout(PROPERTIES, TimeDuration.valueOf(1, TimeUnit.SECONDS)); + PROPERTIES.setClass(MiniRaftCluster.STATEMACHINE_CLASS_KEY, SimpleStateMachine4Testing.class, StateMachine.class); - return properties; + return PROPERTIES; } @BeforeEach diff --git a/ratis-examples/src/test/java/org/apache/ratis/examples/common/TestSubCommand.java b/ratis-examples/src/test/java/org/apache/ratis/examples/common/TestSubCommand.java index 5ef034809a..c47dbb82a1 100644 --- a/ratis-examples/src/test/java/org/apache/ratis/examples/common/TestSubCommand.java +++ b/ratis-examples/src/test/java/org/apache/ratis/examples/common/TestSubCommand.java @@ -21,12 +21,9 @@ import java.util.Collection; import java.util.Collections; -import org.junit.Test; import org.junit.jupiter.api.Assertions; import org.junit.jupiter.params.ParameterizedTest; import org.junit.jupiter.params.provider.MethodSource; -import org.junit.runner.RunWith; -import org.junit.runners.Parameterized; public class TestSubCommand { diff --git a/ratis-grpc/src/test/java/org/apache/ratis/grpc/MiniRaftClusterWithGrpc.java b/ratis-grpc/src/test/java/org/apache/ratis/grpc/MiniRaftClusterWithGrpc.java index 18c65c5be4..387574f858 100644 --- a/ratis-grpc/src/test/java/org/apache/ratis/grpc/MiniRaftClusterWithGrpc.java +++ b/ratis-grpc/src/test/java/org/apache/ratis/grpc/MiniRaftClusterWithGrpc.java @@ -52,7 +52,7 @@ default Factory getFactory() { } } - public static final DelayLocalExecutionInjection sendServerRequestInjection = + public static final DelayLocalExecutionInjection SEND_SERVER_REQUEST_INJECTION = new DelayLocalExecutionInjection(GrpcService.GRPC_SEND_SERVER_REQUEST); protected MiniRaftClusterWithGrpc(String[] ids, String[] listenerIds, RaftProperties properties, Parameters parameters) { @@ -72,7 +72,7 @@ protected Parameters setPropertiesAndInitParameters(RaftPeerId id, RaftGroup gro @Override protected void blockQueueAndSetDelay(String leaderId, int delayMs) throws InterruptedException { - RaftTestUtil.blockQueueAndSetDelay(getServers(), sendServerRequestInjection, + RaftTestUtil.blockQueueAndSetDelay(getServers(), SEND_SERVER_REQUEST_INJECTION, leaderId, delayMs, getTimeoutMax()); } } diff --git a/ratis-netty/src/test/java/org/apache/ratis/netty/MiniRaftClusterWithNetty.java b/ratis-netty/src/test/java/org/apache/ratis/netty/MiniRaftClusterWithNetty.java index 990b63d90d..b912e14927 100644 --- a/ratis-netty/src/test/java/org/apache/ratis/netty/MiniRaftClusterWithNetty.java +++ b/ratis-netty/src/test/java/org/apache/ratis/netty/MiniRaftClusterWithNetty.java @@ -48,7 +48,7 @@ default Factory getFactory() { } } - public static final DelayLocalExecutionInjection sendServerRequest + public static final DelayLocalExecutionInjection SEND_SERVER_REQUEST = new DelayLocalExecutionInjection(NettyRpcService.SEND_SERVER_REQUEST); protected MiniRaftClusterWithNetty(String[] ids, String[] listenerIds, RaftProperties properties) { @@ -64,7 +64,7 @@ protected Parameters setPropertiesAndInitParameters(RaftPeerId id, RaftGroup gro @Override protected void blockQueueAndSetDelay(String leaderId, int delayMs) throws InterruptedException { - RaftTestUtil.blockQueueAndSetDelay(getServers(), sendServerRequest, + RaftTestUtil.blockQueueAndSetDelay(getServers(), SEND_SERVER_REQUEST, leaderId, delayMs, getTimeoutMax()); } } diff --git a/ratis-server/src/test/java/org/apache/ratis/InstallSnapshotNotificationTests.java b/ratis-server/src/test/java/org/apache/ratis/InstallSnapshotNotificationTests.java index 72ddd06f26..6aaa8acc39 100644 --- a/ratis-server/src/test/java/org/apache/ratis/InstallSnapshotNotificationTests.java +++ b/ratis-server/src/test/java/org/apache/ratis/InstallSnapshotNotificationTests.java @@ -83,7 +83,7 @@ public abstract class InstallSnapshotNotificationTests leaderSnapshotInfoRef = new AtomicReference<>(); + private static final AtomicReference LEADER_SNAPSHOT_INFO_REF = new AtomicReference<>(); private static final AtomicInteger numSnapshotRequests = new AtomicInteger(); private static final AtomicInteger numNotifyInstallSnapshotFinished = new AtomicInteger(); @@ -102,7 +102,7 @@ public CompletableFuture notifyInstallSnapshotFromLeader( } numSnapshotRequests.incrementAndGet(); - final SingleFileSnapshotInfo leaderSnapshotInfo = (SingleFileSnapshotInfo) leaderSnapshotInfoRef.get(); + final SingleFileSnapshotInfo leaderSnapshotInfo = (SingleFileSnapshotInfo) LEADER_SNAPSHOT_INFO_REF.get(); LOG.info("{}: leaderSnapshotInfo = {}", getId(), leaderSnapshotInfo); if (leaderSnapshotInfo == null) { return super.notifyInstallSnapshotFromLeader(roleInfoProto, termIndex); @@ -137,7 +137,7 @@ public void notifySnapshotInstalled(RaftProtos.InstallSnapshotResult result, lon return; } numNotifyInstallSnapshotFinished.incrementAndGet(); - final SingleFileSnapshotInfo leaderSnapshotInfo = (SingleFileSnapshotInfo) leaderSnapshotInfoRef.get(); + final SingleFileSnapshotInfo leaderSnapshotInfo = (SingleFileSnapshotInfo) LEADER_SNAPSHOT_INFO_REF.get(); File leaderSnapshotFile = leaderSnapshotInfo.getFile().getPath().toFile(); synchronized (this) { try { @@ -184,7 +184,7 @@ public void testAddNewFollowersNoSnapshot() throws Exception { private void testAddNewFollowers(CLUSTER cluster, int numRequests) throws Exception { final boolean shouldInstallSnapshot = numRequests >= SNAPSHOT_TRIGGER_THRESHOLD; - leaderSnapshotInfoRef.set(null); + LEADER_SNAPSHOT_INFO_REF.set(null); final List logs; int i = 0; try { @@ -236,7 +236,7 @@ private void testAddNewFollowers(CLUSTER cluster, int numRequests) throws Except final SnapshotInfo leaderSnapshotInfo = cluster.getLeader().getStateMachine().getLatestSnapshot(); LOG.info("LeaderSnapshotInfo: {}", leaderSnapshotInfo.getTermIndex()); - final boolean set = leaderSnapshotInfoRef.compareAndSet(null, leaderSnapshotInfo); + final boolean set = LEADER_SNAPSHOT_INFO_REF.compareAndSet(null, leaderSnapshotInfo); Assert.assertTrue(set); // add two more peers @@ -271,7 +271,7 @@ public void testRestartFollower() throws Exception { } private void testRestartFollower(CLUSTER cluster) throws Exception { - leaderSnapshotInfoRef.set(null); + LEADER_SNAPSHOT_INFO_REF.set(null); int i = 0; final RaftServer.Division leader = RaftTestUtil.waitForLeader(cluster); final RaftPeerId leaderId = leader.getId(); @@ -321,7 +321,7 @@ public void testInstallSnapshotNotificationCount() throws Exception { private void testInstallSnapshotNotificationCount(CLUSTER cluster) throws Exception { - leaderSnapshotInfoRef.set(null); + LEADER_SNAPSHOT_INFO_REF.set(null); numSnapshotRequests.set(0); int i = 0; @@ -352,7 +352,7 @@ private void testInstallSnapshotNotificationCount(CLUSTER cluster) throws Except Assert.assertEquals(20, snapshotIndex); final SnapshotInfo leaderSnapshotInfo = cluster.getLeader().getStateMachine().getLatestSnapshot(); Assert.assertEquals(20, leaderSnapshotInfo.getIndex()); - final boolean set = leaderSnapshotInfoRef.compareAndSet(null, leaderSnapshotInfo); + final boolean set = LEADER_SNAPSHOT_INFO_REF.compareAndSet(null, leaderSnapshotInfo); Assert.assertTrue(set); // Wait for the snapshot to be done. @@ -425,7 +425,7 @@ public void testInstallSnapshotInstalledEvent() throws Exception{ } private void testInstallSnapshotInstalledEvent(CLUSTER cluster) throws Exception{ - leaderSnapshotInfoRef.set(null); + LEADER_SNAPSHOT_INFO_REF.set(null); numNotifyInstallSnapshotFinished.set(0); final List logs; int i = 0; @@ -475,7 +475,7 @@ private void testInstallSnapshotInstalledEvent(CLUSTER cluster) throws Exception final SnapshotInfo leaderSnapshotInfo = cluster.getLeader().getStateMachine().getLatestSnapshot(); LOG.info("LeaderSnapshotInfo: {}", leaderSnapshotInfo.getTermIndex()); - final boolean set = leaderSnapshotInfoRef.compareAndSet(null, leaderSnapshotInfo); + final boolean set = LEADER_SNAPSHOT_INFO_REF.compareAndSet(null, leaderSnapshotInfo); Assert.assertTrue(set); // add one new peer @@ -524,7 +524,7 @@ public void testInstallSnapshotDuringBootstrap() throws Exception { } private void testInstallSnapshotDuringBootstrap(CLUSTER cluster) throws Exception { - leaderSnapshotInfoRef.set(null); + LEADER_SNAPSHOT_INFO_REF.set(null); numSnapshotRequests.set(0); int i = 0; try { @@ -553,7 +553,7 @@ private void testInstallSnapshotDuringBootstrap(CLUSTER cluster) throws Exceptio RaftSnapshotBaseTest.assertLeaderContent(cluster); final SnapshotInfo leaderSnapshotInfo = cluster.getLeader().getStateMachine().getLatestSnapshot(); - final boolean set = leaderSnapshotInfoRef.compareAndSet(null, leaderSnapshotInfo); + final boolean set = LEADER_SNAPSHOT_INFO_REF.compareAndSet(null, leaderSnapshotInfo); Assert.assertTrue(set); // add two more peers diff --git a/ratis-server/src/test/java/org/apache/ratis/OutputStreamBaseTest.java b/ratis-server/src/test/java/org/apache/ratis/OutputStreamBaseTest.java index d86170d0b4..0c1a5164fc 100644 --- a/ratis-server/src/test/java/org/apache/ratis/OutputStreamBaseTest.java +++ b/ratis-server/src/test/java/org/apache/ratis/OutputStreamBaseTest.java @@ -22,13 +22,11 @@ import org.apache.ratis.proto.RaftProtos.LogEntryProto.LogEntryBodyCase; import org.apache.ratis.server.RaftServer; import org.apache.ratis.server.impl.MiniRaftCluster; -import org.apache.ratis.server.protocol.TermIndex; import org.apache.ratis.server.raftlog.LogEntryHeader; import org.apache.ratis.server.raftlog.RaftLog; import org.apache.ratis.util.SizeInBytes; import org.apache.ratis.util.StringUtils; import org.junit.Assert; -import org.junit.Ignore; import org.junit.Test; import java.io.IOException; diff --git a/ratis-server/src/test/java/org/apache/ratis/RaftAsyncExceptionTests.java b/ratis-server/src/test/java/org/apache/ratis/RaftAsyncExceptionTests.java index 1ad53a8381..33e0857338 100644 --- a/ratis-server/src/test/java/org/apache/ratis/RaftAsyncExceptionTests.java +++ b/ratis-server/src/test/java/org/apache/ratis/RaftAsyncExceptionTests.java @@ -104,7 +104,7 @@ private void runTestTimeoutException(CLUSTER cluster) throws Exception { final RaftClientReply reply = client.io().send(new SimpleMessage("m0")); Assert.assertTrue(reply.isSuccess()); - RaftClientConfigKeys.Rpc.setRequestTimeout(properties.get(), ONE_SECOND); + RaftClientConfigKeys.Rpc.setRequestTimeout(PROPERTIES.get(), ONE_SECOND); // Block StartTransaction StreamSupport.stream(cluster.getServers().spliterator(), false) .map(cluster::getDivision) diff --git a/ratis-server/src/test/java/org/apache/ratis/RaftAsyncTests.java b/ratis-server/src/test/java/org/apache/ratis/RaftAsyncTests.java index 71c5c5ef06..1ac704f595 100644 --- a/ratis-server/src/test/java/org/apache/ratis/RaftAsyncTests.java +++ b/ratis-server/src/test/java/org/apache/ratis/RaftAsyncTests.java @@ -77,7 +77,7 @@ public abstract class RaftAsyncTests extends Ba public static final int NUM_SERVERS = 3; - private static final DelayLocalExecutionInjection logSyncDelay = RaftServerTestUtil.getLogSyncDelay(); + private static final DelayLocalExecutionInjection LOG_SYNC_DELAY = RaftServerTestUtil.getLogSyncDelay(); { getProperties().setClass(MiniRaftCluster.STATEMACHINE_CLASS_KEY, @@ -430,7 +430,7 @@ void runTestCheckLeadershipFailure(CLUSTER cluster) throws Exception { cluster.getServerAliveStream() .filter(impl -> !impl.getInfo().isLeader()) .map(SimpleStateMachine4Testing::get) - .forEach(peer -> logSyncDelay.setDelayMs(peer.getId().toString(), 1000)); + .forEach(peer -> LOG_SYNC_DELAY.setDelayMs(peer.getId().toString(), 1000)); // trigger append entries request client.async().send(new SimpleMessage("abc")); @@ -446,7 +446,7 @@ void runTestCheckLeadershipFailure(CLUSTER cluster) throws Exception { } finally { // unblock append entries request - logSyncDelay.clear(); + LOG_SYNC_DELAY.clear(); } waitForLeader(cluster); diff --git a/ratis-server/src/test/java/org/apache/ratis/RaftTestUtil.java b/ratis-server/src/test/java/org/apache/ratis/RaftTestUtil.java index ec149f2f45..c11312d417 100644 --- a/ratis-server/src/test/java/org/apache/ratis/RaftTestUtil.java +++ b/ratis-server/src/test/java/org/apache/ratis/RaftTestUtil.java @@ -405,8 +405,8 @@ public ByteString getContent() { } class SimpleOperation { - private static final ClientId clientId = ClientId.randomId(); - private static final AtomicLong callId = new AtomicLong(); + private static final ClientId CLIENT_ID = ClientId.randomId(); + private static final AtomicLong CALL_ID = new AtomicLong(); private final String op; private final StateMachineLogEntryProto smLogEntryProto; @@ -416,7 +416,7 @@ public SimpleOperation(String op) { } public SimpleOperation(String op, boolean hasStateMachineData) { - this(clientId, callId.incrementAndGet(), op, hasStateMachineData); + this(CLIENT_ID, CALL_ID.incrementAndGet(), op, hasStateMachineData); } private SimpleOperation(ClientId clientId, long callId, String op, boolean hasStateMachineData) { diff --git a/ratis-server/src/test/java/org/apache/ratis/WatchRequestTests.java b/ratis-server/src/test/java/org/apache/ratis/WatchRequestTests.java index a9bdd1a3a9..6f381ca13e 100644 --- a/ratis-server/src/test/java/org/apache/ratis/WatchRequestTests.java +++ b/ratis-server/src/test/java/org/apache/ratis/WatchRequestTests.java @@ -49,14 +49,12 @@ import org.slf4j.event.Level; import java.util.ArrayList; -import java.util.Arrays; import java.util.Collection; import java.util.List; import java.util.concurrent.CompletableFuture; import java.util.concurrent.ExecutionException; import java.util.concurrent.ThreadLocalRandom; import java.util.concurrent.TimeUnit; -import java.util.stream.Collectors; import java.util.stream.Stream; import static org.junit.Assert.fail; diff --git a/ratis-server/src/test/java/org/apache/ratis/server/impl/GroupManagementBaseTest.java b/ratis-server/src/test/java/org/apache/ratis/server/impl/GroupManagementBaseTest.java index 311a2150d6..30f9f75797 100644 --- a/ratis-server/src/test/java/org/apache/ratis/server/impl/GroupManagementBaseTest.java +++ b/ratis-server/src/test/java/org/apache/ratis/server/impl/GroupManagementBaseTest.java @@ -63,20 +63,20 @@ public abstract class GroupManagementBaseTest extends BaseTest { Slf4jUtils.setLogLevel(RaftClient.LOG, Level.DEBUG); } - static final RaftProperties prop = new RaftProperties(); + static final RaftProperties PROP = new RaftProperties(); static { // avoid flaky behaviour in CI environment - RaftServerConfigKeys.Rpc.setTimeoutMin(prop, TimeDuration.valueOf(1500, TimeUnit.MILLISECONDS)); - RaftServerConfigKeys.Rpc.setTimeoutMax(prop, TimeDuration.valueOf(2000, TimeUnit.MILLISECONDS)); + RaftServerConfigKeys.Rpc.setTimeoutMin(PROP, TimeDuration.valueOf(1500, TimeUnit.MILLISECONDS)); + RaftServerConfigKeys.Rpc.setTimeoutMax(PROP, TimeDuration.valueOf(2000, TimeUnit.MILLISECONDS)); // it takes 5s+ to finish the blocking group add call - RaftClientConfigKeys.Rpc.setRequestTimeout(prop, TimeDuration.valueOf(12, TimeUnit.SECONDS)); + RaftClientConfigKeys.Rpc.setRequestTimeout(PROP, TimeDuration.valueOf(12, TimeUnit.SECONDS)); } public abstract MiniRaftCluster.Factory getClusterFactory(); public MiniRaftCluster getCluster(int peerNum) { - return getClusterFactory().newCluster(peerNum, prop); + return getClusterFactory().newCluster(peerNum, PROP); } @Test diff --git a/ratis-server/src/test/java/org/apache/ratis/server/impl/MiniRaftCluster.java b/ratis-server/src/test/java/org/apache/ratis/server/impl/MiniRaftCluster.java index f6dd6121cb..3be0f6720d 100644 --- a/ratis-server/src/test/java/org/apache/ratis/server/impl/MiniRaftCluster.java +++ b/ratis-server/src/test/java/org/apache/ratis/server/impl/MiniRaftCluster.java @@ -92,14 +92,14 @@ public abstract class MiniRaftCluster implements Closeable { TimeDuration.valueOf(100, TimeUnit.MILLISECONDS); static final AtomicInteger THREAD_COUNT = new AtomicInteger(0); - public static abstract class Factory { + public abstract static class Factory { public interface Get { - Supplier properties = JavaUtils.memoize(RaftProperties::new); + Supplier PROPERTIES = JavaUtils.memoize(RaftProperties::new); Factory getFactory(); default RaftProperties getProperties() { - return properties.get(); + return PROPERTIES.get(); } default RaftProperties setStateMachine(Class stateMachineClass) { @@ -198,7 +198,7 @@ public CLUSTER newCluster(int numServer, int numListeners, RaftProperties prop) } } - public static abstract class RpcBase extends MiniRaftCluster { + public abstract static class RpcBase extends MiniRaftCluster { public RpcBase(String[] ids, String[] listenerIds, RaftProperties properties, Parameters parameters) { super(ids, listenerIds, properties, parameters); } diff --git a/ratis-server/src/test/java/org/apache/ratis/server/impl/RaftReconfigurationBaseTest.java b/ratis-server/src/test/java/org/apache/ratis/server/impl/RaftReconfigurationBaseTest.java index 3b8e206deb..c8c7fd026f 100644 --- a/ratis-server/src/test/java/org/apache/ratis/server/impl/RaftReconfigurationBaseTest.java +++ b/ratis-server/src/test/java/org/apache/ratis/server/impl/RaftReconfigurationBaseTest.java @@ -70,8 +70,8 @@ public abstract class RaftReconfigurationBaseTest logSyncDelay.setDelayMs(peer.getId().toString(), 1000)); + .forEach(peer -> LOG_SYNC_DELAY.setDelayMs(peer.getId().toString(), 1000)); final CountDownLatch latch = new CountDownLatch(1); final RaftPeer[] peersInRequest2 = cluster.getPeers().toArray(new RaftPeer[0]); @@ -691,7 +691,7 @@ void runTestOverlappedSetConfRequests(CLUSTER cluster) throws Exception { Assert.assertTrue(confChanged.get()); Assert.assertTrue(caughtException.get()); } finally { - logSyncDelay.clear(); + LOG_SYNC_DELAY.clear(); } } @@ -792,7 +792,7 @@ public void testLeaderNotReadyException() throws Exception { try { // delay 1s for each logSync call cluster.getServers().forEach( - peer -> leaderPlaceHolderDelay.setDelayMs(peer.getId().toString(), 2000)); + peer -> LEADER_PLACE_HOLDER_DELAY.setDelayMs(peer.getId().toString(), 2000)); cluster.start(); AtomicBoolean caughtNotReady = new AtomicBoolean(false); @@ -833,7 +833,7 @@ public void testLeaderNotReadyException() throws Exception { Assert.assertTrue(success.get()); Assert.assertTrue(caughtNotReady.get()); } finally { - leaderPlaceHolderDelay.clear(); + LEADER_PLACE_HOLDER_DELAY.clear(); cluster.shutdown(); } } diff --git a/ratis-server/src/test/java/org/apache/ratis/server/simulation/RaftServerReply.java b/ratis-server/src/test/java/org/apache/ratis/server/simulation/RaftServerReply.java index 5d9d871f8c..8b7f88ed4e 100644 --- a/ratis-server/src/test/java/org/apache/ratis/server/simulation/RaftServerReply.java +++ b/ratis-server/src/test/java/org/apache/ratis/server/simulation/RaftServerReply.java @@ -17,7 +17,6 @@ */ package org.apache.ratis.server.simulation; -import org.apache.ratis.proto.RaftProtos; import org.apache.ratis.protocol.RaftGroupId; import org.apache.ratis.protocol.RaftRpcMessage; import org.apache.ratis.proto.RaftProtos.AppendEntriesReplyProto; diff --git a/ratis-test/src/test/java/org/apache/ratis/TestRaftServerNoLeaderTimeout.java b/ratis-test/src/test/java/org/apache/ratis/TestRaftServerNoLeaderTimeout.java index c10dc105a5..61bacc3cf9 100644 --- a/ratis-test/src/test/java/org/apache/ratis/TestRaftServerNoLeaderTimeout.java +++ b/ratis-test/src/test/java/org/apache/ratis/TestRaftServerNoLeaderTimeout.java @@ -48,16 +48,16 @@ public class TestRaftServerNoLeaderTimeout extends BaseTest { private static final int NUM_SERVERS = 3; - private static final RaftProperties properties = new RaftProperties(); + private static final RaftProperties PROPERTIES = new RaftProperties(); private final MiniRaftClusterWithSimulatedRpc cluster = MiniRaftClusterWithSimulatedRpc .FACTORY.newCluster(NUM_SERVERS, getProperties()); private static RaftProperties getProperties() { - RaftServerConfigKeys.Notification.setNoLeaderTimeout(properties, TimeDuration.valueOf(1, TimeUnit.SECONDS)); - properties.setClass(MiniRaftCluster.STATEMACHINE_CLASS_KEY, + RaftServerConfigKeys.Notification.setNoLeaderTimeout(PROPERTIES, TimeDuration.valueOf(1, TimeUnit.SECONDS)); + PROPERTIES.setClass(MiniRaftCluster.STATEMACHINE_CLASS_KEY, SimpleStateMachine4Testing.class, StateMachine.class); - return properties; + return PROPERTIES; } @BeforeEach diff --git a/ratis-test/src/test/java/org/apache/ratis/TestRaftServerSlownessDetection.java b/ratis-test/src/test/java/org/apache/ratis/TestRaftServerSlownessDetection.java index b150a1aa05..895acccfbf 100644 --- a/ratis-test/src/test/java/org/apache/ratis/TestRaftServerSlownessDetection.java +++ b/ratis-test/src/test/java/org/apache/ratis/TestRaftServerSlownessDetection.java @@ -49,6 +49,7 @@ */ //TODO: fix StateMachine.notifySlowness(..); see RATIS-370 @Disabled +@SuppressWarnings({"deprecation", "rawtypes"}) public class TestRaftServerSlownessDetection extends BaseTest { static { Slf4jUtils.setLogLevel(RaftServer.Division.LOG, Level.DEBUG); @@ -56,17 +57,17 @@ public class TestRaftServerSlownessDetection extends BaseTest { public static final int NUM_SERVERS = 3; - protected static final RaftProperties properties = new RaftProperties(); + protected static final RaftProperties PROPERTIES = new RaftProperties(); private final MiniRaftClusterWithSimulatedRpc cluster = MiniRaftClusterWithSimulatedRpc .FACTORY.newCluster(NUM_SERVERS, getProperties()); public RaftProperties getProperties() { RaftServerConfigKeys.Rpc - .setSlownessTimeout(properties, TimeDuration.valueOf(1, TimeUnit.SECONDS)); - properties.setClass(MiniRaftCluster.STATEMACHINE_CLASS_KEY, + .setSlownessTimeout(PROPERTIES, TimeDuration.valueOf(1, TimeUnit.SECONDS)); + PROPERTIES.setClass(MiniRaftCluster.STATEMACHINE_CLASS_KEY, SimpleStateMachine4Testing.class, StateMachine.class); - return properties; + return PROPERTIES; } @BeforeEach diff --git a/ratis-test/src/test/java/org/apache/ratis/grpc/TestLeaderElectionWithGrpc.java b/ratis-test/src/test/java/org/apache/ratis/grpc/TestLeaderElectionWithGrpc.java index ef6bc2a866..797124bc16 100644 --- a/ratis-test/src/test/java/org/apache/ratis/grpc/TestLeaderElectionWithGrpc.java +++ b/ratis-test/src/test/java/org/apache/ratis/grpc/TestLeaderElectionWithGrpc.java @@ -30,7 +30,7 @@ public class TestLeaderElectionWithGrpc public void testEnforceLeader() throws Exception { super.testEnforceLeader(); - MiniRaftClusterWithGrpc.sendServerRequestInjection.clear(); + MiniRaftClusterWithGrpc.SEND_SERVER_REQUEST_INJECTION.clear(); BlockRequestHandlingInjection.getInstance().unblockAll(); } diff --git a/ratis-test/src/test/java/org/apache/ratis/grpc/util/TestGrpcZeroCopy.java b/ratis-test/src/test/java/org/apache/ratis/grpc/util/TestGrpcZeroCopy.java index a5f61ee269..9ffe7ecd08 100644 --- a/ratis-test/src/test/java/org/apache/ratis/grpc/util/TestGrpcZeroCopy.java +++ b/ratis-test/src/test/java/org/apache/ratis/grpc/util/TestGrpcZeroCopy.java @@ -39,26 +39,26 @@ */ public final class TestGrpcZeroCopy extends BaseTest { static class RandomData { - private static final Random random = new Random(); - private static final byte[] array = new byte[4096]; + private static final Random RANDOM = new Random(); + private static final byte[] ARRAY = new byte[4096]; static void fill(long seed, int size, ByteBuf buf) { - random.setSeed(seed); + RANDOM.setSeed(seed); for(int offset = 0; offset < size; ) { - final int remaining = Math.min(size - offset, array.length); - random.nextBytes(array); - buf.writeBytes(array, 0, remaining); + final int remaining = Math.min(size - offset, ARRAY.length); + RANDOM.nextBytes(ARRAY); + buf.writeBytes(ARRAY, 0, remaining); offset += remaining; } } static void verify(long seed, ByteString b) { - random.setSeed(seed); + RANDOM.setSeed(seed); final int size = b.size(); for(int offset = 0; offset < size; ) { - final int remaining = Math.min(size - offset, array.length); - random.nextBytes(array); - final ByteString expected = UnsafeByteOperations.unsafeWrap(array, 0, remaining); + final int remaining = Math.min(size - offset, ARRAY.length); + RANDOM.nextBytes(ARRAY); + final ByteString expected = UnsafeByteOperations.unsafeWrap(ARRAY, 0, remaining); final ByteString computed = b.substring(offset, offset + remaining); Assertions.assertEquals(expected.size(), computed.size()); Assertions.assertEquals(expected, computed); diff --git a/ratis-test/src/test/java/org/apache/ratis/netty/TestLeaderElectionWithNetty.java b/ratis-test/src/test/java/org/apache/ratis/netty/TestLeaderElectionWithNetty.java index f3b760bf22..0b77b5beda 100644 --- a/ratis-test/src/test/java/org/apache/ratis/netty/TestLeaderElectionWithNetty.java +++ b/ratis-test/src/test/java/org/apache/ratis/netty/TestLeaderElectionWithNetty.java @@ -29,7 +29,7 @@ public class TestLeaderElectionWithNetty public void testEnforceLeader() throws Exception { super.testEnforceLeader(); - MiniRaftClusterWithNetty.sendServerRequest.clear(); + MiniRaftClusterWithNetty.SEND_SERVER_REQUEST.clear(); BlockRequestHandlingInjection.getInstance().unblockAll(); } diff --git a/ratis-test/src/test/java/org/apache/ratis/netty/TestTlsConfWithNetty.java b/ratis-test/src/test/java/org/apache/ratis/netty/TestTlsConfWithNetty.java index 77910b2489..abbc56934d 100644 --- a/ratis-test/src/test/java/org/apache/ratis/netty/TestTlsConfWithNetty.java +++ b/ratis-test/src/test/java/org/apache/ratis/netty/TestTlsConfWithNetty.java @@ -57,8 +57,9 @@ /** * Testing {@link TlsConf} and the security related utility methods in {@link NettyUtils}. */ +@SuppressWarnings({"try"}) public class TestTlsConfWithNetty { - private final static Logger LOG = LoggerFactory.getLogger(TestTlsConfWithNetty.class); + private static final Logger LOG = LoggerFactory.getLogger(TestTlsConfWithNetty.class); static String buffer2String(ByteBuf buf){ try { diff --git a/ratis-test/src/test/java/org/apache/ratis/server/TestRaftServerConfigKeys.java b/ratis-test/src/test/java/org/apache/ratis/server/TestRaftServerConfigKeys.java index 2026398e33..a9e509e7a8 100644 --- a/ratis-test/src/test/java/org/apache/ratis/server/TestRaftServerConfigKeys.java +++ b/ratis-test/src/test/java/org/apache/ratis/server/TestRaftServerConfigKeys.java @@ -46,14 +46,14 @@ */ public class TestRaftServerConfigKeys { - private static final Supplier rootTestDir = JavaUtils.memoize( + private static final Supplier ROOT_TEST_DIR = JavaUtils.memoize( () -> new File(BaseTest.getRootTestDir(), JavaUtils.getClassSimpleName(TestRaftServerConfigKeys.class) + Integer.toHexString(ThreadLocalRandom.current().nextInt()))); @AfterAll public static void tearDown() throws IOException { - FileUtils.deleteFully(rootTestDir.get()); + FileUtils.deleteFully(ROOT_TEST_DIR.get()); } /** @@ -63,7 +63,7 @@ public static void tearDown() throws IOException { @Test public void testStorageDirProperty() { final File testDir = new File( - rootTestDir.get(), UUID.randomUUID().toString()); + ROOT_TEST_DIR.get(), UUID.randomUUID().toString()); final List directories = new ArrayList<>(); final RaftProperties properties = new RaftProperties(); @@ -84,7 +84,7 @@ public void testStorageDirProperty() { @Test public void testStorageDir() { final File testDir = new File( - rootTestDir.get(), UUID.randomUUID().toString()); + ROOT_TEST_DIR.get(), UUID.randomUUID().toString()); final List directories = new ArrayList<>(); IntStream.range(0, 10).mapToObj((i) -> new File(testDir, Integer.toString(i))).forEach(directories::add); diff --git a/ratis-test/src/test/java/org/apache/ratis/server/impl/TestRaftServerJmx.java b/ratis-test/src/test/java/org/apache/ratis/server/impl/TestRaftServerJmx.java index 6302c7e8b1..d549e3781f 100644 --- a/ratis-test/src/test/java/org/apache/ratis/server/impl/TestRaftServerJmx.java +++ b/ratis-test/src/test/java/org/apache/ratis/server/impl/TestRaftServerJmx.java @@ -42,7 +42,7 @@ public class TestRaftServerJmx extends BaseTest { @Test @Timeout(value = 30) public void testJmxBeans() throws Exception { - final int NUM_SERVERS = 3; + final int numServers = 3; final MiniRaftClusterWithSimulatedRpc cluster = MiniRaftClusterWithSimulatedRpc.FACTORY.newCluster(3, new RaftProperties()); cluster.start(); @@ -50,7 +50,7 @@ public void testJmxBeans() throws Exception { MBeanServer platformMBeanServer = ManagementFactory.getPlatformMBeanServer(); Set objectInstances = platformMBeanServer.queryMBeans(new ObjectName("Ratis:*"), null); - Assertions.assertEquals(NUM_SERVERS, objectInstances.size()); + Assertions.assertEquals(numServers, objectInstances.size()); for (ObjectInstance instance : objectInstances) { Object groupId = platformMBeanServer.getAttribute(instance.getObjectName(), "GroupId"); diff --git a/ratis-test/src/test/java/org/apache/ratis/server/raftlog/segmented/TestCacheEviction.java b/ratis-test/src/test/java/org/apache/ratis/server/raftlog/segmented/TestCacheEviction.java index dacccf8a58..6ad429249b 100644 --- a/ratis-test/src/test/java/org/apache/ratis/server/raftlog/segmented/TestCacheEviction.java +++ b/ratis-test/src/test/java/org/apache/ratis/server/raftlog/segmented/TestCacheEviction.java @@ -52,7 +52,7 @@ import static org.apache.ratis.server.raftlog.segmented.SegmentedRaftLogTestUtils.MAX_OP_SIZE; public class TestCacheEviction extends BaseTest { - private static final CacheInvalidationPolicy policy = new CacheInvalidationPolicyDefault(); + private static final CacheInvalidationPolicy POLICY = new CacheInvalidationPolicyDefault(); static LogSegmentList prepareSegments(int numSegments, boolean[] cached, long start, long size) { Assertions.assertEquals(numSegments, cached.length); @@ -76,35 +76,35 @@ public void testBasicEviction() throws Exception { new boolean[]{true, true, true, true, true}, 0, 10); // case 1, make sure we do not evict cache for segments behind local flushed index - List evicted = policy.evict(null, 5, 15, segments, maxCached); + List evicted = POLICY.evict(null, 5, 15, segments, maxCached); Assertions.assertEquals(0, evicted.size()); // case 2, suppose the local flushed index is in the 3rd segment, then we // can evict the first two segment - evicted = policy.evict(null, 25, 30, segments, maxCached); + evicted = POLICY.evict(null, 25, 30, segments, maxCached); Assertions.assertEquals(2, evicted.size()); Assertions.assertSame(evicted.get(0), segments.get(0)); Assertions.assertSame(evicted.get(1), segments.get(1)); // case 3, similar with case 2, but the local applied index is less than // the local flushed index. - evicted = policy.evict(null, 25, 15, segments, maxCached); + evicted = POLICY.evict(null, 25, 15, segments, maxCached); Assertions.assertEquals(1, evicted.size()); Assertions.assertSame(evicted.get(0), segments.get(0)); // case 4, the local applied index is very small, then evict cache behind it // first and let the state machine load the segments later - evicted = policy.evict(null, 35, 5, segments, maxCached); + evicted = POLICY.evict(null, 35, 5, segments, maxCached); Assertions.assertEquals(1, evicted.size()); Assertions.assertSame(evicted.get(0), segments.get(2)); Mockito.when(segments.get(2).hasCache()).thenReturn(false); - evicted = policy.evict(null, 35, 5, segments, maxCached); + evicted = POLICY.evict(null, 35, 5, segments, maxCached); Assertions.assertEquals(1, evicted.size()); Assertions.assertSame(evicted.get(0), segments.get(1)); Mockito.when(segments.get(1).hasCache()).thenReturn(false); - evicted = policy.evict(null, 35, 5, segments, maxCached); + evicted = POLICY.evict(null, 35, 5, segments, maxCached); Assertions.assertEquals(0, evicted.size()); } @@ -116,38 +116,38 @@ public void testEvictionWithFollowerIndices() throws Exception { // case 1, no matter where the followers are, we do not evict segments behind local // flushed index - List evicted = policy.evict(new long[]{20, 40, 40}, 5, 15, segments, + List evicted = POLICY.evict(new long[]{20, 40, 40}, 5, 15, segments, maxCached); Assertions.assertEquals(0, evicted.size()); // case 2, the follower indices are behind the local flushed index - evicted = policy.evict(new long[]{30, 40, 45}, 25, 30, segments, maxCached); + evicted = POLICY.evict(new long[]{30, 40, 45}, 25, 30, segments, maxCached); Assertions.assertEquals(2, evicted.size()); Assertions.assertSame(evicted.get(0), segments.get(0)); Assertions.assertSame(evicted.get(1), segments.get(1)); // case 3, similar with case 3 in basic eviction test - evicted = policy.evict(new long[]{30, 40, 45}, 25, 15, segments, maxCached); + evicted = POLICY.evict(new long[]{30, 40, 45}, 25, 15, segments, maxCached); Assertions.assertEquals(1, evicted.size()); Assertions.assertSame(evicted.get(0), segments.get(0)); // case 4, the followers are slower than local flush - evicted = policy.evict(new long[]{15, 45, 45}, 55, 50, segments, maxCached); + evicted = POLICY.evict(new long[]{15, 45, 45}, 55, 50, segments, maxCached); Assertions.assertEquals(1, evicted.size()); Assertions.assertSame(evicted.get(0), segments.get(0)); Mockito.when(segments.get(0).hasCache()).thenReturn(false); - evicted = policy.evict(new long[]{15, 45, 45}, 55, 50, segments, maxCached); + evicted = POLICY.evict(new long[]{15, 45, 45}, 55, 50, segments, maxCached); Assertions.assertEquals(1, evicted.size()); Assertions.assertSame(evicted.get(0), segments.get(2)); Mockito.when(segments.get(2).hasCache()).thenReturn(false); - evicted = policy.evict(new long[]{15, 45, 45}, 55, 50, segments, maxCached); + evicted = POLICY.evict(new long[]{15, 45, 45}, 55, 50, segments, maxCached); Assertions.assertEquals(1, evicted.size()); Assertions.assertSame(evicted.get(0), segments.get(3)); Mockito.when(segments.get(3).hasCache()).thenReturn(false); - evicted = policy.evict(new long[]{15, 45, 45}, 55, 50, segments, maxCached); + evicted = POLICY.evict(new long[]{15, 45, 45}, 55, 50, segments, maxCached); Assertions.assertEquals(0, evicted.size()); } diff --git a/ratis-test/src/test/java/org/apache/ratis/server/raftlog/segmented/TestSegmentedRaftLog.java b/ratis-test/src/test/java/org/apache/ratis/server/raftlog/segmented/TestSegmentedRaftLog.java index 8d6c97d815..149f6286ac 100644 --- a/ratis-test/src/test/java/org/apache/ratis/server/raftlog/segmented/TestSegmentedRaftLog.java +++ b/ratis-test/src/test/java/org/apache/ratis/server/raftlog/segmented/TestSegmentedRaftLog.java @@ -94,9 +94,9 @@ public static long getOpenSegmentSize(RaftLog raftLog) { return ((SegmentedRaftLog)raftLog).getRaftLogCache().getOpenSegment().getTotalFileSize(); } - private static final RaftPeerId peerId = RaftPeerId.valueOf("s0"); - private static final RaftGroupId groupId = RaftGroupId.randomId(); - private static final RaftGroupMemberId memberId = RaftGroupMemberId.valueOf(peerId, groupId); + private static final RaftPeerId PEER_ID = RaftPeerId.valueOf("s0"); + private static final RaftGroupId GROUP_ID = RaftGroupId.randomId(); + private static final RaftGroupMemberId MEMBER_ID = RaftGroupMemberId.valueOf(PEER_ID, GROUP_ID); static class SegmentRange { final long start; @@ -133,7 +133,7 @@ SegmentedRaftLog newSegmentedRaftLog(LongSupplier getSnapshotIndexFromStateMachi static SegmentedRaftLog newSegmentedRaftLog(RaftStorage storage, RaftProperties properties) { return SegmentedRaftLog.newBuilder() - .setMemberId(memberId) + .setMemberId(MEMBER_ID) .setStorage(storage) .setProperties(properties) .build(); @@ -142,7 +142,7 @@ static SegmentedRaftLog newSegmentedRaftLog(RaftStorage storage, RaftProperties private SegmentedRaftLog newSegmentedRaftLogWithSnapshotIndex(RaftStorage storage, RaftProperties properties, LongSupplier getSnapshotIndexFromStateMachine) { return SegmentedRaftLog.newBuilder() - .setMemberId(memberId) + .setMemberId(MEMBER_ID) .setStorage(storage) .setSnapshotIndexSupplier(getSnapshotIndexFromStateMachine) .setProperties(properties) @@ -236,7 +236,7 @@ public void testLoadLogSegments(Boolean useAsyncFlush, Boolean smSyncFlush) thro Assertions.assertArrayEquals(entries, entriesFromLog); Assertions.assertEquals(entries[entries.length - 1], getLastEntry(raftLog)); - final RatisMetricRegistry metricRegistryForLogWorker = RaftLogMetricsBase.createRegistry(memberId); + final RatisMetricRegistry metricRegistryForLogWorker = RaftLogMetricsBase.createRegistry(MEMBER_ID); final DefaultTimekeeperImpl load = (DefaultTimekeeperImpl) metricRegistryForLogWorker.timer("segmentLoadLatency"); assertTrue(load.getTimer().getMeanRate() > 0); @@ -498,7 +498,7 @@ public void testPurgeLogMetric() throws Exception { int segmentSize = 200; long endIndexOfClosedSegment = segmentSize * (endTerm - startTerm - 1) - 1; long expectedIndex = segmentSize * (endTerm - startTerm - 1); - final RatisMetricRegistry metricRegistryForLogWorker = RaftLogMetricsBase.createRegistry(memberId); + final RatisMetricRegistry metricRegistryForLogWorker = RaftLogMetricsBase.createRegistry(MEMBER_ID); purgeAndVerify(startTerm, endTerm, segmentSize, 1, endIndexOfClosedSegment, expectedIndex); final DefaultTimekeeperImpl purge = (DefaultTimekeeperImpl) metricRegistryForLogWorker.timer("purgeLog"); assertTrue(purge.getTimer().getCount() > 0); @@ -543,7 +543,7 @@ public void testAppendEntriesWithInconsistency(Boolean useAsyncFlush, Boolean sm List entries = prepareLogEntries(ranges, null); final RetryCache retryCache = RetryCacheTestUtil.createRetryCache(); - try (SegmentedRaftLog raftLog = RetryCacheTestUtil.newSegmentedRaftLog(memberId, retryCache, storage, properties)) { + try (SegmentedRaftLog raftLog = RetryCacheTestUtil.newSegmentedRaftLog(MEMBER_ID, retryCache, storage, properties)) { raftLog.open(RaftLog.INVALID_LOG_INDEX, null); entries.forEach(entry -> RetryCacheTestUtil.createEntry(retryCache, entry)); // append entries to the raftlog @@ -558,7 +558,7 @@ public void testAppendEntriesWithInconsistency(Boolean useAsyncFlush, Boolean sm List newEntries = prepareLogEntries( Arrays.asList(r1, r2, r3), null); - try (SegmentedRaftLog raftLog = RetryCacheTestUtil.newSegmentedRaftLog(memberId, retryCache, storage, properties)) { + try (SegmentedRaftLog raftLog = RetryCacheTestUtil.newSegmentedRaftLog(MEMBER_ID, retryCache, storage, properties)) { raftLog.open(RaftLog.INVALID_LOG_INDEX, null); LOG.info("newEntries[0] = {}", newEntries.get(0)); final int last = newEntries.size() - 1; @@ -575,7 +575,7 @@ public void testAppendEntriesWithInconsistency(Boolean useAsyncFlush, Boolean sm } // load the raftlog again and check - try (SegmentedRaftLog raftLog = RetryCacheTestUtil.newSegmentedRaftLog(memberId, retryCache, storage, properties)) { + try (SegmentedRaftLog raftLog = RetryCacheTestUtil.newSegmentedRaftLog(MEMBER_ID, retryCache, storage, properties)) { raftLog.open(RaftLog.INVALID_LOG_INDEX, null); checkEntries(raftLog, entries, 0, 650); checkEntries(raftLog, newEntries, 100, 100); @@ -599,7 +599,7 @@ public void testSegmentedRaftLogStateMachineData(Boolean useAsyncFlush, Boolean final SimpleStateMachine4Testing sm = new SimpleStateMachine4Testing(); try (SegmentedRaftLog raftLog = SegmentedRaftLog.newBuilder() - .setMemberId(memberId) + .setMemberId(MEMBER_ID) .setStateMachine(sm) .setStorage(storage) .setProperties(properties) @@ -672,7 +672,7 @@ public void notifyLogFailed(Throwable cause, LogEntryProto entry) { ExecutionException ex; try (SegmentedRaftLog raftLog = SegmentedRaftLog.newBuilder() - .setMemberId(memberId) + .setMemberId(MEMBER_ID) .setStateMachine(sm) .setStorage(storage) .setProperties(properties) diff --git a/ratis-test/src/test/java/org/apache/ratis/server/raftlog/segmented/TestSegmentedRaftLogCache.java b/ratis-test/src/test/java/org/apache/ratis/server/raftlog/segmented/TestSegmentedRaftLogCache.java index fa892b2003..efcb90580f 100644 --- a/ratis-test/src/test/java/org/apache/ratis/server/raftlog/segmented/TestSegmentedRaftLogCache.java +++ b/ratis-test/src/test/java/org/apache/ratis/server/raftlog/segmented/TestSegmentedRaftLogCache.java @@ -43,7 +43,7 @@ import org.junit.jupiter.api.Test; public class TestSegmentedRaftLogCache { - private static final RaftProperties prop = new RaftProperties(); + private static final RaftProperties PROP = new RaftProperties(); private SegmentedRaftLogCache cache; private SegmentedRaftLogMetrics raftLogMetrics; @@ -53,7 +53,7 @@ public class TestSegmentedRaftLogCache { public void setup() { raftLogMetrics = new SegmentedRaftLogMetrics(RaftServerTestUtil.TEST_MEMBER_ID); ratisMetricRegistry = (RatisMetricRegistryImpl) raftLogMetrics.getRegistry(); - cache = new SegmentedRaftLogCache(null, null, prop, raftLogMetrics); + cache = new SegmentedRaftLogCache(null, null, PROP, raftLogMetrics); } @AfterEach diff --git a/ratis-test/src/test/java/org/apache/ratis/server/storage/TestStorageImplUtils.java b/ratis-test/src/test/java/org/apache/ratis/server/storage/TestStorageImplUtils.java index 4f75f793fc..905fc0345d 100644 --- a/ratis-test/src/test/java/org/apache/ratis/server/storage/TestStorageImplUtils.java +++ b/ratis-test/src/test/java/org/apache/ratis/server/storage/TestStorageImplUtils.java @@ -43,7 +43,7 @@ */ public class TestStorageImplUtils extends BaseTest { - private static final Supplier rootTestDir = JavaUtils.memoize( + private static final Supplier ROOT_TEST_DIR = JavaUtils.memoize( () -> new File(BaseTest.getRootTestDir(), JavaUtils.getClassSimpleName(TestStorageImplUtils.class) + Integer.toHexString(ThreadLocalRandom.current().nextInt()))); @@ -57,7 +57,7 @@ static File chooseNewStorageDir(List volumes, String sub) throws IOExcepti @AfterAll public static void tearDown() throws IOException { - FileUtils.deleteFully(rootTestDir.get()); + FileUtils.deleteFully(ROOT_TEST_DIR.get()); } /** @@ -67,7 +67,7 @@ public static void tearDown() throws IOException { */ @Test public void testChooseStorageDirWithOneVolume() throws IOException { - File testDir = new File(rootTestDir.get(), UUID.randomUUID().toString()); + File testDir = new File(ROOT_TEST_DIR.get(), UUID.randomUUID().toString()); List directories = Collections.singletonList(testDir); String subDirOne = UUID.randomUUID().toString(); String subDirTwo = UUID.randomUUID().toString(); @@ -88,7 +88,7 @@ public void testChooseStorageDirWithOneVolume() throws IOException { */ @Test public void testChooseStorageDirWithMultipleVolumes() throws IOException { - File testDir = new File(rootTestDir.get(), UUID.randomUUID().toString()); + File testDir = new File(ROOT_TEST_DIR.get(), UUID.randomUUID().toString()); List directories = new ArrayList<>(); IntStream.range(0, 10).mapToObj((i) -> new File(testDir, Integer.toString(i))).forEach((dir) -> { @@ -136,7 +136,7 @@ public void testChooseStorageDirWithNoVolume() { */ @Test public void testAutoFormatSingleDirectory() throws Exception { - final File testDir = new File(rootTestDir.get(), UUID.randomUUID().toString()); + final File testDir = new File(ROOT_TEST_DIR.get(), UUID.randomUUID().toString()); FileUtils.createDirectories(testDir); final RaftProperties properties = new RaftProperties(); @@ -153,7 +153,7 @@ public void testAutoFormatSingleDirectory() throws Exception { */ @Test public void testAutoFormatMultiDirectories() throws Exception { - final File testDir = new File(rootTestDir.get(), UUID.randomUUID().toString()); + final File testDir = new File(ROOT_TEST_DIR.get(), UUID.randomUUID().toString()); final List directories = new ArrayList<>(); IntStream.range(0, 3).mapToObj((i) -> new File(testDir, Integer.toString(i))).forEach((dir) -> { diff --git a/ratis-test/src/test/java/org/apache/ratis/shell/cli/sh/GroupCommandIntegrationTest.java b/ratis-test/src/test/java/org/apache/ratis/shell/cli/sh/GroupCommandIntegrationTest.java index 6900d4804a..c020075e73 100644 --- a/ratis-test/src/test/java/org/apache/ratis/shell/cli/sh/GroupCommandIntegrationTest.java +++ b/ratis-test/src/test/java/org/apache/ratis/shell/cli/sh/GroupCommandIntegrationTest.java @@ -124,18 +124,18 @@ void runTestGroupInfoCommandWithLogInfoVerification(MiniRaftCluster cluster) thr String info = result.substring(0, hearder.length()); Assertions.assertEquals(hearder, info); long currentTerm = leader.getInfo().getCurrentTerm(); - String LogInfoProtoFormat = "%s {" + NEW_LINE + " term: " + currentTerm + NEW_LINE + " index: %s"; + String logInfoProtoFormat = "%s {" + NEW_LINE + " term: " + currentTerm + NEW_LINE + " index: %s"; Assertions.assertTrue(result.contains( - String.format(LogInfoProtoFormat, "applied", + String.format(logInfoProtoFormat, "applied", leader.getStateMachine().getLastAppliedTermIndex().getIndex()))); Assertions.assertTrue(result.contains( - String.format(LogInfoProtoFormat, "committed", + String.format(logInfoProtoFormat, "committed", leader.getRaftLog().getLastCommittedIndex()))); Assertions.assertTrue(result.contains( - String.format(LogInfoProtoFormat, "lastSnapshot", + String.format(logInfoProtoFormat, "lastSnapshot", leader.getStateMachine().getLatestSnapshot().getIndex()))); Assertions.assertTrue(result.contains( - String.format(LogInfoProtoFormat, "lastEntry", + String.format(logInfoProtoFormat, "lastEntry", leader.getRaftLog().getLastCommittedIndex()))); } From 444810a5567cf3d09bc48e64ee99e5211921c5f4 Mon Sep 17 00:00:00 2001 From: "Doroszlai, Attila" <6454655+adoroszlai@users.noreply.github.com> Date: Wed, 15 May 2024 06:56:10 +0200 Subject: [PATCH 044/397] RATIS-2090. Bump ratis-thirdparty to 1.0.6 (#1093) --- pom.xml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/pom.xml b/pom.xml index 0130258b18..f8249e0ac7 100644 --- a/pom.xml +++ b/pom.xml @@ -208,7 +208,7 @@ 3.3.9 - 1.0.5 + 1.0.6 3.24.4 From 613533497b322cc49b12afc3b0f61cd3854b0622 Mon Sep 17 00:00:00 2001 From: Sammi Chen Date: Wed, 15 May 2024 22:33:27 +0800 Subject: [PATCH 045/397] RATIS-2066. RaftServerProxy supports close callback (#1072) --- .../apache/ratis/statemachine/StateMachine.java | 15 +++++++++++++-- .../apache/ratis/server/impl/LeaderElection.java | 2 +- .../apache/ratis/server/impl/RaftServerProxy.java | 1 + 3 files changed, 15 insertions(+), 3 deletions(-) diff --git a/ratis-server-api/src/main/java/org/apache/ratis/statemachine/StateMachine.java b/ratis-server-api/src/main/java/org/apache/ratis/statemachine/StateMachine.java index b1fc5addae..98d4537847 100644 --- a/ratis-server-api/src/main/java/org/apache/ratis/statemachine/StateMachine.java +++ b/ratis-server-api/src/main/java/org/apache/ratis/statemachine/StateMachine.java @@ -214,9 +214,20 @@ default void notifyLogFailed(Throwable cause, LogEntryProto failedEntry) {} default void notifySnapshotInstalled(InstallSnapshotResult result, long snapshotIndex, RaftPeer peer) {} /** - * Notify the {@link StateMachine} that a raft server has step down. + * Notify the {@link StateMachine} that the server for this division has been shut down. + * @Deprecated please use/override {@link #notifyServerShutdown(RoleInfoProto, boolean)} instead */ - default void notifyServerShutdown(RoleInfoProto roleInfo) {} + @Deprecated + default void notifyServerShutdown(RoleInfoProto roleInfo) { + notifyServerShutdown(roleInfo, false); + } + + /** + * Notify the {@link StateMachine} that either the server for this division or all the servers have been shut down. + * @param roleInfo roleInfo this server + * @param allServer whether all raft servers will be shutdown at this time + */ + default void notifyServerShutdown(RoleInfoProto roleInfo, boolean allServer) {} } /** diff --git a/ratis-server/src/main/java/org/apache/ratis/server/impl/LeaderElection.java b/ratis-server/src/main/java/org/apache/ratis/server/impl/LeaderElection.java index b31fbaba62..da9c513486 100644 --- a/ratis-server/src/main/java/org/apache/ratis/server/impl/LeaderElection.java +++ b/ratis-server/src/main/java/org/apache/ratis/server/impl/LeaderElection.java @@ -334,7 +334,7 @@ private boolean askForVotes(Phase phase, int round) throws InterruptedException, case NOT_IN_CONF: case SHUTDOWN: server.close(); - server.getStateMachine().event().notifyServerShutdown(server.getRoleInfoProto()); + server.getStateMachine().event().notifyServerShutdown(server.getRoleInfoProto(), false); return false; case TIMEOUT: return false; // should retry diff --git a/ratis-server/src/main/java/org/apache/ratis/server/impl/RaftServerProxy.java b/ratis-server/src/main/java/org/apache/ratis/server/impl/RaftServerProxy.java index cb7918e51d..9834d62ab4 100644 --- a/ratis-server/src/main/java/org/apache/ratis/server/impl/RaftServerProxy.java +++ b/ratis-server/src/main/java/org/apache/ratis/server/impl/RaftServerProxy.java @@ -138,6 +138,7 @@ private void close(RaftGroupId groupId, CompletableFuture future } catch (Throwable t) { LOG.warn("{}: Failed to close the division for {}", getId(), groupId, t); } + impl.getStateMachine().event().notifyServerShutdown(impl.getRoleInfoProto(), true); } synchronized List getGroupIds() { From 517b828569269ad39c5db3b9feec8534c81cd914 Mon Sep 17 00:00:00 2001 From: Symious <14933944+symious@users.noreply.github.com> Date: Thu, 16 May 2024 11:30:23 +0800 Subject: [PATCH 046/397] RATIS-2091. Fix other checkstyle issues in test-directory (#1094) --- .../examples/arithmetic/TestArithmetic.java | 3 +- .../examples/filestore/FileStoreWriter.java | 2 +- ...TestFileStoreStreamingWithGrpcCluster.java | 3 +- .../metrics/TestMetricRegistriesLoader.java | 2 +- .../ratis/InstallSnapshotFromLeaderTests.java | 9 +- .../java/org/apache/ratis/RaftBasicTests.java | 60 ++++++------- .../java/org/apache/ratis/RaftTestUtil.java | 3 +- .../org/apache/ratis/RetryCacheTests.java | 3 +- .../org/apache/ratis/WatchRequestTests.java | 69 +++++++-------- .../impl/BlockRequestHandlingInjection.java | 2 +- .../server/impl/LeaderElectionTests.java | 14 ++-- .../ratis/server/impl/MiniRaftCluster.java | 84 ++++++++++--------- .../ratis/server/impl/RaftServerTestUtil.java | 8 +- .../ratis/server/impl/RetryCacheTestUtil.java | 6 +- .../server/impl/TestLogAppenderMetrics.java | 8 +- .../MiniRaftClusterWithSimulatedRpc.java | 2 +- .../impl/SimpleStateMachine4Testing.java | 6 +- .../DataStreamAsyncClusterTests.java | 9 +- .../ratis/datastream/DataStreamBaseTest.java | 22 ++--- .../ratis/datastream/DataStreamTestUtils.java | 3 +- ...WithRpcTypeGrpcAndDataStreamTypeNetty.java | 9 +- ...ithRpcTypeNettyAndDataStreamTypeNetty.java | 5 +- ...WithRpcTypeGrpcAndDataStreamTypeNetty.java | 8 +- .../TestNettyDataStreamWithMock.java | 4 +- .../ratis/grpc/TestGrpcMessageMetrics.java | 6 +- .../ratis/grpc/TestRaftServerWithGrpc.java | 2 +- .../ratis/grpc/TestRetryCacheWithGrpc.java | 3 +- .../ratis/server/ServerRestartTests.java | 9 +- .../server/impl/TestRaftConfiguration.java | 4 +- .../ratis/server/impl/TestRaftServerJmx.java | 29 +++++-- .../server/raftlog/TestRaftLogMetrics.java | 15 ++-- .../segmented/TestBufferedWriteChannel.java | 14 ++-- .../raftlog/segmented/TestLogSegment.java | 3 +- .../segmented/TestSegmentedRaftLog.java | 18 ++-- .../cli/sh/LocalCommandIntegrationTest.java | 10 ++- .../org/apache/ratis/util/TestLifeCycle.java | 3 +- .../org/apache/ratis/util/TestMinMax.java | 3 +- 37 files changed, 262 insertions(+), 201 deletions(-) diff --git a/ratis-examples/src/test/java/org/apache/ratis/examples/arithmetic/TestArithmetic.java b/ratis-examples/src/test/java/org/apache/ratis/examples/arithmetic/TestArithmetic.java index 0c56898f0c..3d9d89a7db 100644 --- a/ratis-examples/src/test/java/org/apache/ratis/examples/arithmetic/TestArithmetic.java +++ b/ratis-examples/src/test/java/org/apache/ratis/examples/arithmetic/TestArithmetic.java @@ -107,7 +107,8 @@ void runGaussLegendre(RaftClient client) throws IOException { final Variable p0 = new Variable("p" + i_1); final Variable a1 = defineVariable(client, "a"+i, DIV.apply(ADD.apply(a0, b0), 2)); final Variable b1 = defineVariable(client, "b"+i, SQRT.apply(MULT.apply(a0, b0))); - final Variable t1 = defineVariable(client, "t"+i, SUBTRACT.apply(t0, MULT.apply(p0, SQUARE.apply(SUBTRACT.apply(a0, a1))))); + final Variable t1 = defineVariable(client, "t"+i, + SUBTRACT.apply(t0, MULT.apply(p0, SQUARE.apply(SUBTRACT.apply(a0, a1))))); final Variable p1 = defineVariable(client, "p"+i, MULT.apply(2, p0)); final Variable pi_i = new Variable("pi_"+i); diff --git a/ratis-examples/src/test/java/org/apache/ratis/examples/filestore/FileStoreWriter.java b/ratis-examples/src/test/java/org/apache/ratis/examples/filestore/FileStoreWriter.java index dbffab8a7d..c2b52fa633 100644 --- a/ratis-examples/src/test/java/org/apache/ratis/examples/filestore/FileStoreWriter.java +++ b/ratis-examples/src/test/java/org/apache/ratis/examples/filestore/FileStoreWriter.java @@ -44,7 +44,7 @@ import java.util.concurrent.Executor; import java.util.concurrent.atomic.AtomicInteger; -class FileStoreWriter implements Closeable { +final class FileStoreWriter implements Closeable { public static final Logger LOG = LoggerFactory.getLogger(FileStoreWriter.class); final long seed = ThreadLocalRandom.current().nextLong(); diff --git a/ratis-examples/src/test/java/org/apache/ratis/examples/filestore/TestFileStoreStreamingWithGrpcCluster.java b/ratis-examples/src/test/java/org/apache/ratis/examples/filestore/TestFileStoreStreamingWithGrpcCluster.java index a6f112760a..a5106444c3 100644 --- a/ratis-examples/src/test/java/org/apache/ratis/examples/filestore/TestFileStoreStreamingWithGrpcCluster.java +++ b/ratis-examples/src/test/java/org/apache/ratis/examples/filestore/TestFileStoreStreamingWithGrpcCluster.java @@ -19,7 +19,8 @@ import org.apache.ratis.datastream.MiniRaftClusterWithRpcTypeGrpcAndDataStreamTypeNetty; -public class TestFileStoreStreamingWithGrpcCluster extends FileStoreStreamingBaseTest +public class TestFileStoreStreamingWithGrpcCluster + extends FileStoreStreamingBaseTest implements MiniRaftClusterWithRpcTypeGrpcAndDataStreamTypeNetty.FactoryGet { } diff --git a/ratis-metrics-default/src/test/java/org/apache/ratis/metrics/TestMetricRegistriesLoader.java b/ratis-metrics-default/src/test/java/org/apache/ratis/metrics/TestMetricRegistriesLoader.java index c7f26740d6..cefc6b7947 100644 --- a/ratis-metrics-default/src/test/java/org/apache/ratis/metrics/TestMetricRegistriesLoader.java +++ b/ratis-metrics-default/src/test/java/org/apache/ratis/metrics/TestMetricRegistriesLoader.java @@ -67,7 +67,7 @@ public void testAddRemoveReporter() { Consumer reporter = v-> cntr.incrementAndGet(); Consumer stopReporter = v-> cntr.incrementAndGet(); r.addReporterRegistration(reporter, stopReporter); - + // check if add and remove of metric do reporting counter increase MetricRegistryInfo info = new MetricRegistryInfo("t1", "t1", "t1", "t1"); r.create(info); diff --git a/ratis-server/src/test/java/org/apache/ratis/InstallSnapshotFromLeaderTests.java b/ratis-server/src/test/java/org/apache/ratis/InstallSnapshotFromLeaderTests.java index 9d6a2f1834..46cfebbd17 100644 --- a/ratis-server/src/test/java/org/apache/ratis/InstallSnapshotFromLeaderTests.java +++ b/ratis-server/src/test/java/org/apache/ratis/InstallSnapshotFromLeaderTests.java @@ -134,7 +134,8 @@ private static class StateMachineWithMultiNestedSnapshotFile extends SimpleState File file2; @Override - public synchronized void initialize(RaftServer server, RaftGroupId groupId, RaftStorage raftStorage) throws IOException { + public synchronized void initialize(RaftServer server, RaftGroupId groupId, RaftStorage raftStorage) + throws IOException { super.initialize(server, groupId, raftStorage); // contains two snapshot files @@ -215,7 +216,8 @@ private static class StateMachineWithSeparatedSnapshotPath extends SimpleStateMa private File tmpDir; @Override - public synchronized void initialize(RaftServer server, RaftGroupId groupId, RaftStorage raftStorage) throws IOException { + public synchronized void initialize(RaftServer server, RaftGroupId groupId, RaftStorage raftStorage) + throws IOException { super.initialize(server, groupId, raftStorage); this.root = new File("/tmp/ratis-tests/statemachine/" + getId().toString()); this.snapshotDir = new File(root, "snapshot"); @@ -238,7 +240,8 @@ public synchronized void pause() { public long takeSnapshot() { final TermIndex lastApplied = getLastAppliedTermIndex(); final File snapshotTmpDir = new File(tmpDir, UUID.randomUUID().toString()); - final File snapshotRealDir = new File(snapshotDir, String.format("%d_%d", lastApplied.getTerm(), lastApplied.getIndex())); + final File snapshotRealDir = new File(snapshotDir, + String.format("%d_%d", lastApplied.getTerm(), lastApplied.getIndex())); try { FileUtils.deleteFully(snapshotRealDir); diff --git a/ratis-server/src/test/java/org/apache/ratis/RaftBasicTests.java b/ratis-server/src/test/java/org/apache/ratis/RaftBasicTests.java index 9f360cb51e..b16905d9a0 100644 --- a/ratis-server/src/test/java/org/apache/ratis/RaftBasicTests.java +++ b/ratis-server/src/test/java/org/apache/ratis/RaftBasicTests.java @@ -96,27 +96,27 @@ public void testBasicAppendEntriesKillLeader() throws Exception { } static CompletableFuture killAndRestartServer( - RaftPeerId id, long killSleepMs, long restartSleepMs, MiniRaftCluster cluster, Logger LOG) { + RaftPeerId id, long killSleepMs, long restartSleepMs, MiniRaftCluster cluster, Logger log) { final CompletableFuture future = new CompletableFuture<>(); new Thread(() -> { try { Thread.sleep(killSleepMs); cluster.killServer(id); Thread.sleep(restartSleepMs); - LOG.info("restart server: " + id); + log.info("restart server: " + id); cluster.restartServer(id, false); future.complete(null); } catch (Exception e) { - ExitUtils.terminate(-1, "Failed to kill/restart server: " + id, e, LOG); + ExitUtils.terminate(-1, "Failed to kill/restart server: " + id, e, log); } }).start(); return future; } static void runTestBasicAppendEntries( - boolean async, boolean killLeader, int numMessages, MiniRaftCluster cluster, Logger LOG) + boolean async, boolean killLeader, int numMessages, MiniRaftCluster cluster, Logger log) throws Exception { - LOG.info("runTestBasicAppendEntries: async? {}, killLeader={}, numMessages={}", + log.info("runTestBasicAppendEntries: async? {}, killLeader={}, numMessages={}", async, killLeader, numMessages); for (RaftServer s : cluster.getServers()) { cluster.restartServer(s.getId(), false); @@ -125,16 +125,16 @@ static void runTestBasicAppendEntries( final long term = leader.getInfo().getCurrentTerm(); final CompletableFuture killAndRestartFollower = killAndRestartServer( - cluster.getFollowers().get(0).getId(), 0, 1000, cluster, LOG); + cluster.getFollowers().get(0).getId(), 0, 1000, cluster, log); final CompletableFuture killAndRestartLeader; if (killLeader) { - LOG.info("killAndRestart leader " + leader.getId()); - killAndRestartLeader = killAndRestartServer(leader.getId(), 2000, 4000, cluster, LOG); + log.info("killAndRestart leader " + leader.getId()); + killAndRestartLeader = killAndRestartServer(leader.getId(), 2000, 4000, cluster, log); } else { killAndRestartLeader = CompletableFuture.completedFuture(null); } - LOG.info(cluster.printServers()); + log.info(cluster.printServers()); final SimpleMessage[] messages = SimpleMessage.create(numMessages); @@ -163,14 +163,14 @@ static void runTestBasicAppendEntries( } } Thread.sleep(cluster.getTimeoutMax().toIntExact(TimeUnit.MILLISECONDS) + 100); - LOG.info(cluster.printAllLogs()); + log.info(cluster.printAllLogs()); killAndRestartFollower.join(); killAndRestartLeader.join(); final List divisions = cluster.getServerAliveStream().collect(Collectors.toList()); for(RaftServer.Division impl: divisions) { - RaftTestUtil.assertLogEntries(impl, term, messages, 50, LOG); + RaftTestUtil.assertLogEntries(impl, term, messages, 50, log); } } @@ -271,16 +271,16 @@ static class Client4TestWithLoad extends Thread { final AtomicReference exceptionInClientThread = new AtomicReference<>(); final MiniRaftCluster cluster; - final Logger LOG; + final Logger log; Client4TestWithLoad(int index, int numMessages, boolean useAsync, - MiniRaftCluster cluster, Logger LOG) { + MiniRaftCluster cluster, Logger log) { super("client-" + index); this.index = index; this.messages = SimpleMessage.create(numMessages, index + "-"); this.useAsync = useAsync; this.cluster = cluster; - this.LOG = LOG; + this.log = log; } boolean isRunning() { @@ -317,10 +317,10 @@ public void run() { } } catch(Exception t) { if (exceptionInClientThread.compareAndSet(null, t)) { - LOG.error(this + " failed", t); + log.error(this + " failed", t); } else { exceptionInClientThread.get().addSuppressed(t); - LOG.error(this + " failed again!", t); + log.error(this + " failed again!", t); } } finally { isRunning.set(false); @@ -345,15 +345,15 @@ public void testWithLoad() throws Exception { } static void testWithLoad(final int numClients, final int numMessages, - boolean useAsync, MiniRaftCluster cluster, Logger LOG) throws Exception { - LOG.info("Running testWithLoad: numClients=" + numClients + boolean useAsync, MiniRaftCluster cluster, Logger log) throws Exception { + log.info("Running testWithLoad: numClients=" + numClients + ", numMessages=" + numMessages + ", async=" + useAsync); waitForLeader(cluster); final List clients = Stream.iterate(0, i -> i+1).limit(numClients) - .map(i -> new Client4TestWithLoad(i, numMessages, useAsync, cluster, LOG)) + .map(i -> new Client4TestWithLoad(i, numMessages, useAsync, cluster, log)) .collect(Collectors.toList()); final AtomicInteger lastStep = new AtomicInteger(); @@ -363,24 +363,24 @@ static void testWithLoad(final int numClients, final int numMessages, @Override public void run() { - LOG.info(cluster.printServers()); - LOG.info(BlockRequestHandlingInjection.getInstance().toString()); - LOG.info(cluster.toString()); - clients.forEach(c -> LOG.info(" " + c)); - JavaUtils.dumpAllThreads(s -> LOG.info(s)); + log.info(cluster.printServers()); + log.info(BlockRequestHandlingInjection.getInstance().toString()); + log.info(cluster.toString()); + clients.forEach(c -> log.info(" " + c)); + JavaUtils.dumpAllThreads(s -> log.info(s)); final int last = lastStep.get(); if (last != previousLastStep) { previousLastStep = last; } else { final RaftServer.Division leader = cluster.getLeader(); - LOG.info("NO PROGRESS at " + last + ", try to restart leader=" + leader); + log.info("NO PROGRESS at " + last + ", try to restart leader=" + leader); if (leader != null) { try { cluster.restartServer(leader.getId(), false); - LOG.info("Restarted leader=" + leader); + log.info("Restarted leader=" + leader); } catch (IOException e) { - LOG.error("Failed to restart leader=" + leader); + log.error("Failed to restart leader=" + leader); } } } @@ -411,10 +411,10 @@ public void run() { RaftTestUtil.changeLeader(cluster, leader.getId()); } } catch (IllegalStateException e) { - LOG.error("Failed to change leader ", e); + log.error("Failed to change leader ", e); } } - LOG.info("Leader change count=" + count); + log.info("Leader change count=" + count); timer.cancel(); for(Client4TestWithLoad c : clients) { @@ -425,7 +425,7 @@ public void run() { } } - public static void testRequestTimeout(boolean async, MiniRaftCluster cluster, Logger LOG) throws Exception { + public static void testRequestTimeout(boolean async, MiniRaftCluster cluster, Logger log) throws Exception { waitForLeader(cluster); final Timestamp startTime = Timestamp.currentTime(); try (final RaftClient client = cluster.createClient()) { diff --git a/ratis-server/src/test/java/org/apache/ratis/RaftTestUtil.java b/ratis-server/src/test/java/org/apache/ratis/RaftTestUtil.java index c11312d417..12a7537542 100644 --- a/ratis-server/src/test/java/org/apache/ratis/RaftTestUtil.java +++ b/ratis-server/src/test/java/org/apache/ratis/RaftTestUtil.java @@ -326,7 +326,8 @@ static List getStateMachineLogEntries(RaftLog log, Consumer print) { + static Void assertLogEntries(RaftLog log, long expectedTerm, SimpleMessage[] expectedMessages, + Consumer print) { final List entries = getStateMachineLogEntries(log, print); try { assertLogEntries(entries, expectedTerm, expectedMessages); diff --git a/ratis-server/src/test/java/org/apache/ratis/RetryCacheTests.java b/ratis-server/src/test/java/org/apache/ratis/RetryCacheTests.java index f81ac9556b..8929dfcc7f 100644 --- a/ratis-server/src/test/java/org/apache/ratis/RetryCacheTests.java +++ b/ratis-server/src/test/java/org/apache/ratis/RetryCacheTests.java @@ -89,7 +89,8 @@ public static void assertReply(RaftClientReply reply, RaftClient client, long ca Assertions.assertTrue(reply.isSuccess()); } - public void assertServer(MiniRaftCluster cluster, ClientId clientId, long callId, long oldLastApplied) throws Exception { + public void assertServer(MiniRaftCluster cluster, ClientId clientId, long callId, long oldLastApplied) + throws Exception { final long leaderApplied = cluster.getLeader().getInfo().getLastAppliedIndex(); // make sure retry cache has the entry for (RaftServer.Division server : cluster.iterateDivisions()) { diff --git a/ratis-server/src/test/java/org/apache/ratis/WatchRequestTests.java b/ratis-server/src/test/java/org/apache/ratis/WatchRequestTests.java index 6f381ca13e..32e4527580 100644 --- a/ratis-server/src/test/java/org/apache/ratis/WatchRequestTests.java +++ b/ratis-server/src/test/java/org/apache/ratis/WatchRequestTests.java @@ -141,26 +141,26 @@ public String toString() { } } - static void runTest(CheckedConsumer testCase, MiniRaftCluster cluster, Logger LOG) + static void runTest(CheckedConsumer testCase, MiniRaftCluster cluster, Logger log) throws Exception { try(final RaftClient client = cluster.createClient(RaftTestUtil.waitForLeader(cluster).getId())) { final int[] numMessages = {1, 10, 20}; for(int n : numMessages) { - final TestParameters p = new TestParameters(n, client, cluster, LOG); - LOG.info("{}) {}, {}", n, p, cluster.printServers()); + final TestParameters p = new TestParameters(n, client, cluster, log); + log.info("{}) {}, {}", n, p, cluster.printServers()); testCase.accept(p); } } } static void runSingleTest(CheckedConsumer testCase, - MiniRaftCluster cluster, Logger LOG) + MiniRaftCluster cluster, Logger log) throws Exception { try(final RaftClient client = cluster.createClient(RaftTestUtil.waitForLeader(cluster).getId())) { final int[] numMessages = {1}; for(int n : numMessages) { - final TestParameters p = new TestParameters(n, client, cluster, LOG); - LOG.info("{}) {}, {}", n, p, cluster.printServers()); + final TestParameters p = new TestParameters(n, client, cluster, log); + log.info("{}) {}, {}", n, p, cluster.printServers()); testCase.accept(p); } } @@ -176,7 +176,8 @@ static class WatchReplies { WatchReplies(long logIndex, CompletableFuture majority, CompletableFuture all, - CompletableFuture majorityCommitted, CompletableFuture allCommitted, Logger log) { + CompletableFuture majorityCommitted, CompletableFuture allCommitted, + Logger log) { this.logIndex = logIndex; this.majority = majority; this.all = all; @@ -218,19 +219,19 @@ RaftClientReply get(CompletableFuture f, String name) throws Ex } static void runTestWatchRequestAsync(TestParameters p) throws Exception { - final Logger LOG = p.log; + final Logger log = p.log; final MiniRaftCluster cluster = p.cluster; final int numMessages = p.numMessages; // blockStartTransaction of the leader so that no transaction can be committed MAJORITY final RaftServer.Division leader = cluster.getLeader(); - LOG.info("block leader {}", leader.getId()); + log.info("block leader {}", leader.getId()); SimpleStateMachine4Testing.get(leader).blockStartTransaction(); // blockFlushStateMachineData a follower so that no transaction can be ALL_COMMITTED final List followers = cluster.getFollowers(); final RaftServer.Division blockedFollower = followers.get(ThreadLocalRandom.current().nextInt(followers.size())); - LOG.info("block follower {}", blockedFollower.getId()); + log.info("block follower {}", blockedFollower.getId()); SimpleStateMachine4Testing.get(blockedFollower).blockFlushStateMachineData(); // send a message @@ -249,9 +250,9 @@ static void runTestWatchRequestAsync(TestParameters p) throws Exception { // unblock leader so that the transaction can be committed. SimpleStateMachine4Testing.get(leader).unblockStartTransaction(); - LOG.info("unblock leader {}", leader.getId()); + log.info("unblock leader {}", leader.getId()); - checkMajority(replies, watches, LOG); + checkMajority(replies, watches, log); Assert.assertEquals(numMessages, watches.size()); @@ -261,16 +262,16 @@ static void runTestWatchRequestAsync(TestParameters p) throws Exception { assertNotDone(watches.stream().map(CompletableFuture::join).map(w -> w.allCommitted)); // unblock follower so that the transaction can be replicated and committed to all. - LOG.info("unblock follower {}", blockedFollower.getId()); + log.info("unblock follower {}", blockedFollower.getId()); SimpleStateMachine4Testing.get(blockedFollower).unblockFlushStateMachineData(); - checkAll(watches, LOG); + checkAll(watches, log); } static void checkMajority(List> replies, - List> watches, Logger LOG) throws Exception { + List> watches, Logger log) throws Exception { for(int i = 0; i < replies.size(); i++) { final RaftClientReply reply = replies.get(i).get(GET_TIMEOUT_SECOND, TimeUnit.SECONDS); - LOG.info("checkMajority {}: receive {}", i, reply); + log.info("checkMajority {}: receive {}", i, reply); final long logIndex = reply.getLogIndex(); Assert.assertTrue(reply.isSuccess()); @@ -296,11 +297,11 @@ static void checkMajority(List> replies, } } - static void checkAll(List> watches, Logger LOG) throws Exception { + static void checkAll(List> watches, Logger log) throws Exception { for(int i = 0; i < watches.size(); i++) { final WatchReplies watchReplies = watches.get(i).get(GET_TIMEOUT_SECOND, TimeUnit.SECONDS); final long logIndex = watchReplies.logIndex; - LOG.info("checkAll {}: logIndex={}", i, logIndex); + log.info("checkAll {}: logIndex={}", i, logIndex); final RaftClientReply watchAllReply = watchReplies.getAll(); final RaftClientReply watchAllCommittedReply = watchReplies.getAllCommitted(); @@ -336,14 +337,14 @@ public void testWatchRequestAsyncChangeLeader() throws Exception { } static void runTestWatchRequestAsyncChangeLeader(TestParameters p) throws Exception { - final Logger LOG = p.log; + final Logger log = p.log; final MiniRaftCluster cluster = p.cluster; final int numMessages = p.numMessages; // blockFlushStateMachineData a follower so that no transaction can be ALL_COMMITTED final List followers = cluster.getFollowers(); final RaftServer.Division blockedFollower = followers.get(ThreadLocalRandom.current().nextInt(followers.size())); - LOG.info("block follower {}", blockedFollower.getId()); + log.info("block follower {}", blockedFollower.getId()); SimpleStateMachine4Testing.get(blockedFollower).blockFlushStateMachineData(); final List> replies = new ArrayList<>(); @@ -355,7 +356,7 @@ static void runTestWatchRequestAsyncChangeLeader(TestParameters p) throws Except Assert.assertEquals(numMessages, watches.size()); // since only one follower is blocked commit, requests can be committed MAJORITY and ALL but not ALL_COMMITTED. - checkMajority(replies, watches, LOG); + checkMajority(replies, watches, log); TimeUnit.SECONDS.sleep(1); assertNotDone(watches.stream().map(CompletableFuture::join).map(w -> w.allCommitted)); @@ -365,8 +366,8 @@ static void runTestWatchRequestAsyncChangeLeader(TestParameters p) throws Except // unblock follower so that the transaction can be replicated and committed to all. SimpleStateMachine4Testing.get(blockedFollower).unblockFlushStateMachineData(); - LOG.info("unblock follower {}", blockedFollower.getId()); - checkAll(watches, LOG); + log.info("unblock follower {}", blockedFollower.getId()); + checkAll(watches, log); } @Test @@ -384,7 +385,7 @@ public void testWatchRequestTimeout() throws Exception { } static void runTestWatchRequestTimeout(TestParameters p) throws Exception { - final Logger LOG = p.log; + final Logger log = p.log; final MiniRaftCluster cluster = p.cluster; final int numMessages = p.numMessages; @@ -394,13 +395,13 @@ static void runTestWatchRequestTimeout(TestParameters p) throws Exception { // blockStartTransaction of the leader so that no transaction can be committed MAJORITY final RaftServer.Division leader = cluster.getLeader(); - LOG.info("block leader {}", leader.getId()); + log.info("block leader {}", leader.getId()); SimpleStateMachine4Testing.get(leader).blockStartTransaction(); // blockFlushStateMachineData a follower so that no transaction can be ALL_COMMITTED final List followers = cluster.getFollowers(); final RaftServer.Division blockedFollower = followers.get(ThreadLocalRandom.current().nextInt(followers.size())); - LOG.info("block follower {}", blockedFollower.getId()); + log.info("block follower {}", blockedFollower.getId()); SimpleStateMachine4Testing.get(blockedFollower).blockFlushStateMachineData(); // send a message @@ -419,13 +420,13 @@ static void runTestWatchRequestTimeout(TestParameters p) throws Exception { // unblock leader so that the transaction can be committed. SimpleStateMachine4Testing.get(leader).unblockStartTransaction(); - LOG.info("unblock leader {}", leader.getId()); + log.info("unblock leader {}", leader.getId()); - checkMajority(replies, watches, LOG); - checkTimeout(replies, watches, LOG); + checkMajority(replies, watches, log); + checkTimeout(replies, watches, log); SimpleStateMachine4Testing.get(blockedFollower).unblockFlushStateMachineData(); - LOG.info("unblock follower {}", blockedFollower.getId()); + log.info("unblock follower {}", blockedFollower.getId()); } @Test @@ -446,7 +447,7 @@ public void testWatchRequestClientTimeout() throws Exception { } static void runTestWatchRequestClientTimeout(TestParameters p) throws Exception { - final Logger LOG = p.log; + final Logger log = p.log; CompletableFuture watchReply; // watch 1000 which will never be committed @@ -459,7 +460,7 @@ static void runTestWatchRequestClientTimeout(TestParameters p) throws Exception watchReply.get(); fail("runTestWatchRequestClientTimeout failed"); } catch (Exception ex) { - LOG.error("error occurred", ex); + log.error("error occurred", ex); Assert.assertTrue(ex.getCause().getClass() == AlreadyClosedException.class || ex.getCause().getClass() == RaftRetryFailureException.class); if (ex.getCause() != null) { @@ -525,10 +526,10 @@ static void runTestWatchMetrics(TestParameters p) throws Exception { } static void checkTimeout(List> replies, - List> watches, Logger LOG) throws Exception { + List> watches, Logger log) throws Exception { for(int i = 0; i < replies.size(); i++) { final RaftClientReply reply = replies.get(i).get(GET_TIMEOUT_SECOND, TimeUnit.SECONDS); - LOG.info("checkTimeout {}: receive {}", i, reply); + log.info("checkTimeout {}: receive {}", i, reply); final long logIndex = reply.getLogIndex(); Assert.assertTrue(reply.isSuccess()); diff --git a/ratis-server/src/test/java/org/apache/ratis/server/impl/BlockRequestHandlingInjection.java b/ratis-server/src/test/java/org/apache/ratis/server/impl/BlockRequestHandlingInjection.java index 3487aa472a..9af62bc430 100644 --- a/ratis-server/src/test/java/org/apache/ratis/server/impl/BlockRequestHandlingInjection.java +++ b/ratis-server/src/test/java/org/apache/ratis/server/impl/BlockRequestHandlingInjection.java @@ -25,7 +25,7 @@ import java.util.concurrent.ConcurrentHashMap; /** Inject code to block a server from handling incoming requests. */ -public class BlockRequestHandlingInjection implements CodeInjectionForTesting.Code { +public final class BlockRequestHandlingInjection implements CodeInjectionForTesting.Code { private static final BlockRequestHandlingInjection INSTANCE = new BlockRequestHandlingInjection(); diff --git a/ratis-server/src/test/java/org/apache/ratis/server/impl/LeaderElectionTests.java b/ratis-server/src/test/java/org/apache/ratis/server/impl/LeaderElectionTests.java index 391541e64a..e3af1546ec 100644 --- a/ratis-server/src/test/java/org/apache/ratis/server/impl/LeaderElectionTests.java +++ b/ratis-server/src/test/java/org/apache/ratis/server/impl/LeaderElectionTests.java @@ -312,15 +312,15 @@ public void testEnforceLeader() throws Exception { } } - static void enforceLeader(MiniRaftCluster cluster, final String newLeader, Logger LOG) throws InterruptedException { - LOG.info(cluster.printServers()); + static void enforceLeader(MiniRaftCluster cluster, final String newLeader, Logger log) throws InterruptedException { + log.info(cluster.printServers()); for(int i = 0; !cluster.tryEnforceLeader(newLeader) && i < 10; i++) { final RaftServer.Division currLeader = cluster.getLeader(); - LOG.info("try enforcing leader to " + newLeader + " but " + + log.info("try enforcing leader to " + newLeader + " but " + (currLeader == null ? "no leader for round " + i : "new leader is " + currLeader.getId())); TimeDuration.ONE_SECOND.sleep(); } - LOG.info(cluster.printServers()); + log.info(cluster.printServers()); final RaftServer.Division leader = cluster.getLeader(); Assertions.assertEquals(newLeader, leader.getId().toString()); @@ -516,7 +516,8 @@ public void testLeaderElectionMetrics() throws IOException, InterruptedException long numLeaderElectionTimeout = ratisMetricRegistry.counter(LEADER_ELECTION_TIMEOUT_COUNT_METRIC).getCount(); assertTrue(numLeaderElectionTimeout > 0); - final DefaultTimekeeperImpl timekeeper = (DefaultTimekeeperImpl) ratisMetricRegistry.timer(LEADER_ELECTION_TIME_TAKEN); + final DefaultTimekeeperImpl timekeeper = + (DefaultTimekeeperImpl) ratisMetricRegistry.timer(LEADER_ELECTION_TIME_TAKEN); final Timer timer = timekeeper.getTimer(); double meanTimeNs = timer.getSnapshot().getMean(); long elapsedNs = timestamp.elapsedTime().toLong(TimeUnit.NANOSECONDS); @@ -605,7 +606,8 @@ void runTestListenerRejectRequestVote(CLUSTER cluster) throws IOException, Inter final TermIndex lastEntry = leader.getRaftLog().getLastEntryTermIndex(); RaftServer.Division listener = cluster.getListeners().get(0); final RaftProtos.RequestVoteRequestProto r = ServerProtoUtils.toRequestVoteRequestProto( - leader.getMemberId(), listener.getId(), leader.getRaftLog().getLastEntryTermIndex().getTerm() + 1, lastEntry, true); + leader.getMemberId(), listener.getId(), + leader.getRaftLog().getLastEntryTermIndex().getTerm() + 1, lastEntry, true); RaftProtos.RequestVoteReplyProto listenerReply = listener.getRaftServer().requestVote(r); Assertions.assertFalse(listenerReply.getServerReply().getSuccess()); } diff --git a/ratis-server/src/test/java/org/apache/ratis/server/impl/MiniRaftCluster.java b/ratis-server/src/test/java/org/apache/ratis/server/impl/MiniRaftCluster.java index 3be0f6720d..1dc9e8f526 100644 --- a/ratis-server/src/test/java/org/apache/ratis/server/impl/MiniRaftCluster.java +++ b/ratis-server/src/test/java/org/apache/ratis/server/impl/MiniRaftCluster.java @@ -120,16 +120,18 @@ default void runWithNewCluster(int numServers, CheckedConsumer testCase) throws Exception { + default void runWithNewCluster(int numServers, boolean startCluster, CheckedConsumer testCase) + throws Exception { runWithNewCluster(numServers, 0, startCluster, testCase); } - default void runWithNewCluster(int numServers, int numListeners, CheckedConsumer testCase) throws Exception { + default void runWithNewCluster(int numServers, int numListeners, CheckedConsumer testCase) + throws Exception { runWithNewCluster(numServers, numListeners, true, testCase); } - default void runWithNewCluster(int numServers, int numListeners, boolean startCluster, CheckedConsumer testCase) - throws Exception { + default void runWithNewCluster(int numServers, int numListeners, boolean startCluster, + CheckedConsumer testCase) throws Exception { final StackTraceElement caller = JavaUtils.getCallerStackTraceElement(); LOG.info("Running " + caller.getMethodName()); final CLUSTER cluster = newCluster(numServers, numListeners); @@ -151,7 +153,8 @@ default void runWithSameCluster(int numServers, CheckedConsumer testCase) throws Exception { + default void runWithSameCluster(int numServers, int numListeners, CheckedConsumer testCase) + throws Exception { final StackTraceElement caller = JavaUtils.getCallerStackTraceElement(); LOG.info("Running " + caller.getMethodName()); CLUSTER cluster = null; @@ -321,15 +324,15 @@ public MiniRaftCluster initServers() { return this; } - public RaftServerProxy putNewServer(RaftPeerId id, RaftGroup group, boolean format) { - final RaftServerProxy s = newRaftServer(id, group, format); + public RaftServerProxy putNewServer(RaftPeerId id, RaftGroup raftGroup, boolean format) { + final RaftServerProxy s = newRaftServer(id, raftGroup, format); peers.put(s.getId(), s.getPeer()); Preconditions.assertTrue(servers.put(id, s) == null); return s; } - private Collection putNewServers(Iterable peers, boolean format, RaftGroup raftGroup) { - return StreamSupport.stream(peers.spliterator(), false) + private Collection putNewServers(Iterable peerIds, boolean format, RaftGroup raftGroup) { + return StreamSupport.stream(peerIds.spliterator(), false) .map(id -> putNewServer(id, raftGroup, format)) .collect(Collectors.toList()); } @@ -355,13 +358,14 @@ public RaftServer.Division restartServer(RaftPeerId serverId, boolean format) th return restartServer(serverId, group, format); } - public RaftServer.Division restartServer(RaftPeerId serverId, RaftGroup group, boolean format) throws IOException { + public RaftServer.Division restartServer(RaftPeerId serverId, RaftGroup raftGroup, boolean format) + throws IOException { killServer(serverId); servers.remove(serverId); - final RaftServer proxy = putNewServer(serverId, group, format); + final RaftServer proxy = putNewServer(serverId, raftGroup, format); proxy.start(); - return group == null? null: proxy.getDivision(group.getGroupId()); + return raftGroup == null? null: proxy.getDivision(raftGroup.getGroupId()); } public void restart(boolean format) throws IOException { @@ -377,8 +381,8 @@ public TimeDuration getTimeoutMax() { return RaftServerConfigKeys.Rpc.timeoutMax(properties); } - private RaftServerProxy newRaftServer(RaftPeerId id, RaftGroup group, boolean format) { - LOG.info("newRaftServer: {}, {}, format? {}", id, group, format); + private RaftServerProxy newRaftServer(RaftPeerId id, RaftGroup raftGroup, boolean format) { + LOG.info("newRaftServer: {}, {}, format? {}", id, raftGroup, format); try { final File dir = getStorageDir(id); if (format) { @@ -387,27 +391,27 @@ private RaftServerProxy newRaftServer(RaftPeerId id, RaftGroup group, boolean fo } final RaftProperties prop = new RaftProperties(properties); RaftServerConfigKeys.setStorageDir(prop, Collections.singletonList(dir)); - return ServerImplUtils.newRaftServer(id, group, + return ServerImplUtils.newRaftServer(id, raftGroup, format? RaftStorage.StartupOption.FORMAT: RaftStorage.StartupOption.RECOVER, - getStateMachineRegistry(prop), null, prop, setPropertiesAndInitParameters(id, group, prop)); + getStateMachineRegistry(prop), null, prop, setPropertiesAndInitParameters(id, raftGroup, prop)); } catch (IOException e) { throw new RuntimeException(e); } } protected abstract Parameters setPropertiesAndInitParameters( - RaftPeerId id, RaftGroup group, RaftProperties properties); + RaftPeerId id, RaftGroup raftGroup, RaftProperties raftProperties); public void setStateMachineRegistry(StateMachine.Registry stateMachineRegistry) { this.stateMachineRegistry = stateMachineRegistry; } - StateMachine.Registry getStateMachineRegistry(RaftProperties properties) { + StateMachine.Registry getStateMachineRegistry(RaftProperties raftProperties) { if (stateMachineRegistry != null) { return stateMachineRegistry; } - final Class smClass = properties.getClass( + final Class smClass = raftProperties.getClass( STATEMACHINE_CLASS_KEY, null, StateMachine.class); if (smClass == null) { return STATEMACHINE_REGISTRY_DEFAULT; @@ -423,7 +427,7 @@ StateMachine.Registry getStateMachineRegistry(RaftProperties properties) { try { final Class[] argClasses = {RaftProperties.class}; - return ReflectionUtils.newInstance(smClass, argClasses, properties); + return ReflectionUtils.newInstance(smClass, argClasses, raftProperties); } catch(RuntimeException e) { exception.addSuppressed(e); } @@ -495,8 +499,8 @@ public PeerChanges addNewPeers(String[] ids, boolean startNewPeer, return new PeerChanges(p, np, RaftPeer.emptyArray()); } - void startServers(Iterable servers) throws IOException { - for(RaftServer s : servers) { + void startServers(Iterable raftServers) throws IOException { + for(RaftServer s : raftServers) { s.start(); peers.put(s.getId(), s.getPeer()); } @@ -507,12 +511,12 @@ void startServers(Iterable servers) throws IOException { */ public PeerChanges removePeers(int number, boolean removeLeader, Collection excluded) throws InterruptedException { - Collection peers = new ArrayList<>(group.getPeers()); + Collection raftPeers = new ArrayList<>(group.getPeers()); List removedPeers = new ArrayList<>(number); if (removeLeader) { final RaftPeer leader = RaftTestUtil.waitForLeader(this).getPeer(); Preconditions.assertTrue(!excluded.contains(leader)); - peers.remove(leader); + raftPeers.remove(leader); removedPeers.add(leader); } final List followers = getFollowers(); @@ -520,12 +524,12 @@ public PeerChanges removePeers(int number, boolean removeLeader, removed < (removeLeader ? number - 1 : number); i++) { RaftPeer toRemove = followers.get(i).getPeer(); if (!excluded.contains(toRemove)) { - peers.remove(toRemove); + raftPeers.remove(toRemove); removedPeers.add(toRemove); removed++; } } - final RaftPeer[] p = peers.toArray(RaftPeer.emptyArray()); + final RaftPeer[] p = raftPeers.toArray(RaftPeer.emptyArray()); group = RaftGroup.valueOf(group.getGroupId(), p); return new PeerChanges(p, RaftPeer.emptyArray(), removedPeers.toArray(RaftPeer.emptyArray())); } @@ -727,13 +731,13 @@ public List getPeers() { return toRaftPeers(getServers()); } - RaftPeer getPeer(RaftPeerId id, RaftGroup group) { + RaftPeer getPeer(RaftPeerId id, RaftGroup raftGroup) { RaftPeer p = peers.get(id); if (p != null) { return p; } - if (group != null) { - p = group.getPeer(id); + if (raftGroup != null) { + p = raftGroup.getPeer(id); } if (p == null) { p = Optional.ofNullable(servers.get(id)).map(RaftServerProxy::getPeer).orElse(null); @@ -768,21 +772,21 @@ public RaftClient createClient(RaftPeerId leaderId, RetryPolicy retryPolicy) { return createClient(leaderId, group, retryPolicy); } - public RaftClient createClient(RaftPeerId leaderId, RaftGroup group) { - return createClient(leaderId, group, getDefaultRetryPolicy()); + public RaftClient createClient(RaftPeerId leaderId, RaftGroup raftGroup) { + return createClient(leaderId, raftGroup, getDefaultRetryPolicy()); } public RaftClient createClient(RaftPeer primaryServer) { return createClient(null, group, getDefaultRetryPolicy(), primaryServer); } - public RaftClient createClient(RaftPeerId leaderId, RaftGroup group, RetryPolicy retryPolicy) { - return createClient(leaderId, group, retryPolicy, null); + public RaftClient createClient(RaftPeerId leaderId, RaftGroup raftGroup, RetryPolicy retryPolicy) { + return createClient(leaderId, raftGroup, retryPolicy, null); } - public RaftClient createClient(RaftPeerId leaderId, RaftGroup group, RetryPolicy retryPolicy, RaftPeer primaryServer) { + public RaftClient createClient(RaftPeerId leaderId, RaftGroup raftGroup, RetryPolicy retryPolicy, RaftPeer primaryServer) { RaftClient.Builder builder = RaftClient.newBuilder() - .setRaftGroup(group) + .setRaftGroup(raftGroup) .setLeaderId(leaderId) .setProperties(properties) .setParameters(parameters) @@ -810,15 +814,15 @@ public RaftClientRequest newRaftClientRequest( public SetConfigurationRequest newSetConfigurationRequest( ClientId clientId, RaftPeerId leaderId, - RaftPeer... peers) { + RaftPeer... raftPeers) { return new SetConfigurationRequest(clientId, leaderId, getGroupId(), CallId.getDefault(), - SetConfigurationRequest.Arguments.newBuilder().setServersInNewConf(peers).build()); + SetConfigurationRequest.Arguments.newBuilder().setServersInNewConf(raftPeers).build()); } - public void setConfiguration(RaftPeer... peers) throws IOException { + public void setConfiguration(RaftPeer... raftPeers) throws IOException { try(RaftClient client = createClient()) { - LOG.info("Start changing the configuration: {}", Arrays.asList(peers)); - final RaftClientReply reply = client.admin().setConfiguration(peers); + LOG.info("Start changing the configuration: {}", Arrays.asList(raftPeers)); + final RaftClientReply reply = client.admin().setConfiguration(raftPeers); Preconditions.assertTrue(reply.isSuccess()); } } diff --git a/ratis-server/src/test/java/org/apache/ratis/server/impl/RaftServerTestUtil.java b/ratis-server/src/test/java/org/apache/ratis/server/impl/RaftServerTestUtil.java index 58a51e0514..0054e16018 100644 --- a/ratis-server/src/test/java/org/apache/ratis/server/impl/RaftServerTestUtil.java +++ b/ratis-server/src/test/java/org/apache/ratis/server/impl/RaftServerTestUtil.java @@ -58,9 +58,12 @@ import java.util.stream.Collectors; import java.util.stream.Stream; -public class RaftServerTestUtil { +public final class RaftServerTestUtil { static final Logger LOG = LoggerFactory.getLogger(RaftServerTestUtil.class); + private RaftServerTestUtil() { + } + public static final RaftGroupMemberId TEST_MEMBER_ID = RaftGroupMemberId.valueOf( RaftPeerId.valueOf("test"), RaftGroupId.emptyGroupId()); @@ -248,7 +251,8 @@ private static Collection parseMinorityPeers(MiniRaftCluster cluster, } // All new peers has been added. Handle the removed peers. - List peersToRemove = peers.stream().filter(peer -> !peersInNewConf.contains(peer)).collect(Collectors.toList()); + List peersToRemove = peers.stream().filter(peer -> !peersInNewConf.contains(peer)) + .collect(Collectors.toList()); if (!peersToRemove.isEmpty()) { return peersInNewConf; } diff --git a/ratis-server/src/test/java/org/apache/ratis/server/impl/RetryCacheTestUtil.java b/ratis-server/src/test/java/org/apache/ratis/server/impl/RetryCacheTestUtil.java index e5a55e49ca..439245b496 100644 --- a/ratis-server/src/test/java/org/apache/ratis/server/impl/RetryCacheTestUtil.java +++ b/ratis-server/src/test/java/org/apache/ratis/server/impl/RetryCacheTestUtil.java @@ -36,7 +36,11 @@ import static org.mockito.Mockito.mock; import static org.mockito.Mockito.when; -public class RetryCacheTestUtil { +public final class RetryCacheTestUtil { + + private RetryCacheTestUtil() { + } + public static RetryCache createRetryCache(){ return new RetryCacheImpl(TimeDuration.valueOf(60, TimeUnit.SECONDS), null); } diff --git a/ratis-server/src/test/java/org/apache/ratis/server/impl/TestLogAppenderMetrics.java b/ratis-server/src/test/java/org/apache/ratis/server/impl/TestLogAppenderMetrics.java index 4bd075ef66..6a46734689 100644 --- a/ratis-server/src/test/java/org/apache/ratis/server/impl/TestLogAppenderMetrics.java +++ b/ratis-server/src/test/java/org/apache/ratis/server/impl/TestLogAppenderMetrics.java @@ -83,16 +83,16 @@ long getNextIndex() { return nextIndex; } - void updateNextIndex(long nextIndex) { - this.nextIndex = nextIndex; + void updateNextIndex(long index) { + this.nextIndex = index; } long getMatchIndex() { return matchIndex; } - void updateMatchIndex(long matchIndex) { - this.matchIndex = matchIndex; + void updateMatchIndex(long index) { + this.matchIndex = index; } Timestamp getLastRpcTime() { diff --git a/ratis-server/src/test/java/org/apache/ratis/server/simulation/MiniRaftClusterWithSimulatedRpc.java b/ratis-server/src/test/java/org/apache/ratis/server/simulation/MiniRaftClusterWithSimulatedRpc.java index d0e7e9f5c8..922796cb80 100644 --- a/ratis-server/src/test/java/org/apache/ratis/server/simulation/MiniRaftClusterWithSimulatedRpc.java +++ b/ratis-server/src/test/java/org/apache/ratis/server/simulation/MiniRaftClusterWithSimulatedRpc.java @@ -34,7 +34,7 @@ import static org.apache.ratis.conf.ConfUtils.requireMin; -public class MiniRaftClusterWithSimulatedRpc extends MiniRaftCluster { +public final class MiniRaftClusterWithSimulatedRpc extends MiniRaftCluster { static final Logger LOG = LoggerFactory.getLogger(MiniRaftClusterWithSimulatedRpc.class); public static final Factory FACTORY diff --git a/ratis-server/src/test/java/org/apache/ratis/statemachine/impl/SimpleStateMachine4Testing.java b/ratis-server/src/test/java/org/apache/ratis/statemachine/impl/SimpleStateMachine4Testing.java index 312c9508d3..afab27680f 100644 --- a/ratis-server/src/test/java/org/apache/ratis/statemachine/impl/SimpleStateMachine4Testing.java +++ b/ratis-server/src/test/java/org/apache/ratis/statemachine/impl/SimpleStateMachine4Testing.java @@ -208,12 +208,12 @@ private void put(LogEntryProto entry) { } @Override - public synchronized void initialize(RaftServer server, RaftGroupId groupId, + public synchronized void initialize(RaftServer server, RaftGroupId raftGroupId, RaftStorage raftStorage) throws IOException { LOG.info("Initializing " + this); - this.groupId = groupId; + this.groupId = raftGroupId; getLifeCycle().startAndTransition(() -> { - super.initialize(server, groupId, raftStorage); + super.initialize(server, raftGroupId, raftStorage); storage.init(raftStorage); loadSnapshot(storage.getLatestSnapshot()); diff --git a/ratis-test/src/test/java/org/apache/ratis/datastream/DataStreamAsyncClusterTests.java b/ratis-test/src/test/java/org/apache/ratis/datastream/DataStreamAsyncClusterTests.java index 2fcf500e2c..a343502069 100644 --- a/ratis-test/src/test/java/org/apache/ratis/datastream/DataStreamAsyncClusterTests.java +++ b/ratis-test/src/test/java/org/apache/ratis/datastream/DataStreamAsyncClusterTests.java @@ -103,15 +103,18 @@ void runTestDataStream(CLUSTER cluster) throws Exception { long runMultipleStreams(CLUSTER cluster, boolean stepDownLeader) { final List> futures = new ArrayList<>(); - futures.add(CompletableFuture.supplyAsync(() -> runTestDataStream(cluster, 5, 10, 100_000, 10, stepDownLeader), executor)); - futures.add(CompletableFuture.supplyAsync(() -> runTestDataStream(cluster, 2, 20, 1_000, 5_000, stepDownLeader), executor)); + futures.add(CompletableFuture.supplyAsync(() -> + runTestDataStream(cluster, 5, 10, 100_000, 10, stepDownLeader), executor)); + futures.add(CompletableFuture.supplyAsync(() -> + runTestDataStream(cluster, 2, 20, 1_000, 5_000, stepDownLeader), executor)); return futures.stream() .map(CompletableFuture::join) .max(Long::compareTo) .orElseThrow(IllegalStateException::new); } - void runTestDataStream(CLUSTER cluster, boolean stepDownLeader, CheckedBiFunction runMethod) throws Exception { + void runTestDataStream(CLUSTER cluster, boolean stepDownLeader, + CheckedBiFunction runMethod) throws Exception { RaftTestUtil.waitForLeader(cluster); final long maxIndex = runMethod.apply(cluster, stepDownLeader); diff --git a/ratis-test/src/test/java/org/apache/ratis/datastream/DataStreamBaseTest.java b/ratis-test/src/test/java/org/apache/ratis/datastream/DataStreamBaseTest.java index 2ac01ac1f5..ee56f334e1 100644 --- a/ratis-test/src/test/java/org/apache/ratis/datastream/DataStreamBaseTest.java +++ b/ratis-test/src/test/java/org/apache/ratis/datastream/DataStreamBaseTest.java @@ -44,8 +44,8 @@ abstract class DataStreamBaseTest extends BaseTest { RaftConfiguration getRaftConf() { - final List peers = servers.stream().map(Server::getPeer).collect(Collectors.toList()); - return RaftServerTestUtil.newRaftConfiguration(peers); + final List peerList = servers.stream().map(Server::getPeer).collect(Collectors.toList()); + return RaftServerTestUtil.newRaftConfiguration(peerList); } static class Server { @@ -90,21 +90,21 @@ Server getPrimaryServer() { return servers.get(0); } - void setup(RaftGroupId groupId, List peers, List raftServers) throws Exception { - raftGroup = RaftGroup.valueOf(groupId, peers); - this.peers = peers; - servers = new ArrayList<>(peers.size()); + void setup(RaftGroupId groupId, List peerList, List raftServers) throws Exception { + raftGroup = RaftGroup.valueOf(groupId, peerList); + this.peers = peerList; + servers = new ArrayList<>(peerList.size()); // start stream servers on raft peers. - for (int i = 0; i < peers.size(); i++) { - final Server server = new Server(peers.get(i), raftServers.get(i)); - server.addRaftPeers(removePeerFromList(peers.get(i), peers)); + for (int i = 0; i < peerList.size(); i++) { + final Server server = new Server(peerList.get(i), raftServers.get(i)); + server.addRaftPeers(removePeerFromList(peerList.get(i), peerList)); server.start(); servers.add(server); } } - private Collection removePeerFromList(RaftPeer peer, List peers) { - List otherPeers = new ArrayList<>(peers); + private Collection removePeerFromList(RaftPeer peer, List peerList) { + List otherPeers = new ArrayList<>(peerList); otherPeers.remove(peer); return otherPeers; } diff --git a/ratis-test/src/test/java/org/apache/ratis/datastream/DataStreamTestUtils.java b/ratis-test/src/test/java/org/apache/ratis/datastream/DataStreamTestUtils.java index 40c005c08d..dfbd98addd 100644 --- a/ratis-test/src/test/java/org/apache/ratis/datastream/DataStreamTestUtils.java +++ b/ratis-test/src/test/java/org/apache/ratis/datastream/DataStreamTestUtils.java @@ -336,7 +336,8 @@ static void assertHeader(RaftServer server, RaftClientRequest header, int dataSi Assertions.assertEquals(RaftClientRequest.dataStreamRequestType(), header.getType()); // check stream - final MultiDataStreamStateMachine stateMachine = (MultiDataStreamStateMachine) server.getDivision(header.getRaftGroupId()).getStateMachine(); + final MultiDataStreamStateMachine stateMachine = + (MultiDataStreamStateMachine) server.getDivision(header.getRaftGroupId()).getStateMachine(); final SingleDataStream stream = stateMachine.getSingleDataStream(header); final MyDataChannel channel = stream.getDataChannel(); Assertions.assertEquals(dataSize, channel.getBytesWritten()); diff --git a/ratis-test/src/test/java/org/apache/ratis/datastream/MiniRaftClusterWithRpcTypeGrpcAndDataStreamTypeNetty.java b/ratis-test/src/test/java/org/apache/ratis/datastream/MiniRaftClusterWithRpcTypeGrpcAndDataStreamTypeNetty.java index b9e20fb82e..82fab6477a 100644 --- a/ratis-test/src/test/java/org/apache/ratis/datastream/MiniRaftClusterWithRpcTypeGrpcAndDataStreamTypeNetty.java +++ b/ratis-test/src/test/java/org/apache/ratis/datastream/MiniRaftClusterWithRpcTypeGrpcAndDataStreamTypeNetty.java @@ -30,7 +30,7 @@ /** * A {@link MiniRaftCluster} with {{@link SupportedRpcType#GRPC}} and {@link SupportedDataStreamType#NETTY}. */ -public class MiniRaftClusterWithRpcTypeGrpcAndDataStreamTypeNetty extends MiniRaftClusterWithGrpc { +public final class MiniRaftClusterWithRpcTypeGrpcAndDataStreamTypeNetty extends MiniRaftClusterWithGrpc { static class Factory extends MiniRaftCluster.Factory { private final Parameters parameters; @@ -49,15 +49,16 @@ public MiniRaftClusterWithRpcTypeGrpcAndDataStreamTypeNetty newCluster(String[] public static final Factory FACTORY = new Factory(null); - public interface FactoryGet extends MiniRaftCluster.Factory.Get { + public interface FactoryGet + extends MiniRaftCluster.Factory.Get { @Override default MiniRaftCluster.Factory getFactory() { return FACTORY; } } - private MiniRaftClusterWithRpcTypeGrpcAndDataStreamTypeNetty(String[] ids, String[] listenerIds, RaftProperties properties, - Parameters parameters) { + private MiniRaftClusterWithRpcTypeGrpcAndDataStreamTypeNetty(String[] ids, String[] listenerIds, + RaftProperties properties, Parameters parameters) { super(ids, listenerIds, properties, parameters); } diff --git a/ratis-test/src/test/java/org/apache/ratis/datastream/MiniRaftClusterWithRpcTypeNettyAndDataStreamTypeNetty.java b/ratis-test/src/test/java/org/apache/ratis/datastream/MiniRaftClusterWithRpcTypeNettyAndDataStreamTypeNetty.java index 1e5149b431..9179fa0393 100644 --- a/ratis-test/src/test/java/org/apache/ratis/datastream/MiniRaftClusterWithRpcTypeNettyAndDataStreamTypeNetty.java +++ b/ratis-test/src/test/java/org/apache/ratis/datastream/MiniRaftClusterWithRpcTypeNettyAndDataStreamTypeNetty.java @@ -30,7 +30,7 @@ /** * A {@link MiniRaftCluster} with {{@link SupportedRpcType#NETTY}} and {@link SupportedDataStreamType#NETTY}. */ -public class MiniRaftClusterWithRpcTypeNettyAndDataStreamTypeNetty extends MiniRaftClusterWithNetty { +public final class MiniRaftClusterWithRpcTypeNettyAndDataStreamTypeNetty extends MiniRaftClusterWithNetty { public static final Factory FACTORY = new Factory() { @Override @@ -49,7 +49,8 @@ default Factory getFactor } } - private MiniRaftClusterWithRpcTypeNettyAndDataStreamTypeNetty(String[] ids, String[] listenerIds, RaftProperties properties) { + private MiniRaftClusterWithRpcTypeNettyAndDataStreamTypeNetty(String[] ids, String[] listenerIds, + RaftProperties properties) { super(ids, listenerIds, properties); } diff --git a/ratis-test/src/test/java/org/apache/ratis/datastream/TestDataStreamSslWithRpcTypeGrpcAndDataStreamTypeNetty.java b/ratis-test/src/test/java/org/apache/ratis/datastream/TestDataStreamSslWithRpcTypeGrpcAndDataStreamTypeNetty.java index 8e6d892c83..247a7bcac1 100644 --- a/ratis-test/src/test/java/org/apache/ratis/datastream/TestDataStreamSslWithRpcTypeGrpcAndDataStreamTypeNetty.java +++ b/ratis-test/src/test/java/org/apache/ratis/datastream/TestDataStreamSslWithRpcTypeGrpcAndDataStreamTypeNetty.java @@ -36,12 +36,12 @@ public class TestDataStreamSslWithRpcTypeGrpcAndDataStreamTypeNetty } Parameters newParameters() { - final Parameters parameters = new Parameters(); + final Parameters parameters1 = new Parameters(); final TlsConf serverTlsConfig = SecurityTestUtils.newServerTlsConfig(true); - NettyConfigKeys.DataStream.Server.setTlsConf(parameters, serverTlsConfig); + NettyConfigKeys.DataStream.Server.setTlsConf(parameters1, serverTlsConfig); final TlsConf clientTlsConfig = SecurityTestUtils.newClientTlsConfig(true); - NettyConfigKeys.DataStream.Client.setTlsConf(parameters, clientTlsConfig); - return parameters; + NettyConfigKeys.DataStream.Client.setTlsConf(parameters1, clientTlsConfig); + return parameters1; } private final Supplier parameters = JavaUtils.memoize(this::newParameters); diff --git a/ratis-test/src/test/java/org/apache/ratis/datastream/TestNettyDataStreamWithMock.java b/ratis-test/src/test/java/org/apache/ratis/datastream/TestNettyDataStreamWithMock.java index 1d8c67a43d..8193f1d984 100644 --- a/ratis-test/src/test/java/org/apache/ratis/datastream/TestNettyDataStreamWithMock.java +++ b/ratis-test/src/test/java/org/apache/ratis/datastream/TestNettyDataStreamWithMock.java @@ -114,8 +114,8 @@ private void testMockCluster(int numServers, RaftException leaderException, submitException != null ? submitException : leaderException, getStateMachineException); } - void runTestMockCluster(RaftGroupId groupId, List raftServers, ClientId clientId, int bufferSize, int bufferNum, - Exception expectedException, Exception headerException) throws Exception { + void runTestMockCluster(RaftGroupId groupId, List raftServers, ClientId clientId, int bufferSize, + int bufferNum, Exception expectedException, Exception headerException) throws Exception { try { final List peers = raftServers.stream() .map(TestNettyDataStreamWithMock::newRaftPeer) diff --git a/ratis-test/src/test/java/org/apache/ratis/grpc/TestGrpcMessageMetrics.java b/ratis-test/src/test/java/org/apache/ratis/grpc/TestGrpcMessageMetrics.java index aee13223b8..812c691e20 100644 --- a/ratis-test/src/test/java/org/apache/ratis/grpc/TestGrpcMessageMetrics.java +++ b/ratis-test/src/test/java/org/apache/ratis/grpc/TestGrpcMessageMetrics.java @@ -60,7 +60,8 @@ static void sendMessages(MiniRaftCluster cluster) throws Exception { client.async().send(new RaftTestUtil.SimpleMessage("abc")); } // Wait for commits to happen on leader - JavaUtils.attempt(() -> assertMessageCount(cluster.getLeader()), 100, HUNDRED_MILLIS, cluster.getLeader().getId() + "-assertMessageCount", null); + JavaUtils.attempt(() -> assertMessageCount(cluster.getLeader()), 100, HUNDRED_MILLIS, + cluster.getLeader().getId() + "-assertMessageCount", null); } static void assertMessageCount(RaftServer.Division server) { @@ -68,6 +69,7 @@ static void assertMessageCount(RaftServer.Division server) { GrpcService service = (GrpcService) RaftServerTestUtil.getServerRpc(server); RatisMetricRegistry registry = service.getServerInterceptor().getMetrics().getRegistry(); String counter_prefix = serverId + "_" + "ratis.grpc.RaftServerProtocolService"; - Assertions.assertTrue(registry.counter(counter_prefix + "_" + "requestVote" + "_OK_completed_total").getCount() > 0); + Assertions.assertTrue( + registry.counter(counter_prefix + "_" + "requestVote" + "_OK_completed_total").getCount() > 0); } } \ No newline at end of file diff --git a/ratis-test/src/test/java/org/apache/ratis/grpc/TestRaftServerWithGrpc.java b/ratis-test/src/test/java/org/apache/ratis/grpc/TestRaftServerWithGrpc.java index 05d772c17e..e6e39e8ee6 100644 --- a/ratis-test/src/test/java/org/apache/ratis/grpc/TestRaftServerWithGrpc.java +++ b/ratis-test/src/test/java/org/apache/ratis/grpc/TestRaftServerWithGrpc.java @@ -302,7 +302,7 @@ void testRequestMetrics(MiniRaftClusterWithGrpc cluster) throws Exception { } } } - + static ByteString randomByteString(int size) { final ByteString.Output out = ByteString.newOutput(size); final ThreadLocalRandom random = ThreadLocalRandom.current(); diff --git a/ratis-test/src/test/java/org/apache/ratis/grpc/TestRetryCacheWithGrpc.java b/ratis-test/src/test/java/org/apache/ratis/grpc/TestRetryCacheWithGrpc.java index b244691437..5a704689da 100644 --- a/ratis-test/src/test/java/org/apache/ratis/grpc/TestRetryCacheWithGrpc.java +++ b/ratis-test/src/test/java/org/apache/ratis/grpc/TestRetryCacheWithGrpc.java @@ -89,7 +89,8 @@ void assertRetryCacheEntry(RaftClient client, long callId, boolean exist) throws assertRetryCacheEntry(client, callId, exist, false); } - void assertRetryCacheEntry(RaftClient client, long callId, boolean exist, boolean eventually) throws InterruptedException { + void assertRetryCacheEntry(RaftClient client, long callId, boolean exist, boolean eventually) + throws InterruptedException { Supplier lookup = () -> RetryCacheTestUtil.get(leader, client.getId(), callId); Consumer assertion = exist ? Assertions::assertNotNull : Assertions::assertNull; if (eventually) { diff --git a/ratis-test/src/test/java/org/apache/ratis/server/ServerRestartTests.java b/ratis-test/src/test/java/org/apache/ratis/server/ServerRestartTests.java index db4e92b7ce..ee93b68855 100644 --- a/ratis-test/src/test/java/org/apache/ratis/server/ServerRestartTests.java +++ b/ratis-test/src/test/java/org/apache/ratis/server/ServerRestartTests.java @@ -164,7 +164,8 @@ static void writeSomething(Supplier newMessage, MiniRaftCluster cluster } } - static void assertTruncatedLog(RaftPeerId id, File openLogFile, long lastIndex, MiniRaftCluster cluster) throws Exception { + static void assertTruncatedLog(RaftPeerId id, File openLogFile, long lastIndex, MiniRaftCluster cluster) + throws Exception { // truncate log if (openLogFile.length() > 0) { FileUtils.truncateFile(openLogFile, openLogFile.length() - 1); @@ -216,14 +217,14 @@ void runTestRestartWithCorruptedLogHeader(MiniRaftCluster cluster) throws Except } static void assertCorruptedLogHeader(RaftPeerId id, File openLogFile, int partialLength, - MiniRaftCluster cluster, Logger LOG) throws Exception { + MiniRaftCluster cluster, Logger log) throws Exception { Preconditions.assertTrue(partialLength < SegmentedRaftLogFormat.getHeaderLength()); try(final RandomAccessFile raf = new RandomAccessFile(openLogFile, "rw")) { final ByteBuffer header = SegmentedRaftLogFormat.getHeaderBytebuffer(); - LOG.info("header = {}", StringUtils.bytes2HexString(header)); + log.info("header = {}", StringUtils.bytes2HexString(header)); final byte[] corrupted = new byte[header.remaining()]; header.get(corrupted, 0, partialLength); - LOG.info("corrupted = {}", StringUtils.bytes2HexString(corrupted)); + log.info("corrupted = {}", StringUtils.bytes2HexString(corrupted)); raf.write(corrupted); } final RaftServer.Division server = cluster.restartServer(id, false); diff --git a/ratis-test/src/test/java/org/apache/ratis/server/impl/TestRaftConfiguration.java b/ratis-test/src/test/java/org/apache/ratis/server/impl/TestRaftConfiguration.java index 77ec0ed29a..fa2c524e71 100644 --- a/ratis-test/src/test/java/org/apache/ratis/server/impl/TestRaftConfiguration.java +++ b/ratis-test/src/test/java/org/apache/ratis/server/impl/TestRaftConfiguration.java @@ -39,8 +39,8 @@ public void testIsHighestPriority() { Integer node3 = 2; PeerConfiguration peerConfig = new PeerConfiguration(raftPeersWithPriority(node1, node2, node3)); RaftConfiguration config = RaftConfigurationImpl.newBuilder().setConf(peerConfig).build(); - RaftPeer[] allRaftPeers = peerConfig.getPeers(RaftProtos.RaftPeerRole.FOLLOWER).toArray(new RaftPeer[peerConfig.getPeers( - RaftProtos.RaftPeerRole.FOLLOWER).size()]); + RaftPeer[] allRaftPeers = peerConfig.getPeers(RaftProtos.RaftPeerRole.FOLLOWER).toArray( + new RaftPeer[peerConfig.getPeers(RaftProtos.RaftPeerRole.FOLLOWER).size()]); // First member should not have highest priority assertFalse(RaftServerTestUtil.isHighestPriority(config, diff --git a/ratis-test/src/test/java/org/apache/ratis/server/impl/TestRaftServerJmx.java b/ratis-test/src/test/java/org/apache/ratis/server/impl/TestRaftServerJmx.java index d549e3781f..b24ec72c06 100644 --- a/ratis-test/src/test/java/org/apache/ratis/server/impl/TestRaftServerJmx.java +++ b/ratis-test/src/test/java/org/apache/ratis/server/impl/TestRaftServerJmx.java @@ -87,20 +87,33 @@ public void testRegister() throws JMException { static void runRegister(boolean expectToSucceed, String name, JmxRegister jmx) { final RaftServerMXBean mBean = new RaftServerMXBean() { @Override - public String getId() { return null; } + public String getId() { + return null; + } @Override - public String getLeaderId() { return null; } + public String getLeaderId() { + return null; + } @Override - public long getCurrentTerm() { return 0; } + public long getCurrentTerm() { + return 0; + } @Override - public String getGroupId() { return null; } + public String getGroupId() { + return null; + } @Override - public String getRole() { return null; } + public String getRole() { + return null; + } @Override - public List getFollowers() { return null; } + public List getFollowers() { + return null; + } @Override - public List getGroups() { return null; } - + public List getGroups() { + return null; + } }; final String id = RaftPeerId.valueOf(name).toString(); final String groupId = RaftGroupId.randomId().toString(); diff --git a/ratis-test/src/test/java/org/apache/ratis/server/raftlog/TestRaftLogMetrics.java b/ratis-test/src/test/java/org/apache/ratis/server/raftlog/TestRaftLogMetrics.java index 65493d7a37..6f57d42a5d 100644 --- a/ratis-test/src/test/java/org/apache/ratis/server/raftlog/TestRaftLogMetrics.java +++ b/ratis-test/src/test/java/org/apache/ratis/server/raftlog/TestRaftLogMetrics.java @@ -110,7 +110,8 @@ static void runTestRaftLogMetrics(MiniRaftCluster cluster) throws Exception { } // Wait for commits to happen on leader - JavaUtils.attempt(() -> assertCommitCount(cluster.getLeader(), numMsg), 10, HUNDRED_MILLIS, cluster.getLeader().getId() + "-assertCommitCount", null); + JavaUtils.attempt(() -> assertCommitCount(cluster.getLeader(), numMsg), 10, HUNDRED_MILLIS, + cluster.getLeader().getId() + "-assertCommitCount", null); } static void assertCommitCount(RaftServer.Division server, int expectedMsgs) { @@ -147,7 +148,8 @@ static void assertFlushCount(RaftServer.Division server) throws Exception { } static void assertRaftLogWritePathMetrics(RaftServer.Division server) throws Exception { - final String syncTimeMetric = RaftStorageTestUtils.getRaftLogFullMetric(server.getMemberId().toString(), RAFT_LOG_SYNC_TIME); + final String syncTimeMetric = RaftStorageTestUtils.getRaftLogFullMetric(server.getMemberId().toString(), + RAFT_LOG_SYNC_TIME); final RatisMetricRegistryImpl ratisMetricRegistry = getRegistry(server.getMemberId()); //Test sync count @@ -173,13 +175,16 @@ static void assertRaftLogWritePathMetrics(RaftServer.Division server) throws Exc Assertions.assertTrue(ratisMetricRegistry.counter(RAFT_LOG_FLUSH_COUNT).getCount() > 0); Assertions.assertTrue(ratisMetricRegistry.counter(RAFT_LOG_APPEND_ENTRY_COUNT).getCount() > 0); - final DefaultTimekeeperImpl appendEntry = (DefaultTimekeeperImpl) ratisMetricRegistry.timer(RAFT_LOG_APPEND_ENTRY_LATENCY); + final DefaultTimekeeperImpl appendEntry = + (DefaultTimekeeperImpl) ratisMetricRegistry.timer(RAFT_LOG_APPEND_ENTRY_LATENCY); Assertions.assertTrue(appendEntry.getTimer().getMeanRate() > 0); - final DefaultTimekeeperImpl taskQueue = (DefaultTimekeeperImpl) ratisMetricRegistry.timer(RAFT_LOG_TASK_QUEUE_TIME); + final DefaultTimekeeperImpl taskQueue = + (DefaultTimekeeperImpl) ratisMetricRegistry.timer(RAFT_LOG_TASK_QUEUE_TIME); Assertions.assertTrue(taskQueue.getTimer().getMeanRate() > 0); - final DefaultTimekeeperImpl enqueueDelay = (DefaultTimekeeperImpl) ratisMetricRegistry.timer(RAFT_LOG_TASK_ENQUEUE_DELAY); + final DefaultTimekeeperImpl enqueueDelay = + (DefaultTimekeeperImpl) ratisMetricRegistry.timer(RAFT_LOG_TASK_ENQUEUE_DELAY); Assertions.assertTrue(enqueueDelay.getTimer().getMeanRate() > 0); final DefaultTimekeeperImpl write = (DefaultTimekeeperImpl) ratisMetricRegistry.timer( diff --git a/ratis-test/src/test/java/org/apache/ratis/server/raftlog/segmented/TestBufferedWriteChannel.java b/ratis-test/src/test/java/org/apache/ratis/server/raftlog/segmented/TestBufferedWriteChannel.java index cefd5f6747..4df5f4992c 100644 --- a/ratis-test/src/test/java/org/apache/ratis/server/raftlog/segmented/TestBufferedWriteChannel.java +++ b/ratis-test/src/test/java/org/apache/ratis/server/raftlog/segmented/TestBufferedWriteChannel.java @@ -54,7 +54,7 @@ public long read(ByteBuffer[] dsts, int offset, int length) { } @Override - public int read(ByteBuffer dst, long position) { + public int read(ByteBuffer dst, long pos) { throw new UnsupportedOperationException(); } @@ -64,7 +64,7 @@ public long write(ByteBuffer[] srcs, int offset, int length) { } @Override - public int write(ByteBuffer src, long position) { + public int write(ByteBuffer src, long pos) { throw new UnsupportedOperationException(); } @@ -103,27 +103,27 @@ public void force(boolean metaData) { } @Override - public long transferTo(long position, long count, WritableByteChannel target) { + public long transferTo(long pos, long count, WritableByteChannel target) { throw new UnsupportedOperationException(); } @Override - public long transferFrom(ReadableByteChannel src, long position, long count) { + public long transferFrom(ReadableByteChannel src, long pos, long count) { throw new UnsupportedOperationException(); } @Override - public MappedByteBuffer map(MapMode mode, long position, long size) { + public MappedByteBuffer map(MapMode mode, long pos, long size) { throw new UnsupportedOperationException(); } @Override - public FileLock lock(long position, long size, boolean shared) { + public FileLock lock(long pos, long size, boolean shared) { throw new UnsupportedOperationException(); } @Override - public FileLock tryLock(long position, long size, boolean shared) { + public FileLock tryLock(long pos, long size, boolean shared) { throw new UnsupportedOperationException(); } diff --git a/ratis-test/src/test/java/org/apache/ratis/server/raftlog/segmented/TestLogSegment.java b/ratis-test/src/test/java/org/apache/ratis/server/raftlog/segmented/TestLogSegment.java index 8355c67333..4e04e9e62c 100644 --- a/ratis-test/src/test/java/org/apache/ratis/server/raftlog/segmented/TestLogSegment.java +++ b/ratis-test/src/test/java/org/apache/ratis/server/raftlog/segmented/TestLogSegment.java @@ -88,7 +88,8 @@ public void tearDown() throws Exception { File prepareLog(boolean isOpen, long startIndex, int numEntries, long term, boolean isLastEntryPartiallyWritten) throws IOException { if (!isOpen) { - Preconditions.assertTrue(!isLastEntryPartiallyWritten, "For closed log, the last entry cannot be partially written."); + Preconditions.assertTrue(!isLastEntryPartiallyWritten, + "For closed log, the last entry cannot be partially written."); } RaftStorage storage = RaftStorageTestUtils.newRaftStorage(storageDir); final File file = LogSegmentStartEnd.valueOf(startIndex, startIndex + numEntries - 1, isOpen).getFile(storage); diff --git a/ratis-test/src/test/java/org/apache/ratis/server/raftlog/segmented/TestSegmentedRaftLog.java b/ratis-test/src/test/java/org/apache/ratis/server/raftlog/segmented/TestSegmentedRaftLog.java index 149f6286ac..b564501f36 100644 --- a/ratis-test/src/test/java/org/apache/ratis/server/raftlog/segmented/TestSegmentedRaftLog.java +++ b/ratis-test/src/test/java/org/apache/ratis/server/raftlog/segmented/TestSegmentedRaftLog.java @@ -263,7 +263,8 @@ static List prepareLogEntries(SegmentRange range, return eList; } - static LogEntryProto prepareLogEntry(long term, long index, Supplier stringSupplier, boolean hasStataMachineData) { + static LogEntryProto prepareLogEntry(long term, long index, Supplier stringSupplier, + boolean hasStataMachineData) { final SimpleOperation m = stringSupplier == null? new SimpleOperation("m" + index, hasStataMachineData): new SimpleOperation(stringSupplier.get(), hasStataMachineData); @@ -514,7 +515,8 @@ public void testPurgeOnClosedSegmentsWithPurgeGap() throws Exception { purgeAndVerify(startTerm, endTerm, segmentSize, 1000, endIndexOfClosedSegment, expectedIndex); } - private void purgeAndVerify(int startTerm, int endTerm, int segmentSize, int purgeGap, long purgeIndex, long expectedIndex) throws Exception { + private void purgeAndVerify(int startTerm, int endTerm, int segmentSize, int purgeGap, long purgeIndex, + long expectedIndex) throws Exception { List ranges = prepareRanges(startTerm, endTerm, segmentSize, 0); List entries = prepareLogEntries(ranges, null); @@ -543,7 +545,8 @@ public void testAppendEntriesWithInconsistency(Boolean useAsyncFlush, Boolean sm List entries = prepareLogEntries(ranges, null); final RetryCache retryCache = RetryCacheTestUtil.createRetryCache(); - try (SegmentedRaftLog raftLog = RetryCacheTestUtil.newSegmentedRaftLog(MEMBER_ID, retryCache, storage, properties)) { + try (SegmentedRaftLog raftLog = + RetryCacheTestUtil.newSegmentedRaftLog(MEMBER_ID, retryCache, storage, properties)) { raftLog.open(RaftLog.INVALID_LOG_INDEX, null); entries.forEach(entry -> RetryCacheTestUtil.createEntry(retryCache, entry)); // append entries to the raftlog @@ -558,7 +561,8 @@ public void testAppendEntriesWithInconsistency(Boolean useAsyncFlush, Boolean sm List newEntries = prepareLogEntries( Arrays.asList(r1, r2, r3), null); - try (SegmentedRaftLog raftLog = RetryCacheTestUtil.newSegmentedRaftLog(MEMBER_ID, retryCache, storage, properties)) { + try (SegmentedRaftLog raftLog = + RetryCacheTestUtil.newSegmentedRaftLog(MEMBER_ID, retryCache, storage, properties)) { raftLog.open(RaftLog.INVALID_LOG_INDEX, null); LOG.info("newEntries[0] = {}", newEntries.get(0)); final int last = newEntries.size() - 1; @@ -575,7 +579,8 @@ public void testAppendEntriesWithInconsistency(Boolean useAsyncFlush, Boolean sm } // load the raftlog again and check - try (SegmentedRaftLog raftLog = RetryCacheTestUtil.newSegmentedRaftLog(MEMBER_ID, retryCache, storage, properties)) { + try (SegmentedRaftLog raftLog = + RetryCacheTestUtil.newSegmentedRaftLog(MEMBER_ID, retryCache, storage, properties)) { raftLog.open(RaftLog.INVALID_LOG_INDEX, null); checkEntries(raftLog, entries, 0, 650); checkEntries(raftLog, newEntries, 100, 100); @@ -706,7 +711,8 @@ void assertIndices(RaftLog raftLog, long expectedFlushIndex, long expectedNextIn Assertions.assertEquals(expectedNextIndex, raftLog.getNextIndex()); } - void assertIndicesMultipleAttempts(RaftLog raftLog, long expectedFlushIndex, long expectedNextIndex) throws Exception { + void assertIndicesMultipleAttempts(RaftLog raftLog, long expectedFlushIndex, long expectedNextIndex) + throws Exception { JavaUtils.attempt(() -> assertIndices(raftLog, expectedFlushIndex, expectedNextIndex), 10, HUNDRED_MILLIS, "assertIndices", LOG); } diff --git a/ratis-test/src/test/java/org/apache/ratis/shell/cli/sh/LocalCommandIntegrationTest.java b/ratis-test/src/test/java/org/apache/ratis/shell/cli/sh/LocalCommandIntegrationTest.java index a85c247d66..afc13837c5 100644 --- a/ratis-test/src/test/java/org/apache/ratis/shell/cli/sh/LocalCommandIntegrationTest.java +++ b/ratis-test/src/test/java/org/apache/ratis/shell/cli/sh/LocalCommandIntegrationTest.java @@ -61,14 +61,16 @@ public void testDuplicatedPeerIds() throws Exception { testDuplicatedPeers(duplicatedIdsList, "ID", "peer1_ID1"); } - private void testDuplicatedPeers(String[] peersList, String expectedErrorMessagePart, String expectedDuplicatedValue) throws Exception { + private void testDuplicatedPeers(String[] peersList, String expectedErrorMessagePart, String expectedDuplicatedValue) + throws Exception { for (String peersStr : peersList) { StringPrintStream out = new StringPrintStream(); RatisShell shell = new RatisShell(out.getPrintStream()); int ret = shell.run("local", "raftMetaConf", "-peers", peersStr, "-path", "test"); Assertions.assertEquals(-1, ret); String message = out.toString().trim(); - Assertions.assertEquals(String.format("Found duplicated %s: %s. Please make sure the %s of peer have no duplicated value.", + Assertions.assertEquals( + String.format("Found duplicated %s: %s. Please make sure the %s of peer have no duplicated value.", expectedErrorMessagePart, expectedDuplicatedValue, expectedErrorMessagePart), message); } } @@ -101,8 +103,8 @@ public void testRunMethod(@TempDir Path tempDir) throws Exception { String addressRegex = "^[a-zA-Z0-9.-]+:\\d+$"; Pattern pattern = Pattern.compile(addressRegex); - peers.forEach(p -> Assertions.assertTrue( - pattern.matcher(p.getAddress()).matches())); + peers.forEach(peerProto -> Assertions.assertTrue( + pattern.matcher(peerProto.getAddress()).matches())); String peersListStrFromNewMetaConf; if (containsPeerId(peersListStr)) { diff --git a/ratis-test/src/test/java/org/apache/ratis/util/TestLifeCycle.java b/ratis-test/src/test/java/org/apache/ratis/util/TestLifeCycle.java index f08ceb3461..36ef250bf8 100644 --- a/ratis-test/src/test/java/org/apache/ratis/util/TestLifeCycle.java +++ b/ratis-test/src/test/java/org/apache/ratis/util/TestLifeCycle.java @@ -82,7 +82,8 @@ public void invalidTransitions() { testInvalidTransition((from, subject, to) -> subject.transitionAndGet(any -> to), true); } - private static void testInvalidTransition(TriConsumer op, boolean shouldThrow) { + private static void testInvalidTransition(TriConsumer op, + boolean shouldThrow) { LifeCycle subject = new LifeCycle("subject"); for (LifeCycle.State to : new LifeCycle.State[] { RUNNING, EXCEPTION, CLOSING }) { LifeCycle.State from = subject.getCurrentState(); diff --git a/ratis-test/src/test/java/org/apache/ratis/util/TestMinMax.java b/ratis-test/src/test/java/org/apache/ratis/util/TestMinMax.java index 07a9882ad2..0329a1ee97 100644 --- a/ratis-test/src/test/java/org/apache/ratis/util/TestMinMax.java +++ b/ratis-test/src/test/java/org/apache/ratis/util/TestMinMax.java @@ -43,7 +43,8 @@ public void testMinMax() { static void runTestMinMax(LongStream stream) { final List list = stream.collect(ArrayList::new, List::add, List::addAll); - final LongMinMax longMinMax = toLongStream(list).collect(LongMinMax::new, LongMinMax::accumulate, LongMinMax::combine); + final LongMinMax longMinMax = toLongStream(list) + .collect(LongMinMax::new, LongMinMax::accumulate, LongMinMax::combine); if (longMinMax.isInitialized()) { Assertions.assertEquals(toLongStream(list).min().getAsLong(), longMinMax.getMin()); Assertions.assertEquals(toLongStream(list).max().getAsLong(), longMinMax.getMax()); From e2c944841635c7e8fa4fc1375d14d4f659feee55 Mon Sep 17 00:00:00 2001 From: Symious <14933944+symious@users.noreply.github.com> Date: Fri, 17 May 2024 11:11:26 +0800 Subject: [PATCH 047/397] RATIS-2081. Comment that we skip checkstyle on test directory on purpose (#1084) --- pom.xml | 2 ++ 1 file changed, 2 insertions(+) diff --git a/pom.xml b/pom.xml index f8249e0ac7..8292a85e77 100644 --- a/pom.xml +++ b/pom.xml @@ -786,6 +786,8 @@ maven-checkstyle-plugin ${maven-checkstyle-plugin.version} + + dev-support/checkstyle.xml true false From 000318d01397239240c1165b48383c1f334ca8a6 Mon Sep 17 00:00:00 2001 From: Tsz-Wo Nicholas Sze Date: Wed, 22 May 2024 10:17:40 -0700 Subject: [PATCH 048/397] RATIS-2098. RaftLogTruncateTests may fail. (#1097) --- .../apache/ratis/RaftLogTruncateTests.java | 28 ++++++++++++------- 1 file changed, 18 insertions(+), 10 deletions(-) diff --git a/ratis-test/src/test/java/org/apache/ratis/RaftLogTruncateTests.java b/ratis-test/src/test/java/org/apache/ratis/RaftLogTruncateTests.java index c21110ea0e..327163de0f 100644 --- a/ratis-test/src/test/java/org/apache/ratis/RaftLogTruncateTests.java +++ b/ratis-test/src/test/java/org/apache/ratis/RaftLogTruncateTests.java @@ -36,6 +36,7 @@ import org.apache.ratis.statemachine.StateMachine; import org.apache.ratis.statemachine.TransactionContext; import org.apache.ratis.statemachine.impl.SimpleStateMachine4Testing; +import org.apache.ratis.util.JavaUtils; import org.apache.ratis.util.MemoizedSupplier; import org.apache.ratis.util.Slf4jUtils; import org.apache.ratis.util.TimeDuration; @@ -178,9 +179,11 @@ void runTestLogTruncate(MiniRaftCluster cluster, RaftServer.Division oldLeader, LOG.info("done"); } - // kill the old leader - LOG.info("Before killServer {}: {}", oldLeader.getId(), cluster.printServers()); - cluster.killServer(oldLeader.getId()); + // kill remaining peers + LOG.info("Before killServer {}: {}", remainingPeers, cluster.printServers()); + for (RaftPeerId p : remainingPeers) { + cluster.killServer(p); + } LOG.info("After killServer {}: {}", remainingPeers, cluster.printServers()); // restart the earlier followers @@ -198,8 +201,10 @@ void runTestLogTruncate(MiniRaftCluster cluster, RaftServer.Division oldLeader, newLeaderLog.getLastEntryTermIndex()); Assertions.assertTrue(killedPeers.contains(newLeader.getId())); - // restart the old leader - cluster.restartServer(oldLeader.getId(), false); + // restart remaining peers + for (RaftPeerId p : remainingPeers) { + cluster.restartServer(p, false); + } // check RaftLog truncate for (RaftPeerId f : remainingPeers) { @@ -219,8 +224,11 @@ void runTestLogTruncate(MiniRaftCluster cluster, RaftServer.Division oldLeader, for (RaftPeer peer : cluster.getGroup().getPeers()) { final RaftServer.Division division = cluster.getDivision(peer.getId()); assertLogEntries(division, oldLeaderTerm, expectedMessages); - assertEmptyTransactionContextMap(division); - } + final String name = "assertEmptyTransactionContextMap:" + division.getId(); + JavaUtils.attempt(() -> assertEmptyTransactionContextMap(division), + 10, HUNDRED_MILLIS, name, LOG); + + } if (!exceptions.isEmpty()) { LOG.info("{} exceptions", exceptions.size()); @@ -231,9 +239,9 @@ void runTestLogTruncate(MiniRaftCluster cluster, RaftServer.Division oldLeader, } } - static void assertEmptyTransactionContextMap(RaftServer.Division division) { - Assertions.assertTrue(RaftServerTestUtil.getTransactionContextMap(division).isEmpty(), - () -> division.getId() + " TransactionContextMap is non-empty"); + static void assertEmptyTransactionContextMap(RaftServer.Division d) { + final Map> map = RaftServerTestUtil.getTransactionContextMap(d); + Assertions.assertTrue(map.isEmpty(), () -> d.getId() + " TransactionContextMap is non-empty: " + map); } static void assertEntriesInTransactionContextMap(RaftServer.Division division, From 0bb203b67f3a64d4f2e3d75ddcfd3bb97c6472bd Mon Sep 17 00:00:00 2001 From: Potato Date: Fri, 24 May 2024 16:25:18 +0800 Subject: [PATCH 049/397] RATIS-2097 Add a conf to enable/disable Metadata entries. (#1101) --- .../ratis/server/RaftServerConfigKeys.java | 12 +++ .../ratis/server/impl/LeaderStateImpl.java | 6 +- .../ratis/server/impl/LogMetadataTests.java | 80 +++++++++++++++++++ .../grpc/TestLogMetadataTestsWithGrpc.java | 27 +++++++ 4 files changed, 124 insertions(+), 1 deletion(-) create mode 100644 ratis-server/src/test/java/org/apache/ratis/server/impl/LogMetadataTests.java create mode 100644 ratis-test/src/test/java/org/apache/ratis/grpc/TestLogMetadataTestsWithGrpc.java diff --git a/ratis-server-api/src/main/java/org/apache/ratis/server/RaftServerConfigKeys.java b/ratis-server-api/src/main/java/org/apache/ratis/server/RaftServerConfigKeys.java index f6dcbaa66b..8e1d92b754 100644 --- a/ratis-server-api/src/main/java/org/apache/ratis/server/RaftServerConfigKeys.java +++ b/ratis-server-api/src/main/java/org/apache/ratis/server/RaftServerConfigKeys.java @@ -475,6 +475,18 @@ static void setAsyncFlushEnabled(RaftProperties properties, boolean asyncFlush) setBoolean(properties::setBoolean, ASYNC_FLUSH_ENABLED_KEY, asyncFlush); } + /** Log metadata can guarantee that a server can recover commit index and state machine + * even if a majority of servers are dead by consuming a little extra space. */ + String LOG_METADATA_ENABLED_KEY = PREFIX + ".log-metadata.enabled"; + boolean LOG_METADATA_ENABLED_DEFAULT = true; + static boolean logMetadataEnabled(RaftProperties properties) { + return getBoolean(properties::getBoolean, + LOG_METADATA_ENABLED_KEY, LOG_METADATA_ENABLED_DEFAULT, getDefaultLog()); + } + static void setLogMetadataEnabled(RaftProperties properties, boolean logMetadata) { + setBoolean(properties::setBoolean, LOG_METADATA_ENABLED_KEY, logMetadata); + } + /** The policy to handle corrupted raft log. */ enum CorruptionPolicy { /** Rethrow the exception. */ diff --git a/ratis-server/src/main/java/org/apache/ratis/server/impl/LeaderStateImpl.java b/ratis-server/src/main/java/org/apache/ratis/server/impl/LeaderStateImpl.java index dd2e564f74..eacf506154 100644 --- a/ratis-server/src/main/java/org/apache/ratis/server/impl/LeaderStateImpl.java +++ b/ratis-server/src/main/java/org/apache/ratis/server/impl/LeaderStateImpl.java @@ -351,6 +351,7 @@ boolean isApplied() { private final MemoizedSupplier startupLogEntry = MemoizedSupplier.valueOf(StartupLogEntry::new); private final AtomicBoolean isStopped = new AtomicBoolean(); + private final boolean logMetadataEnabled; private final int stagingCatchupGap; private final RaftServerMetricsImpl raftServerMetrics; private final LogAppenderMetrics logAppenderMetrics; @@ -381,6 +382,7 @@ boolean isApplied() { this.pendingStepDown = new PendingStepDown(this); this.readIndexHeartbeats = new ReadIndexHeartbeats(); this.lease = new LeaderLease(properties); + this.logMetadataEnabled = RaftServerConfigKeys.Log.logMetadataEnabled(properties); long maxPendingRequests = RaftServerConfigKeys.Write.elementLimit(properties); double followerGapRatioMax = RaftServerConfigKeys.Write.followerGapRatioMax(properties); @@ -944,7 +946,9 @@ private boolean hasMajority(Predicate isAcked) { private void updateCommit(LogEntryHeader[] entriesToCommit) { final long newCommitIndex = raftLog.getLastCommittedIndex(); - logMetadata(newCommitIndex); + if (logMetadataEnabled) { + logMetadata(newCommitIndex); + } commitIndexChanged(); boolean hasConfiguration = false; diff --git a/ratis-server/src/test/java/org/apache/ratis/server/impl/LogMetadataTests.java b/ratis-server/src/test/java/org/apache/ratis/server/impl/LogMetadataTests.java new file mode 100644 index 0000000000..a834e27144 --- /dev/null +++ b/ratis-server/src/test/java/org/apache/ratis/server/impl/LogMetadataTests.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.ratis.server.impl; + +import java.util.function.Predicate; +import java.util.stream.Collectors; +import org.apache.ratis.BaseTest; +import org.apache.ratis.RaftTestUtil; +import org.apache.ratis.conf.RaftProperties; +import org.apache.ratis.protocol.RaftPeer; +import org.apache.ratis.protocol.RaftPeerId; +import org.apache.ratis.server.RaftServer; +import org.apache.ratis.server.RaftServerConfigKeys; +import org.apache.ratis.server.raftlog.RaftLog; +import org.junit.Assert; +import org.junit.Test; + +public abstract class LogMetadataTests + extends BaseTest + implements MiniRaftCluster.Factory.Get { + + @Test + public void testLogMetadataEnabled() throws Exception { + testLogMetadataBasicTest(true, x -> x > RaftLog.INVALID_LOG_INDEX); + } + + @Test + public void testLogMetadataDisabled() throws Exception { + testLogMetadataBasicTest(false, x -> x == RaftLog.INVALID_LOG_INDEX); + } + + public void testLogMetadataBasicTest(boolean logMetadata, Predicate checker) + throws Exception { + final RaftProperties prop = getProperties(); + RaftServerConfigKeys.Log.setLogMetadataEnabled(prop, logMetadata); + + final MiniRaftCluster cluster = newCluster(3); + try { + cluster.start(); + RaftTestUtil.waitForLeader(cluster); + final RaftServer.Division leader = cluster.getLeader(); + RaftPeerId leaderId = leader.getId(); + + cluster.getLeaderAndSendFirstMessage(true); + + // kill majority servers + for (RaftPeerId id : cluster.getGroup().getPeers().stream().map(RaftPeer::getId) + .filter(x -> !x.equals(leaderId)).collect(Collectors.toList())) { + cluster.killServer(id); + } + + // only restart one server + cluster.restartServer(leaderId, false); + + long commitIndex = cluster.getServer(leaderId).getDivision(cluster.getGroupId()).getRaftLog() + .getLastCommittedIndex(); + + Assert.assertTrue(checker.test(commitIndex)); + } finally { + cluster.shutdown(); + } + } +} diff --git a/ratis-test/src/test/java/org/apache/ratis/grpc/TestLogMetadataTestsWithGrpc.java b/ratis-test/src/test/java/org/apache/ratis/grpc/TestLogMetadataTestsWithGrpc.java new file mode 100644 index 0000000000..1c69c42bbb --- /dev/null +++ b/ratis-test/src/test/java/org/apache/ratis/grpc/TestLogMetadataTestsWithGrpc.java @@ -0,0 +1,27 @@ +/* + * 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.ratis.grpc; + +import org.apache.ratis.server.impl.LogMetadataTests; + +public class TestLogMetadataTestsWithGrpc extends LogMetadataTests + implements MiniRaftClusterWithGrpc.FactoryGet{ + +} From 4f5250f22795331de8dfd4767e6833d25ace1cbc Mon Sep 17 00:00:00 2001 From: Symious <14933944+symious@users.noreply.github.com> Date: Fri, 24 May 2024 17:51:52 +0800 Subject: [PATCH 050/397] RATIS-2099. Cache TermIndexImpl instead of using anonymous class (#1100) --- .../apache/ratis/server/protocol/TermIndex.java | 17 +++++++++++++++-- 1 file changed, 15 insertions(+), 2 deletions(-) diff --git a/ratis-server-api/src/main/java/org/apache/ratis/server/protocol/TermIndex.java b/ratis-server-api/src/main/java/org/apache/ratis/server/protocol/TermIndex.java index 7def686bb5..6a9bd1cfb7 100644 --- a/ratis-server-api/src/main/java/org/apache/ratis/server/protocol/TermIndex.java +++ b/ratis-server-api/src/main/java/org/apache/ratis/server/protocol/TermIndex.java @@ -19,13 +19,21 @@ import org.apache.ratis.proto.RaftProtos.LogEntryProto; import org.apache.ratis.proto.RaftProtos.TermIndexProto; +import org.apache.ratis.thirdparty.com.google.common.cache.Cache; +import org.apache.ratis.thirdparty.com.google.common.cache.CacheBuilder; import java.util.Comparator; import java.util.Optional; +import java.util.concurrent.ExecutionException; +import java.util.concurrent.TimeUnit; /** The term and the log index defined in the Raft consensus algorithm. */ public interface TermIndex extends Comparable { - TermIndex[] EMPTY_ARRAY = {}; + /** An LRU Cache for {@link TermIndex} instances */ + Cache PRIVATE_CACHE = CacheBuilder.newBuilder() + .maximumSize(1 << 16) + .expireAfterAccess(1, TimeUnit.MINUTES) + .build(); /** @return the term. */ long getTerm(); @@ -60,7 +68,7 @@ static TermIndex valueOf(LogEntryProto proto) { /** @return a {@link TermIndex} object. */ static TermIndex valueOf(long term, long index) { - return new TermIndex() { + final TermIndex key = new TermIndex() { @Override public long getTerm() { return term; @@ -98,5 +106,10 @@ public String toString() { return String.format("(t:%s, i:%s)", longToString(term), longToString(index)); } }; + try { + return PRIVATE_CACHE.get(key, () -> key); + } catch (ExecutionException e) { + throw new IllegalStateException("Failed to valueOf(" + term + ", " + index + "), key=" + key, e); + } } } \ No newline at end of file From 96e26ecc689c52323c9cb05e0d8f25d3a4916dd2 Mon Sep 17 00:00:00 2001 From: Symious <14933944+symious@users.noreply.github.com> Date: Sat, 25 May 2024 10:58:20 +0800 Subject: [PATCH 051/397] RATIS-2101. Move TermIndex.PRIVATE_CACHE to Util.CACHE (#1103) --- .../apache/ratis/server/protocol/TermIndex.java | 15 +++++++++------ 1 file changed, 9 insertions(+), 6 deletions(-) diff --git a/ratis-server-api/src/main/java/org/apache/ratis/server/protocol/TermIndex.java b/ratis-server-api/src/main/java/org/apache/ratis/server/protocol/TermIndex.java index 6a9bd1cfb7..a8aa670613 100644 --- a/ratis-server-api/src/main/java/org/apache/ratis/server/protocol/TermIndex.java +++ b/ratis-server-api/src/main/java/org/apache/ratis/server/protocol/TermIndex.java @@ -29,11 +29,14 @@ /** The term and the log index defined in the Raft consensus algorithm. */ public interface TermIndex extends Comparable { - /** An LRU Cache for {@link TermIndex} instances */ - Cache PRIVATE_CACHE = CacheBuilder.newBuilder() - .maximumSize(1 << 16) - .expireAfterAccess(1, TimeUnit.MINUTES) - .build(); + class Util { + /** An LRU Cache for {@link TermIndex} instances */ + private static final Cache CACHE = CacheBuilder.newBuilder() + .maximumSize(1 << 16) + .expireAfterAccess(1, TimeUnit.MINUTES) + .build(); + } + TermIndex[] EMPTY_ARRAY = {}; /** @return the term. */ long getTerm(); @@ -107,7 +110,7 @@ public String toString() { } }; try { - return PRIVATE_CACHE.get(key, () -> key); + return Util.CACHE.get(key, () -> key); } catch (ExecutionException e) { throw new IllegalStateException("Failed to valueOf(" + term + ", " + index + "), key=" + key, e); } From f4c7b8384c4bbff30a027c1e0e9493fc34486a79 Mon Sep 17 00:00:00 2001 From: Chung En Lee Date: Sat, 25 May 2024 04:00:48 +0100 Subject: [PATCH 052/397] RATIS-2100. The `closeFuture` never completed while closing from the `NEW` state (#1102) --- .../org/apache/ratis/server/leader/LogAppenderDaemon.java | 5 ++++- 1 file changed, 4 insertions(+), 1 deletion(-) diff --git a/ratis-server/src/main/java/org/apache/ratis/server/leader/LogAppenderDaemon.java b/ratis-server/src/main/java/org/apache/ratis/server/leader/LogAppenderDaemon.java index 847617426f..5de3f3b4da 100644 --- a/ratis-server/src/main/java/org/apache/ratis/server/leader/LogAppenderDaemon.java +++ b/ratis-server/src/main/java/org/apache/ratis/server/leader/LogAppenderDaemon.java @@ -108,8 +108,11 @@ private void run() { }; public CompletableFuture tryToClose() { - if (lifeCycle.transition(TRY_TO_CLOSE) == CLOSING) { + final State state = lifeCycle.transition(TRY_TO_CLOSE); + if (state == CLOSING) { daemon.interrupt(); + } else if (state == CLOSED) { + closeFuture.complete(state); } return closeFuture; } From c9ed63f1e54650a23658c611606934bc5467d21c Mon Sep 17 00:00:00 2001 From: Ivan Andika Date: Wed, 29 May 2024 02:43:23 +0800 Subject: [PATCH 053/397] RATIS-2089. Add CommitInfoProto in NotReplicatedException (#1105) --- .../ratis/client/impl/ClientProtoUtils.java | 3 ++- .../exceptions/NotReplicatedException.java | 15 +++++++++++++++ .../java/org/apache/ratis/WatchRequestTests.java | 1 + 3 files changed, 18 insertions(+), 1 deletion(-) diff --git a/ratis-client/src/main/java/org/apache/ratis/client/impl/ClientProtoUtils.java b/ratis-client/src/main/java/org/apache/ratis/client/impl/ClientProtoUtils.java index cab9606a0e..5e217e7da2 100644 --- a/ratis-client/src/main/java/org/apache/ratis/client/impl/ClientProtoUtils.java +++ b/ratis-client/src/main/java/org/apache/ratis/client/impl/ClientProtoUtils.java @@ -398,7 +398,8 @@ static RaftClientReply toRaftClientReply(RaftClientReplyProto replyProto) { e = new NotLeaderException(serverMemberId, suggestedLeader, peers); } else if (replyProto.getExceptionDetailsCase() == NOTREPLICATEDEXCEPTION) { final NotReplicatedExceptionProto nre = replyProto.getNotReplicatedException(); - e = new NotReplicatedException(nre.getCallId(), nre.getReplication(), nre.getLogIndex()); + e = new NotReplicatedException(nre.getCallId(), nre.getReplication(), nre.getLogIndex(), + replyProto.getCommitInfosList()); } else if (replyProto.getExceptionDetailsCase().equals(STATEMACHINEEXCEPTION)) { e = toStateMachineException(serverMemberId, replyProto.getStateMachineException()); } else if (replyProto.getExceptionDetailsCase().equals(DATASTREAMEXCEPTION)) { diff --git a/ratis-common/src/main/java/org/apache/ratis/protocol/exceptions/NotReplicatedException.java b/ratis-common/src/main/java/org/apache/ratis/protocol/exceptions/NotReplicatedException.java index 5f48654eec..37ff816245 100644 --- a/ratis-common/src/main/java/org/apache/ratis/protocol/exceptions/NotReplicatedException.java +++ b/ratis-common/src/main/java/org/apache/ratis/protocol/exceptions/NotReplicatedException.java @@ -17,12 +17,17 @@ */ package org.apache.ratis.protocol.exceptions; +import org.apache.ratis.proto.RaftProtos.CommitInfoProto; import org.apache.ratis.proto.RaftProtos.ReplicationLevel; +import java.util.Collection; + public class NotReplicatedException extends RaftException { private final long callId; private final ReplicationLevel requiredReplication; private final long logIndex; + /** This is only populated on client-side since RaftClientReply already has commitInfos */ + private Collection commitInfos; public NotReplicatedException(long callId, ReplicationLevel requiredReplication, long logIndex) { super("Request with call Id " + callId + " and log index " + logIndex @@ -32,6 +37,12 @@ public NotReplicatedException(long callId, ReplicationLevel requiredReplication, this.logIndex = logIndex; } + public NotReplicatedException(long callId, ReplicationLevel requiredReplication, long logIndex, + Collection commitInfos) { + this(callId, requiredReplication, logIndex); + this.commitInfos = commitInfos; + } + public long getCallId() { return callId; } @@ -43,4 +54,8 @@ public ReplicationLevel getRequiredReplication() { public long getLogIndex() { return logIndex; } + + public Collection getCommitInfos() { + return commitInfos; + } } diff --git a/ratis-server/src/test/java/org/apache/ratis/WatchRequestTests.java b/ratis-server/src/test/java/org/apache/ratis/WatchRequestTests.java index 32e4527580..b842ee9db5 100644 --- a/ratis-server/src/test/java/org/apache/ratis/WatchRequestTests.java +++ b/ratis-server/src/test/java/org/apache/ratis/WatchRequestTests.java @@ -559,5 +559,6 @@ static void assertNotReplicatedException(long logIndex, ReplicationLevel replica Assert.assertNotNull(nre); Assert.assertEquals(logIndex, nre.getLogIndex()); Assert.assertEquals(replication, nre.getRequiredReplication()); + Assert.assertNotNull(nre.getCommitInfos()); } } From d9355d60df54e6c31ef529e56ef11b4230d7944d Mon Sep 17 00:00:00 2001 From: Siyao Meng <50227127+smengcl@users.noreply.github.com> Date: Wed, 29 May 2024 11:13:47 -0700 Subject: [PATCH 054/397] RATIS-2102. AsyncApi#send() is not handling retry and reply correctly for replication levels higher than MAJORITY (#1104) --- .../ratis/server/impl/RaftServerImpl.java | 46 ++++++++++++++----- .../java/org/apache/ratis/RaftAsyncTests.java | 33 +++++++++++++ 2 files changed, 68 insertions(+), 11 deletions(-) diff --git a/ratis-server/src/main/java/org/apache/ratis/server/impl/RaftServerImpl.java b/ratis-server/src/main/java/org/apache/ratis/server/impl/RaftServerImpl.java index f6d8dceb03..e049fdddf7 100644 --- a/ratis-server/src/main/java/org/apache/ratis/server/impl/RaftServerImpl.java +++ b/ratis-server/src/main/java/org/apache/ratis/server/impl/RaftServerImpl.java @@ -832,20 +832,14 @@ private CompletableFuture appendTransaction( leaderState.notifySenders(); } - final CompletableFuture future = pending.getFuture(); - if (request.is(TypeCase.WRITE)) { - // check replication - final ReplicationLevel replication = request.getType().getWrite().getReplication(); - if (replication != ReplicationLevel.MAJORITY) { - return future.thenCompose(reply -> waitForReplication(reply, replication)); - } - } - - return future; + return pending.getFuture(); } /** Wait until the given replication requirement is satisfied. */ private CompletableFuture waitForReplication(RaftClientReply reply, ReplicationLevel replication) { + if (!reply.isSuccess()) { + return CompletableFuture.completedFuture(reply); + } final RaftClientRequest.Type type = RaftClientRequest.watchRequestType(reply.getLogIndex(), replication); final RaftClientRequest watch = RaftClientRequest.newBuilder() .setServerId(reply.getServerId()) @@ -854,7 +848,24 @@ private CompletableFuture waitForReplication(RaftClientReply re .setCallId(reply.getCallId()) .setType(type) .build(); - return watchAsync(watch).thenApply(r -> reply); + return watchAsync(watch).thenApply(watchReply -> combineReplies(reply, watchReply)); + } + + private RaftClientReply combineReplies(RaftClientReply reply, RaftClientReply watchReply) { + final RaftClientReply combinedReply = RaftClientReply.newBuilder() + .setServerId(getMemberId()) + // from write reply + .setClientId(reply.getClientId()) + .setCallId(reply.getCallId()) + .setMessage(reply.getMessage()) + .setLogIndex(reply.getLogIndex()) + // from watchReply + .setSuccess(watchReply.isSuccess()) + .setException(watchReply.getException()) + .setCommitInfos(watchReply.getCommitInfos()) + .build(); + LOG.debug("combinedReply={}", combinedReply); + return combinedReply; } void stepDownOnJvmPause() { @@ -930,6 +941,19 @@ private CompletableFuture replyFuture(ReferenceCountedObject writeAsync(ReferenceCountedObject requestRef) { + final RaftClientRequest request = requestRef.get(); + final CompletableFuture future = writeAsyncImpl(requestRef); + if (request.is(TypeCase.WRITE)) { + // check replication + final ReplicationLevel replication = request.getType().getWrite().getReplication(); + if (replication != ReplicationLevel.MAJORITY) { + return future.thenCompose(r -> waitForReplication(r, replication)); + } + } + return future; + } + + private CompletableFuture writeAsyncImpl(ReferenceCountedObject requestRef) { final RaftClientRequest request = requestRef.get(); final CompletableFuture reply = checkLeaderState(request); if (reply != null) { diff --git a/ratis-server/src/test/java/org/apache/ratis/RaftAsyncTests.java b/ratis-server/src/test/java/org/apache/ratis/RaftAsyncTests.java index 1ac704f595..925b8bbadb 100644 --- a/ratis-server/src/test/java/org/apache/ratis/RaftAsyncTests.java +++ b/ratis-server/src/test/java/org/apache/ratis/RaftAsyncTests.java @@ -24,6 +24,7 @@ import org.apache.ratis.conf.RaftProperties; import org.apache.ratis.proto.RaftProtos.CommitInfoProto; import org.apache.ratis.proto.RaftProtos.LogEntryProto; +import org.apache.ratis.proto.RaftProtos.ReplicationLevel; import org.apache.ratis.protocol.RaftGroupId; import org.apache.ratis.protocol.RaftPeer; import org.apache.ratis.protocol.exceptions.AlreadyClosedException; @@ -357,6 +358,38 @@ void runTestStaleReadAsync(CLUSTER cluster) throws Exception { } } + @Test + public void testWriteAsyncCustomReplicationLevel() throws Exception { + // verify that send(msg, ALL_COMMITTED) would reply with all servers committed past the log index + runWithNewCluster(NUM_SERVERS, this::runTestWriteAsyncCustomReplicationLevel); + } + + void runTestWriteAsyncCustomReplicationLevel(CLUSTER cluster) throws Exception { + final int numMessages = 20; + try (RaftClient client = cluster.createClient()) { + RaftTestUtil.waitForLeader(cluster); + + // submit some messages + for (int i = 0; i < numMessages; i++) { + final String s = "" + i; + LOG.info("sendAsync with ALL_COMMITTED " + s); + client.async().send(new SimpleMessage(s), ReplicationLevel.ALL_COMMITTED).whenComplete((reply, exception) -> { + if (exception != null) { + LOG.error("Failed to send message " + s, exception); + // reply should be null in case of exception + Assert.assertNull(reply); + return; + } + Assert.assertTrue(reply.isSuccess()); + Assert.assertNull(reply.getException()); + // verify that all servers have caught up to log index when the reply is returned + reply.getCommitInfos().forEach(commitInfoProto -> + Assert.assertTrue(commitInfoProto.getCommitIndex() >= reply.getLogIndex())); + }); + } + } + } + @Test public void testRequestTimeout() throws Exception { final TimeDuration oldExpiryTime = RaftServerConfigKeys.RetryCache.expiryTime(getProperties()); From 7e6c8991a11a672e7916c3fafb7ba804ae655b0a Mon Sep 17 00:00:00 2001 From: William Song <48054931+SzyWilliam@users.noreply.github.com> Date: Thu, 30 May 2024 15:44:21 +0800 Subject: [PATCH 055/397] RATIS-2084. Follower reply ALREADY_INSTALLED when install old snapshots from leader (#1091) --- .../ratis/server/impl/RaftServerImpl.java | 1 + .../apache/ratis/server/impl/ServerState.java | 4 ++ .../impl/SnapshotInstallationHandler.java | 8 ++- .../ratis/InstallSnapshotFromLeaderTests.java | 70 +++++++++++++++++++ .../java/org/apache/ratis/RaftTestUtil.java | 14 ++++ .../server/impl/LeaderElectionTests.java | 38 ++++------ .../ratis/grpc/TestLeaderInstallSnapshot.java | 6 ++ 7 files changed, 112 insertions(+), 29 deletions(-) diff --git a/ratis-server/src/main/java/org/apache/ratis/server/impl/RaftServerImpl.java b/ratis-server/src/main/java/org/apache/ratis/server/impl/RaftServerImpl.java index e049fdddf7..c1a716bd0c 100644 --- a/ratis-server/src/main/java/org/apache/ratis/server/impl/RaftServerImpl.java +++ b/ratis-server/src/main/java/org/apache/ratis/server/impl/RaftServerImpl.java @@ -1857,6 +1857,7 @@ CompletableFuture applyLogToStateMachine(LogEntryProto next) throws Raf * @param logEntry the log entry being truncated */ void notifyTruncatedLogEntry(LogEntryProto logEntry) { + Optional.ofNullable(getState()).ifPresent(s -> s.truncate(logEntry.getIndex())); if (logEntry.hasStateMachineLogEntry()) { getTransactionManager().remove(TermIndex.valueOf(logEntry)); diff --git a/ratis-server/src/main/java/org/apache/ratis/server/impl/ServerState.java b/ratis-server/src/main/java/org/apache/ratis/server/impl/ServerState.java index be74567fc0..d183930b9d 100644 --- a/ratis-server/src/main/java/org/apache/ratis/server/impl/ServerState.java +++ b/ratis-server/src/main/java/org/apache/ratis/server/impl/ServerState.java @@ -387,6 +387,10 @@ void setRaftConf(RaftConfiguration conf) { LOG.trace("{}: {}", getMemberId(), configurationManager); } + void truncate(long logIndex) { + configurationManager.removeConfigurations(logIndex); + } + void updateConfiguration(List entries) { if (entries != null && !entries.isEmpty()) { configurationManager.removeConfigurations(entries.get(0).getIndex()); diff --git a/ratis-server/src/main/java/org/apache/ratis/server/impl/SnapshotInstallationHandler.java b/ratis-server/src/main/java/org/apache/ratis/server/impl/SnapshotInstallationHandler.java index f03e2d883d..4a63e64ee0 100644 --- a/ratis-server/src/main/java/org/apache/ratis/server/impl/SnapshotInstallationHandler.java +++ b/ratis-server/src/main/java/org/apache/ratis/server/impl/SnapshotInstallationHandler.java @@ -133,6 +133,7 @@ private InstallSnapshotReplyProto installSnapshotImpl(InstallSnapshotRequestProt if (request.hasLastRaftConfigurationLogEntryProto()) { // Set the configuration included in the snapshot final LogEntryProto proto = request.getLastRaftConfigurationLogEntryProto(); + state.truncate(proto.getIndex()); if (!state.getRaftConf().equals(LogProtoUtils.toRaftConfiguration(proto))) { LOG.info("{}: set new configuration {} from snapshot", getMemberId(), proto); state.setRaftConf(proto); @@ -175,9 +176,10 @@ private InstallSnapshotReplyProto checkAndInstallSnapshot(InstallSnapshotRequest // Check and append the snapshot chunk. We simply put this in lock // considering a follower peer requiring a snapshot installation does not // have a lot of requests - Preconditions.assertTrue(state.getLog().getLastCommittedIndex() < lastIncludedIndex, - "%s log's commit index is %s, last included index in snapshot is %s", - getMemberId(), state.getLog().getLastCommittedIndex(), lastIncludedIndex); + if (state.getLog().getLastCommittedIndex() >= lastIncludedIndex) { + return toInstallSnapshotReplyProto(leaderId, getMemberId(), + currentTerm, snapshotChunkRequest.getRequestIndex(), InstallSnapshotResult.ALREADY_INSTALLED); + } //TODO: We should only update State with installed snapshot once the request is done. state.installSnapshot(request); diff --git a/ratis-server/src/test/java/org/apache/ratis/InstallSnapshotFromLeaderTests.java b/ratis-server/src/test/java/org/apache/ratis/InstallSnapshotFromLeaderTests.java index 46cfebbd17..b83a7dfdd3 100644 --- a/ratis-server/src/test/java/org/apache/ratis/InstallSnapshotFromLeaderTests.java +++ b/ratis-server/src/test/java/org/apache/ratis/InstallSnapshotFromLeaderTests.java @@ -21,7 +21,10 @@ import org.apache.ratis.conf.RaftProperties; import org.apache.ratis.protocol.RaftClientReply; import org.apache.ratis.protocol.RaftGroupId; +import org.apache.ratis.protocol.RaftPeer; import org.apache.ratis.protocol.RaftPeerId; +import org.apache.ratis.protocol.exceptions.RaftRetryFailureException; +import org.apache.ratis.retry.RetryPolicies; import org.apache.ratis.server.RaftServer; import org.apache.ratis.server.RaftServerConfigKeys; import org.apache.ratis.server.impl.MiniRaftCluster; @@ -85,6 +88,12 @@ public void testSeparateSnapshotInstallPath() throws Exception { runWithNewCluster(1, this::testMultiFileInstallSnapshot); } + public void testInstallSnapshotLeaderSwitch() throws Exception { + getProperties().setClass(MiniRaftCluster.STATEMACHINE_CLASS_KEY, + StateMachineWithSeparatedSnapshotPath.class, StateMachine.class); + runWithNewCluster(3, this::testInstallSnapshotDuringLeaderSwitch); + } + private void testMultiFileInstallSnapshot(CLUSTER cluster) throws Exception { try { int i = 0; @@ -127,6 +136,67 @@ private void testMultiFileInstallSnapshot(CLUSTER cluster) throws Exception { } } + private void testInstallSnapshotDuringLeaderSwitch(CLUSTER cluster) throws Exception { + try { + RaftTestUtil.waitForLeader(cluster); + final RaftPeerId leaderId = cluster.getLeader().getId(); + + // perform operations and force all peers to take snapshot + try (final RaftClient client = cluster.createClient(leaderId)) { + for (int i = 0; i < SNAPSHOT_TRIGGER_THRESHOLD * 2; i++) { + final RaftClientReply + reply = client.io().send(new RaftTestUtil.SimpleMessage("m" + i)); + Assertions.assertTrue(reply.isSuccess()); + } + + for (final RaftPeer peer: cluster.getPeers()) { + final RaftClientReply snapshotReply = client.getSnapshotManagementApi(leaderId).create(3000); + Assertions.assertTrue(snapshotReply.isSuccess()); + } + } + final SnapshotInfo snapshot = cluster.getLeader().getStateMachine().getLatestSnapshot(); + Assertions.assertNotNull(snapshot); + + // isolate two followers (majority) in old configuration + final List oldFollowers = cluster.getFollowers(); + for (RaftServer.Division f: oldFollowers) { + RaftTestUtil.isolate(cluster, f.getId()); + } + + // add two more peers and install snapshot from leaders + final MiniRaftCluster.PeerChanges change = cluster.addNewPeers(2, true, + true); + try (final RaftClient client = cluster.createClient(leaderId, RetryPolicies.noRetry())) { + Assertions.assertThrows(RaftRetryFailureException.class, + () -> client.admin().setConfiguration(change.allPeersInNewConf)); + } + + final SnapshotInfo snapshotInfo = cluster.getDivision(change.newPeers[0].getId()) + .getStateMachine().getLatestSnapshot(); + Assertions.assertNotNull(snapshotInfo); + + // recover the old followers and isolate the leader to force leader switch + RaftTestUtil.isolate(cluster, leaderId); + for (RaftServer.Division f: oldFollowers) { + RaftTestUtil.deIsolate(cluster, f.getId()); + } + RaftTestUtil.waitForLeader(cluster); + + try (final RaftClient client = cluster.createClient(cluster.getLeader().getId())) { + // successfully setConfiguration during leader switch + final RaftClientReply setConf = client.admin().setConfiguration(change.allPeersInNewConf); + Assertions.assertTrue(setConf.isSuccess()); + + RaftTestUtil.deIsolate(cluster, leaderId); + final RaftClientReply + reply = client.io().send(new RaftTestUtil.SimpleMessage("final")); + Assertions.assertTrue(reply.isSuccess()); + } + } finally { + cluster.shutdown(); + } + } + private static class StateMachineWithMultiNestedSnapshotFile extends SimpleStateMachine4Testing { File snapshotRoot; diff --git a/ratis-server/src/test/java/org/apache/ratis/RaftTestUtil.java b/ratis-server/src/test/java/org/apache/ratis/RaftTestUtil.java index 12a7537542..d94d3a0326 100644 --- a/ratis-server/src/test/java/org/apache/ratis/RaftTestUtil.java +++ b/ratis-server/src/test/java/org/apache/ratis/RaftTestUtil.java @@ -512,6 +512,20 @@ static void blockQueueAndSetDelay(Iterable servers, Thread.sleep(3 * maxTimeout.toLong(TimeUnit.MILLISECONDS)); } + static void isolate(MiniRaftCluster cluster, RaftPeerId id) { + try { + BlockRequestHandlingInjection.getInstance().blockReplier(id.toString()); + cluster.setBlockRequestsFrom(id.toString(), true); + } catch (Exception e) { + e.printStackTrace(); + } + } + + static void deIsolate(MiniRaftCluster cluster, RaftPeerId id) { + BlockRequestHandlingInjection.getInstance().unblockReplier(id.toString()); + cluster.setBlockRequestsFrom(id.toString(), false); + } + static Thread sendMessageInNewThread(MiniRaftCluster cluster, RaftPeerId leaderId, SimpleMessage... messages) { Thread t = new Thread(() -> { try (final RaftClient client = cluster.createClient(leaderId)) { diff --git a/ratis-server/src/test/java/org/apache/ratis/server/impl/LeaderElectionTests.java b/ratis-server/src/test/java/org/apache/ratis/server/impl/LeaderElectionTests.java index e3af1546ec..b29b537abe 100644 --- a/ratis-server/src/test/java/org/apache/ratis/server/impl/LeaderElectionTests.java +++ b/ratis-server/src/test/java/org/apache/ratis/server/impl/LeaderElectionTests.java @@ -122,12 +122,12 @@ void runTestLostMajorityHeartbeats(CLUSTER cluster) throws Exception { final TimeDuration maxTimeout = RaftServerConfigKeys.Rpc.timeoutMax(getProperties()); final RaftServer.Division leader = waitForLeader(cluster); try { - isolate(cluster, leader.getId()); + RaftTestUtil.isolate(cluster, leader.getId()); maxTimeout.sleep(); maxTimeout.sleep(); RaftServerTestUtil.assertLostMajorityHeartbeatsRecently(leader); } finally { - deIsolate(cluster, leader.getId()); + RaftTestUtil.deIsolate(cluster, leader.getId()); } } @@ -164,12 +164,12 @@ void runTestListenerNotStartLeaderElection(CLUSTER cluster) throws Exception { final RaftServer.Division listener = cluster.getListeners().get(0); final RaftPeerId listenerId = listener.getId(); try { - isolate(cluster, listenerId); + RaftTestUtil.isolate(cluster, listenerId); maxTimeout.sleep(); maxTimeout.sleep(); Assertions.assertEquals(RaftProtos.RaftPeerRole.LISTENER, listener.getInfo().getCurrentRole()); } finally { - deIsolate(cluster, listener.getId()); + RaftTestUtil.deIsolate(cluster, listener.getId()); } } @@ -247,7 +247,7 @@ public void testTransferLeaderTimeout() throws Exception { RaftServer.Division newLeader = followers.get(0); // isolate new leader, so that transfer leadership will timeout - isolate(cluster, newLeader.getId()); + RaftTestUtil.isolate(cluster, newLeader.getId()); List peers = cluster.getPeers(); @@ -287,7 +287,7 @@ public void testTransferLeaderTimeout() throws Exception { Assertions.assertEquals(leader.getId().toString(), reply.getReplierId()); Assertions.assertTrue(reply.isSuccess()); - deIsolate(cluster, newLeader.getId()); + RaftTestUtil.deIsolate(cluster, newLeader.getId()); } cluster.shutdown(); @@ -364,32 +364,18 @@ protected void testDisconnectLeader() throws Exception { try (RaftClient client = cluster.createClient(leader.getId())) { client.io().send(new RaftTestUtil.SimpleMessage("message")); Thread.sleep(1000); - isolate(cluster, leader.getId()); + RaftTestUtil.isolate(cluster, leader.getId()); RaftClientReply reply = client.io().send(new RaftTestUtil.SimpleMessage("message")); Assertions.assertNotEquals(reply.getReplierId(), leader.getId().toString()); Assertions.assertTrue(reply.isSuccess()); } finally { - deIsolate(cluster, leader.getId()); + RaftTestUtil.deIsolate(cluster, leader.getId()); } cluster.shutdown(); } } - private void isolate(MiniRaftCluster cluster, RaftPeerId id) { - try { - BlockRequestHandlingInjection.getInstance().blockReplier(id.toString()); - cluster.setBlockRequestsFrom(id.toString(), true); - } catch (Exception e) { - e.printStackTrace(); - } - } - - private void deIsolate(MiniRaftCluster cluster, RaftPeerId id) { - BlockRequestHandlingInjection.getInstance().unblockReplier(id.toString()); - cluster.setBlockRequestsFrom(id.toString(), false); - } - @Test public void testAddListener() throws Exception { try (final MiniRaftCluster cluster = newCluster(3)) { @@ -570,7 +556,7 @@ public void testPreVote() { assertEquals(followers.size(), 2); RaftServer.Division follower = followers.get(0); - isolate(cluster, follower.getId()); + RaftTestUtil.isolate(cluster, follower.getId()); // send message so that the isolated follower's log lag the others RaftClientReply reply = client.io().send(new RaftTestUtil.SimpleMessage("message")); Assertions.assertTrue(reply.isSuccess()); @@ -578,7 +564,7 @@ public void testPreVote() { final long savedTerm = leader.getInfo().getCurrentTerm(); LOG.info("Wait follower {} timeout and trigger pre-vote", follower.getId()); Thread.sleep(2000); - deIsolate(cluster, follower.getId()); + RaftTestUtil.deIsolate(cluster, follower.getId()); Thread.sleep(2000); // with pre-vote leader will not step down RaftServer.Division newleader = waitForLeader(cluster); @@ -669,14 +655,14 @@ void runTestLeaderLease(CLUSTER cluster, long leaseTimeoutMs) throws Exception { Assertions.assertTrue(leader.getInfo().isLeaderReady()); RaftServerTestUtil.assertLeaderLease(leader, true); - isolate(cluster, leader.getId()); + RaftTestUtil.isolate(cluster, leader.getId()); Thread.sleep(leaseTimeoutMs); Assertions.assertTrue(leader.getInfo().isLeader()); Assertions.assertTrue(leader.getInfo().isLeaderReady()); RaftServerTestUtil.assertLeaderLease(leader, false); } finally { - deIsolate(cluster, leader.getId()); + RaftTestUtil.deIsolate(cluster, leader.getId()); } } diff --git a/ratis-test/src/test/java/org/apache/ratis/grpc/TestLeaderInstallSnapshot.java b/ratis-test/src/test/java/org/apache/ratis/grpc/TestLeaderInstallSnapshot.java index 22c590c9dd..b85cd13535 100644 --- a/ratis-test/src/test/java/org/apache/ratis/grpc/TestLeaderInstallSnapshot.java +++ b/ratis-test/src/test/java/org/apache/ratis/grpc/TestLeaderInstallSnapshot.java @@ -46,4 +46,10 @@ public void testSeparateSnapshotInstallPath(Boolean separateHeartbeat) throws Ex super.testSeparateSnapshotInstallPath(); } + @ParameterizedTest + @MethodSource("data") + public void testInstallSnapshotLeaderSwitch(Boolean separateHeartbeat) throws Exception { + GrpcConfigKeys.Server.setHeartbeatChannel(getProperties(), separateHeartbeat); + super.testInstallSnapshotLeaderSwitch(); + } } From eac5ec7f109e8ac54477a875e393b2f4b5bf36ac Mon Sep 17 00:00:00 2001 From: Siyao Meng <50227127+smengcl@users.noreply.github.com> Date: Tue, 4 Jun 2024 19:57:39 -0700 Subject: [PATCH 056/397] RATIS-2106. Add configuration reference for RaftClient (#1107) --- .../ratis/client/RaftClientConfigKeys.java | 7 +-- .../src/site/markdown/configurations.md | 63 ++++++++++++++++++- 2 files changed, 65 insertions(+), 5 deletions(-) diff --git a/ratis-client/src/main/java/org/apache/ratis/client/RaftClientConfigKeys.java b/ratis-client/src/main/java/org/apache/ratis/client/RaftClientConfigKeys.java index 7360a9cadb..925324c21c 100644 --- a/ratis-client/src/main/java/org/apache/ratis/client/RaftClientConfigKeys.java +++ b/ratis-client/src/main/java/org/apache/ratis/client/RaftClientConfigKeys.java @@ -42,7 +42,7 @@ interface Rpc { String PREFIX = RaftClientConfigKeys.PREFIX + ".rpc"; String REQUEST_TIMEOUT_KEY = PREFIX + ".request.timeout"; - TimeDuration REQUEST_TIMEOUT_DEFAULT = TimeDuration.valueOf(3000, TimeUnit.MILLISECONDS); + TimeDuration REQUEST_TIMEOUT_DEFAULT = TimeDuration.valueOf(3, TimeUnit.SECONDS); static TimeDuration requestTimeout(RaftProperties properties) { return getTimeDuration(properties.getTimeDuration(REQUEST_TIMEOUT_DEFAULT.getUnit()), REQUEST_TIMEOUT_KEY, REQUEST_TIMEOUT_DEFAULT, getDefaultLog()); @@ -52,8 +52,7 @@ static void setRequestTimeout(RaftProperties properties, TimeDuration timeoutDur } String WATCH_REQUEST_TIMEOUT_KEY = PREFIX + ".watch.request.timeout"; - TimeDuration WATCH_REQUEST_TIMEOUT_DEFAULT = - TimeDuration.valueOf(10000, TimeUnit.MILLISECONDS); + TimeDuration WATCH_REQUEST_TIMEOUT_DEFAULT = TimeDuration.valueOf(10, TimeUnit.SECONDS); static TimeDuration watchRequestTimeout(RaftProperties properties) { return getTimeDuration(properties.getTimeDuration(WATCH_REQUEST_TIMEOUT_DEFAULT.getUnit()), WATCH_REQUEST_TIMEOUT_KEY, WATCH_REQUEST_TIMEOUT_DEFAULT, getDefaultLog()); @@ -125,7 +124,7 @@ static void setFlushRequestBytesMin(RaftProperties properties, SizeInBytes flush } String REQUEST_TIMEOUT_KEY = PREFIX + ".request.timeout"; - TimeDuration REQUEST_TIMEOUT_DEFAULT = TimeDuration.valueOf(10000, TimeUnit.MILLISECONDS); + TimeDuration REQUEST_TIMEOUT_DEFAULT = TimeDuration.valueOf(10, TimeUnit.SECONDS); static TimeDuration requestTimeout(RaftProperties properties) { return getTimeDuration(properties.getTimeDuration(REQUEST_TIMEOUT_DEFAULT.getUnit()), REQUEST_TIMEOUT_KEY, REQUEST_TIMEOUT_DEFAULT, getDefaultLog()); diff --git a/ratis-docs/src/site/markdown/configurations.md b/ratis-docs/src/site/markdown/configurations.md index 95505fd549..71eae7d3d1 100644 --- a/ratis-docs/src/site/markdown/configurations.md +++ b/ratis-docs/src/site/markdown/configurations.md @@ -675,4 +675,65 @@ For examples, 2. However, setConf to a 3-member group by removing 2 of members and adding 2 new members is a majority-add. - Note also that adding 1 new member to an 1-member group is always allowed, - although it is a majority-add. \ No newline at end of file + although it is a majority-add. + + +## Client Configurations + +Client configurations are located at `RaftClientConfigKeys`. + +### RPC - Configurations related to Client RPC timeout. + +| **Property** | `raft.client.rpc.request.timeout` | +|:----------------|:------------------------------------------| +| **Description** | client side timeout for sending a request | +| **Type** | TimeDuration | +| **Default** | 3s | + +| **Property** | `raft.client.rpc.watch.request.timeout` | +|:----------------|:------------------------------------------------| +| **Description** | client side timeout for sending a watch request | +| **Type** | TimeDuration | +| **Default** | 10s | + +### Async - Configurations related to async requests. + +| **Property** | `raft.client.async.outstanding-requests.max` | +|:----------------|:---------------------------------------------| +| **Description** | maximum number of outstanding async requests | +| **Type** | int | +| **Default** | 100 | + +### DataStream - Configurations related to DataStream Api. + +| **Property** | `raft.client.data-stream.outstanding-requests.max` | +|:----------------|:---------------------------------------------------| +| **Description** | maximum number of outstanding data stream requests | +| **Type** | int | +| **Default** | 100 | + +| **Property** | `raft.client.data-stream.flush.request.count.min` | +|:----------------|:-----------------------------------------------------------------| +| **Description** | minimum number of requests before data stream flush would happen | +| **Type** | int | +| **Default** | 0 | + +| **Property** | `raft.client.data-stream.flush.request.bytes.min` | +|:----------------|:--------------------------------------------------------------| +| **Description** | minimum number of bytes before data stream flush would happen | +| **Type** | SizeInBytes | +| **Default** | 1MB | + +| **Property** | `raft.client.data-stream.request.timeout` | +|:----------------|:------------------------------------------| +| **Description** | timeout for data stream request | +| **Type** | TimeDuration | +| **Default** | 10s | + +### MessageStream - Configurations related to MessageStream Api. + +| **Property** | `raft.client.message-stream.submessage-size` | +|:----------------|:---------------------------------------------| +| **Description** | maximum size of a sub message | +| **Type** | SizeInBytes | +| **Default** | 1MB | From 029cf5206fc1de4ea8a9d0d548f5fc8fc57251e4 Mon Sep 17 00:00:00 2001 From: Ivan Andika Date: Wed, 5 Jun 2024 20:01:42 +0800 Subject: [PATCH 057/397] RATIS-2109. RaftLogBase#updateCommitIndex should return true only if commitIndex is increased (#1109) --- .../java/org/apache/ratis/server/raftlog/RaftLogBase.java | 6 ++---- 1 file changed, 2 insertions(+), 4 deletions(-) diff --git a/ratis-server/src/main/java/org/apache/ratis/server/raftlog/RaftLogBase.java b/ratis-server/src/main/java/org/apache/ratis/server/raftlog/RaftLogBase.java index 1ad108aefe..14851b1a4e 100644 --- a/ratis-server/src/main/java/org/apache/ratis/server/raftlog/RaftLogBase.java +++ b/ratis-server/src/main/java/org/apache/ratis/server/raftlog/RaftLogBase.java @@ -124,15 +124,13 @@ public boolean updateCommitIndex(long majorityIndex, long currentTerm, boolean i final long newCommitIndex = Math.min(majorityIndex, getFlushIndex()); if (oldCommittedIndex < newCommitIndex) { if (!isLeader) { - commitIndex.updateIncreasingly(newCommitIndex, traceIndexChange); - return true; + return commitIndex.updateIncreasingly(newCommitIndex, traceIndexChange); } // Only update last committed index for current term. See §5.4.2 in paper for details. final TermIndex entry = getTermIndex(newCommitIndex); if (entry != null && entry.getTerm() == currentTerm) { - commitIndex.updateIncreasingly(newCommitIndex, traceIndexChange); - return true; + return commitIndex.updateIncreasingly(newCommitIndex, traceIndexChange); } } } From 9ae65d751e8b24e2e4e67ac80bf6566120262e50 Mon Sep 17 00:00:00 2001 From: "Doroszlai, Attila" <6454655+adoroszlai@users.noreply.github.com> Date: Thu, 13 Jun 2024 08:26:25 +0200 Subject: [PATCH 058/397] RATIS-2112. Improve repeat-test workflow (#1112) --- .github/workflows/repeat-test.yml | 6 +++--- dev-support/checks/unit.sh | 6 ++++++ 2 files changed, 9 insertions(+), 3 deletions(-) diff --git a/.github/workflows/repeat-test.yml b/.github/workflows/repeat-test.yml index e3c05bec6a..f431b1a0fe 100644 --- a/.github/workflows/repeat-test.yml +++ b/.github/workflows/repeat-test.yml @@ -114,12 +114,12 @@ jobs: if: ${{ !cancelled() }} - name: Archive build results uses: actions/upload-artifact@v4 - if: always() + if: ${{ failure() }} with: - name: result-${{ env.TEST_CLASS }}-split-${{ matrix.split }} + name: result-${{ github.run_number }}-${{ github.run_id }}-split-${{ matrix.split }} path: target/unit count-failures: - if: ${{ always() }} + if: ${{ failure() }} needs: test runs-on: ubuntu-20.04 steps: diff --git a/dev-support/checks/unit.sh b/dev-support/checks/unit.sh index f7a4f3017e..389c5c5275 100755 --- a/dev-support/checks/unit.sh +++ b/dev-support/checks/unit.sh @@ -65,6 +65,12 @@ for i in $(seq 1 ${ITERATIONS}); do fi if [[ ${ITERATIONS} -gt 1 ]]; then + if ! grep -q "Running .*Test" "${REPORT_DIR}/output.log"; then + echo "No tests were run" >> "${REPORT_DIR}/summary.txt" + irc=1 + FAIL_FAST=true + fi + if [[ ${irc} == 0 ]]; then rm -fr "${REPORT_DIR}" fi From 0f1f99843375e45b142e460318434be0759cb7df Mon Sep 17 00:00:00 2001 From: Angerszhuuuu Date: Thu, 13 Jun 2024 17:15:56 +0800 Subject: [PATCH 059/397] RATIS-2111 Reinitialize should load the latest snapshot (#1111) --- .../ratis/examples/arithmetic/ArithmeticStateMachine.java | 2 +- .../ratis/examples/counter/server/CounterStateMachine.java | 2 +- .../ratis/statemachine/impl/SimpleStateMachineStorage.java | 4 ++++ 3 files changed, 6 insertions(+), 2 deletions(-) diff --git a/ratis-examples/src/main/java/org/apache/ratis/examples/arithmetic/ArithmeticStateMachine.java b/ratis-examples/src/main/java/org/apache/ratis/examples/arithmetic/ArithmeticStateMachine.java index 28e3fb1c7d..fa0dc6d8e2 100644 --- a/ratis-examples/src/main/java/org/apache/ratis/examples/arithmetic/ArithmeticStateMachine.java +++ b/ratis-examples/src/main/java/org/apache/ratis/examples/arithmetic/ArithmeticStateMachine.java @@ -81,7 +81,7 @@ public void initialize(RaftServer server, RaftGroupId groupId, @Override public void reinitialize() throws IOException { close(); - loadSnapshot(storage.getLatestSnapshot()); + loadSnapshot(storage.loadLatestSnapshot()); } @Override diff --git a/ratis-examples/src/main/java/org/apache/ratis/examples/counter/server/CounterStateMachine.java b/ratis-examples/src/main/java/org/apache/ratis/examples/counter/server/CounterStateMachine.java index b88a763e0f..b5352f0adc 100644 --- a/ratis-examples/src/main/java/org/apache/ratis/examples/counter/server/CounterStateMachine.java +++ b/ratis-examples/src/main/java/org/apache/ratis/examples/counter/server/CounterStateMachine.java @@ -138,7 +138,7 @@ public void initialize(RaftServer server, RaftGroupId groupId, RaftStorage raftS */ @Override public void reinitialize() throws IOException { - load(storage.getLatestSnapshot()); + load(storage.loadLatestSnapshot()); } /** diff --git a/ratis-server/src/main/java/org/apache/ratis/statemachine/impl/SimpleStateMachineStorage.java b/ratis-server/src/main/java/org/apache/ratis/statemachine/impl/SimpleStateMachineStorage.java index 88cc57dab5..7e8afbaa85 100644 --- a/ratis-server/src/main/java/org/apache/ratis/statemachine/impl/SimpleStateMachineStorage.java +++ b/ratis-server/src/main/java/org/apache/ratis/statemachine/impl/SimpleStateMachineStorage.java @@ -217,6 +217,10 @@ public SingleFileSnapshotInfo getLatestSnapshot() { if (s != null) { return s; } + return loadLatestSnapshot(); + } + + public SingleFileSnapshotInfo loadLatestSnapshot() { final File dir = stateMachineDir; if (dir == null) { return null; From e3519848e63c08daa55221742c7e3e5138393f96 Mon Sep 17 00:00:00 2001 From: Tsz-Wo Nicholas Sze Date: Thu, 22 Aug 2024 10:50:50 -0700 Subject: [PATCH 060/397] ./mvnw versions:set -DnewVersion=3.1.1-SNAPSHOT --- pom.xml | 4 ++-- ratis-assembly/pom.xml | 2 +- ratis-client/pom.xml | 2 +- ratis-common/pom.xml | 2 +- ratis-docs/pom.xml | 2 +- ratis-examples/pom.xml | 2 +- ratis-experiments/pom.xml | 2 +- ratis-grpc/pom.xml | 2 +- ratis-metrics-api/pom.xml | 2 +- ratis-metrics-default/pom.xml | 2 +- ratis-metrics-dropwizard3/pom.xml | 2 +- ratis-netty/pom.xml | 2 +- ratis-proto/pom.xml | 2 +- ratis-replicated-map/pom.xml | 2 +- ratis-resource-bundle/pom.xml | 2 +- ratis-server-api/pom.xml | 2 +- ratis-server/pom.xml | 2 +- ratis-shell/pom.xml | 2 +- ratis-test/pom.xml | 2 +- ratis-tools/pom.xml | 2 +- 20 files changed, 21 insertions(+), 21 deletions(-) diff --git a/pom.xml b/pom.xml index 8292a85e77..60c145506a 100644 --- a/pom.xml +++ b/pom.xml @@ -24,7 +24,7 @@ ratis org.apache.ratis - 3.1.0-SNAPSHOT + 3.1.1-SNAPSHOT Apache Ratis pom @@ -162,7 +162,7 @@ - 2023-11-19T14:23:30Z + 2024-08-22T17:50:05Z UTF-8 UTF-8 diff --git a/ratis-assembly/pom.xml b/ratis-assembly/pom.xml index 98826e142a..37fe569206 100644 --- a/ratis-assembly/pom.xml +++ b/ratis-assembly/pom.xml @@ -17,7 +17,7 @@ ratis org.apache.ratis - 3.1.0-SNAPSHOT + 3.1.1-SNAPSHOT ratis-assembly diff --git a/ratis-client/pom.xml b/ratis-client/pom.xml index d1b08d5a6a..3597fdddbe 100644 --- a/ratis-client/pom.xml +++ b/ratis-client/pom.xml @@ -17,7 +17,7 @@ ratis org.apache.ratis - 3.1.0-SNAPSHOT + 3.1.1-SNAPSHOT ratis-client diff --git a/ratis-common/pom.xml b/ratis-common/pom.xml index 9205e81c2c..066aa57ecf 100644 --- a/ratis-common/pom.xml +++ b/ratis-common/pom.xml @@ -17,7 +17,7 @@ ratis org.apache.ratis - 3.1.0-SNAPSHOT + 3.1.1-SNAPSHOT ratis-common diff --git a/ratis-docs/pom.xml b/ratis-docs/pom.xml index 47f1175a46..b44781db58 100644 --- a/ratis-docs/pom.xml +++ b/ratis-docs/pom.xml @@ -20,7 +20,7 @@ https://maven.apache.org/xsd/maven-4.0.0.xsd"> ratis org.apache.ratis - 3.1.0-SNAPSHOT + 3.1.1-SNAPSHOT ratis-docs diff --git a/ratis-examples/pom.xml b/ratis-examples/pom.xml index 7454e928eb..fddbdc94be 100644 --- a/ratis-examples/pom.xml +++ b/ratis-examples/pom.xml @@ -17,7 +17,7 @@ ratis org.apache.ratis - 3.1.0-SNAPSHOT + 3.1.1-SNAPSHOT ratis-examples diff --git a/ratis-experiments/pom.xml b/ratis-experiments/pom.xml index 2099142ea3..a8c43bf814 100644 --- a/ratis-experiments/pom.xml +++ b/ratis-experiments/pom.xml @@ -17,7 +17,7 @@ ratis org.apache.ratis - 3.1.0-SNAPSHOT + 3.1.1-SNAPSHOT ratis-experiments diff --git a/ratis-grpc/pom.xml b/ratis-grpc/pom.xml index 71377b5efc..aa3d76ff51 100644 --- a/ratis-grpc/pom.xml +++ b/ratis-grpc/pom.xml @@ -17,7 +17,7 @@ ratis org.apache.ratis - 3.1.0-SNAPSHOT + 3.1.1-SNAPSHOT ratis-grpc diff --git a/ratis-metrics-api/pom.xml b/ratis-metrics-api/pom.xml index e51a9f9632..4606f41d68 100644 --- a/ratis-metrics-api/pom.xml +++ b/ratis-metrics-api/pom.xml @@ -18,7 +18,7 @@ ratis org.apache.ratis - 3.1.0-SNAPSHOT + 3.1.1-SNAPSHOT ratis-metrics-api diff --git a/ratis-metrics-default/pom.xml b/ratis-metrics-default/pom.xml index 4fe8073922..f59b94fd7e 100644 --- a/ratis-metrics-default/pom.xml +++ b/ratis-metrics-default/pom.xml @@ -18,7 +18,7 @@ ratis org.apache.ratis - 3.1.0-SNAPSHOT + 3.1.1-SNAPSHOT ratis-metrics-default diff --git a/ratis-metrics-dropwizard3/pom.xml b/ratis-metrics-dropwizard3/pom.xml index 42fff0445b..c948c75797 100644 --- a/ratis-metrics-dropwizard3/pom.xml +++ b/ratis-metrics-dropwizard3/pom.xml @@ -18,7 +18,7 @@ ratis org.apache.ratis - 3.1.0-SNAPSHOT + 3.1.1-SNAPSHOT ratis-metrics-dropwizard3 diff --git a/ratis-netty/pom.xml b/ratis-netty/pom.xml index 1966db8ac4..ead5cb43d5 100644 --- a/ratis-netty/pom.xml +++ b/ratis-netty/pom.xml @@ -17,7 +17,7 @@ ratis org.apache.ratis - 3.1.0-SNAPSHOT + 3.1.1-SNAPSHOT ratis-netty diff --git a/ratis-proto/pom.xml b/ratis-proto/pom.xml index e0cf2c8587..33e8e6abed 100644 --- a/ratis-proto/pom.xml +++ b/ratis-proto/pom.xml @@ -17,7 +17,7 @@ ratis org.apache.ratis - 3.1.0-SNAPSHOT + 3.1.1-SNAPSHOT ratis-proto diff --git a/ratis-replicated-map/pom.xml b/ratis-replicated-map/pom.xml index 5b4b8f21c1..57ddfe3ec8 100644 --- a/ratis-replicated-map/pom.xml +++ b/ratis-replicated-map/pom.xml @@ -17,7 +17,7 @@ ratis org.apache.ratis - 3.1.0-SNAPSHOT + 3.1.1-SNAPSHOT ratis-replicated-map diff --git a/ratis-resource-bundle/pom.xml b/ratis-resource-bundle/pom.xml index a879a74238..64c6a284b2 100644 --- a/ratis-resource-bundle/pom.xml +++ b/ratis-resource-bundle/pom.xml @@ -23,7 +23,7 @@ ratis org.apache.ratis - 3.1.0-SNAPSHOT + 3.1.1-SNAPSHOT .. diff --git a/ratis-server-api/pom.xml b/ratis-server-api/pom.xml index f3c4b617a0..672175ec1b 100644 --- a/ratis-server-api/pom.xml +++ b/ratis-server-api/pom.xml @@ -17,7 +17,7 @@ ratis org.apache.ratis - 3.1.0-SNAPSHOT + 3.1.1-SNAPSHOT ratis-server-api diff --git a/ratis-server/pom.xml b/ratis-server/pom.xml index 38af72d9db..b3abd5a9e9 100644 --- a/ratis-server/pom.xml +++ b/ratis-server/pom.xml @@ -17,7 +17,7 @@ ratis org.apache.ratis - 3.1.0-SNAPSHOT + 3.1.1-SNAPSHOT ratis-server diff --git a/ratis-shell/pom.xml b/ratis-shell/pom.xml index 78ab6679fd..a9304ee715 100644 --- a/ratis-shell/pom.xml +++ b/ratis-shell/pom.xml @@ -17,7 +17,7 @@ ratis org.apache.ratis - 3.1.0-SNAPSHOT + 3.1.1-SNAPSHOT ratis-shell diff --git a/ratis-test/pom.xml b/ratis-test/pom.xml index 01d54ba15b..f00ba94393 100644 --- a/ratis-test/pom.xml +++ b/ratis-test/pom.xml @@ -17,7 +17,7 @@ ratis org.apache.ratis - 3.1.0-SNAPSHOT + 3.1.1-SNAPSHOT ratis-test diff --git a/ratis-tools/pom.xml b/ratis-tools/pom.xml index c1971ced96..fd13c14fc6 100644 --- a/ratis-tools/pom.xml +++ b/ratis-tools/pom.xml @@ -17,7 +17,7 @@ ratis org.apache.ratis - 3.1.0-SNAPSHOT + 3.1.1-SNAPSHOT ratis-tools From edc419325cead97cb902b0229d67e338cbb2a56d Mon Sep 17 00:00:00 2001 From: DaveTeng0 <109315747+DaveTeng0@users.noreply.github.com> Date: Fri, 14 Jun 2024 07:20:03 -0700 Subject: [PATCH 061/397] RATIS-2095. Extract common logic of ratis-shell to RaftUtils for reuse (#1098) --- .../org/apache/ratis/shell/cli/RaftUtils.java | 124 +++++++++++++++++- .../cli/sh/command/AbstractRatisCommand.java | 98 ++++---------- 2 files changed, 145 insertions(+), 77 deletions(-) diff --git a/ratis-shell/src/main/java/org/apache/ratis/shell/cli/RaftUtils.java b/ratis-shell/src/main/java/org/apache/ratis/shell/cli/RaftUtils.java index 1239fc56c4..9c5d90e4bf 100644 --- a/ratis-shell/src/main/java/org/apache/ratis/shell/cli/RaftUtils.java +++ b/ratis-shell/src/main/java/org/apache/ratis/shell/cli/RaftUtils.java @@ -20,20 +20,38 @@ import org.apache.ratis.client.RaftClient; import org.apache.ratis.client.RaftClientConfigKeys; import org.apache.ratis.conf.RaftProperties; +import org.apache.ratis.protocol.GroupInfoReply; +import org.apache.ratis.protocol.RaftClientReply; import org.apache.ratis.protocol.RaftGroup; +import org.apache.ratis.protocol.RaftGroupId; +import org.apache.ratis.protocol.RaftPeer; import org.apache.ratis.protocol.RaftPeerId; +import org.apache.ratis.protocol.exceptions.RaftException; import org.apache.ratis.retry.ExponentialBackoffRetry; import org.apache.ratis.util.TimeDuration; +import org.apache.ratis.util.function.CheckedFunction; +import java.io.IOException; +import java.io.PrintStream; import java.net.InetSocketAddress; -import java.util.Properties; +import java.util.ArrayList; +import java.util.Collection; import java.util.concurrent.TimeUnit; +import java.util.function.Consumer; +import java.util.List; +import java.util.Optional; +import java.util.Properties; +import java.util.function.Supplier; +import java.util.stream.Collectors; +import java.util.UUID; /** * Helper class for raft operations. */ public final class RaftUtils { + public static final RaftGroupId DEFAULT_RAFT_GROUP_ID = RaftGroupId.randomId(); + private RaftUtils() { // prevent instantiation } @@ -86,4 +104,108 @@ public static RaftClient createClient(RaftGroup raftGroup) { .setRetryPolicy(retryPolicy) .build(); } + + /** + * Apply the given function to the given parameter a list. + * + * @param list the input parameter list + * @param function the function to be applied + * @param parameter type + * @param return value type + * @param the exception type thrown by the given function. + * @return the first non-null value returned by the given function applied to the given list. + */ + private static RETURN applyFunctionReturnFirstNonNull( + Collection list, CheckedFunction function) { + for (PARAMETER parameter : list) { + try { + RETURN ret = function.apply(parameter); + if (ret != null) { + return ret; + } + } catch (Throwable e) { + e.printStackTrace(); + } + } + return null; + } + + public static List buildRaftPeersFromStr(String peers) { + List addresses = new ArrayList<>(); + String[] peersArray = peers.split(","); + for (String peer : peersArray) { + addresses.add(parseInetSocketAddress(peer)); + } + + return addresses.stream() + .map(addr -> RaftPeer.newBuilder() + .setId(RaftUtils.getPeerId(addr)) + .setAddress(addr) + .build() + ).collect(Collectors.toList()); + } + + public static RaftGroupId buildRaftGroupIdFromStr(String groupId) { + return groupId != null && groupId.isEmpty() ? RaftGroupId.valueOf(UUID.fromString(groupId)) + : DEFAULT_RAFT_GROUP_ID; + } + + public static RaftGroupId retrieveRemoteGroupId(RaftGroupId raftGroupIdFromConfig, + List peers, + RaftClient client, PrintStream printStream) throws IOException { + if (!DEFAULT_RAFT_GROUP_ID .equals(raftGroupIdFromConfig)) { + return raftGroupIdFromConfig; + } + + final RaftGroupId remoteGroupId; + final List groupIds = applyFunctionReturnFirstNonNull(peers, + p -> client.getGroupManagementApi((p.getId())).list().getGroupIds()); + + if (groupIds == null) { + printStream.println("Failed to get group ID from " + peers); + throw new IOException("Failed to get group ID from " + peers); + } else if (groupIds.size() == 1) { + remoteGroupId = groupIds.get(0); + } else { + String message = "Unexpected multiple group IDs " + groupIds + + ". In such case, the target group ID must be specified."; + printStream.println(message); + throw new IOException(message); + } + return remoteGroupId; + } + + public static GroupInfoReply retrieveGroupInfoByGroupId(RaftGroupId remoteGroupId, List peers, + RaftClient client, PrintStream printStream) + throws IOException { + GroupInfoReply groupInfoReply = applyFunctionReturnFirstNonNull(peers, + p -> client.getGroupManagementApi((p.getId())).info(remoteGroupId)); + processReply(groupInfoReply, printStream::println, + () -> "Failed to get group info for group id " + remoteGroupId.getUuid() + " from " + peers); + return groupInfoReply; + } + + public static void processReply(RaftClientReply reply, Consumer printer, Supplier message) + throws IOException { + if (reply == null || !reply.isSuccess()) { + final RaftException e = Optional.ofNullable(reply) + .map(RaftClientReply::getException) + .orElseGet(() -> new RaftException("Reply: " + reply)); + printer.accept(message.get()); + throw new IOException(e.getMessage(), e); + } + } + + public static InetSocketAddress parseInetSocketAddress(String address) { + try { + final String[] hostPortPair = address.split(":"); + if (hostPortPair.length < 2) { + throw new IllegalArgumentException("Unexpected address format ."); + } + return new InetSocketAddress(hostPortPair[0], Integer.parseInt(hostPortPair[1])); + } catch (Exception e) { + throw new IllegalArgumentException("Failed to parse the server address parameter \"" + address + "\".", e); + } + } + } diff --git a/ratis-shell/src/main/java/org/apache/ratis/shell/cli/sh/command/AbstractRatisCommand.java b/ratis-shell/src/main/java/org/apache/ratis/shell/cli/sh/command/AbstractRatisCommand.java index 1888c0e0ea..91bdc873b7 100644 --- a/ratis-shell/src/main/java/org/apache/ratis/shell/cli/sh/command/AbstractRatisCommand.java +++ b/ratis-shell/src/main/java/org/apache/ratis/shell/cli/sh/command/AbstractRatisCommand.java @@ -18,8 +18,12 @@ package org.apache.ratis.shell.cli.sh.command; import org.apache.commons.cli.Option; -import org.apache.ratis.protocol.*; -import org.apache.ratis.protocol.exceptions.RaftException; +import org.apache.ratis.protocol.RaftClientReply; +import org.apache.ratis.protocol.RaftGroup; +import org.apache.ratis.protocol.RaftGroupId; +import org.apache.ratis.protocol.RaftPeer; +import org.apache.ratis.protocol.RaftPeerId; +import org.apache.ratis.protocol.GroupInfoReply; import org.apache.ratis.shell.cli.RaftUtils; import org.apache.commons.cli.CommandLine; import org.apache.commons.cli.Options; @@ -30,48 +34,30 @@ import org.apache.ratis.proto.RaftProtos.RaftPeerRole; import org.apache.ratis.proto.RaftProtos.RoleInfoProto; import org.apache.ratis.util.ProtoUtils; -import org.apache.ratis.util.function.CheckedFunction; import java.io.IOException; +import java.io.PrintStream; import java.net.InetSocketAddress; -import java.util.*; +import java.util.ArrayList; +import java.util.Collections; +import java.util.List; +import java.util.Set; import java.util.function.BiConsumer; import java.util.function.Supplier; import java.util.stream.Collectors; import java.util.stream.Stream; +import static org.apache.ratis.shell.cli.RaftUtils.buildRaftGroupIdFromStr; +import static org.apache.ratis.shell.cli.RaftUtils.buildRaftPeersFromStr; +import static org.apache.ratis.shell.cli.RaftUtils.retrieveGroupInfoByGroupId; +import static org.apache.ratis.shell.cli.RaftUtils.retrieveRemoteGroupId; + /** * The base class for the ratis shell which need to connect to server. */ public abstract class AbstractRatisCommand extends AbstractCommand { public static final String PEER_OPTION_NAME = "peers"; public static final String GROUPID_OPTION_NAME = "groupid"; - public static final RaftGroupId DEFAULT_RAFT_GROUP_ID = RaftGroupId.randomId(); - - /** - * Execute a given function with input parameter from the members of a list. - * - * @param list the input parameters - * @param function the function to be executed - * @param parameter type - * @param return value type - * @param the exception type thrown by the given function. - * @return the value returned by the given function. - */ - public static K run(Collection list, CheckedFunction function) { - for (T t : list) { - try { - K ret = function.apply(t); - if (ret != null) { - return ret; - } - } catch (Throwable e) { - e.printStackTrace(); - } - } - return null; - } - private RaftGroup raftGroup; private GroupInfoReply groupInfoReply; @@ -81,46 +67,13 @@ protected AbstractRatisCommand(Context context) { @Override public int run(CommandLine cl) throws IOException { - List addresses = new ArrayList<>(); - String peersStr = cl.getOptionValue(PEER_OPTION_NAME); - String[] peersArray = peersStr.split(","); - for (String peer : peersArray) { - addresses.add(parseInetSocketAddress(peer)); - } - - final RaftGroupId raftGroupIdFromConfig = cl.hasOption(GROUPID_OPTION_NAME)? - RaftGroupId.valueOf(UUID.fromString(cl.getOptionValue(GROUPID_OPTION_NAME))) - : DEFAULT_RAFT_GROUP_ID; - - List peers = addresses.stream() - .map(addr -> RaftPeer.newBuilder() - .setId(RaftUtils.getPeerId(addr)) - .setAddress(addr) - .build() - ).collect(Collectors.toList()); + List peers = buildRaftPeersFromStr(cl.getOptionValue(PEER_OPTION_NAME)); + RaftGroupId raftGroupIdFromConfig = buildRaftGroupIdFromStr(cl.getOptionValue(GROUPID_OPTION_NAME)); raftGroup = RaftGroup.valueOf(raftGroupIdFromConfig, peers); + PrintStream printStream = getPrintStream(); try (final RaftClient client = RaftUtils.createClient(raftGroup)) { - final RaftGroupId remoteGroupId; - if (raftGroupIdFromConfig != DEFAULT_RAFT_GROUP_ID) { - remoteGroupId = raftGroupIdFromConfig; - } else { - final List groupIds = run(peers, - p -> client.getGroupManagementApi((p.getId())).list().getGroupIds()); - - if (groupIds == null) { - println("Failed to get group ID from " + peers); - return -1; - } else if (groupIds.size() == 1) { - remoteGroupId = groupIds.get(0); - } else { - println("There are more than one groups, you should specific one. " + groupIds); - return -2; - } - } - - groupInfoReply = run(peers, p -> client.getGroupManagementApi((p.getId())).info(remoteGroupId)); - processReply(groupInfoReply, - () -> "Failed to get group info for group id " + remoteGroupId.getUuid() + " from " + peers); + final RaftGroupId remoteGroupId = retrieveRemoteGroupId(raftGroupIdFromConfig, peers, client, printStream); + groupInfoReply = retrieveGroupInfoByGroupId(remoteGroupId, peers, client, printStream); raftGroup = groupInfoReply.getGroup(); } return 0; @@ -168,14 +121,7 @@ protected RaftPeerProto getLeader(RoleInfoProto roleInfo) { } protected void processReply(RaftClientReply reply, Supplier messageSupplier) throws IOException { - if (reply == null || !reply.isSuccess()) { - final RaftException e = Optional.ofNullable(reply) - .map(RaftClientReply::getException) - .orElseGet(() -> new RaftException("Reply: " + reply)); - final String message = messageSupplier.get(); - printf("%s. Error: %s%n", message, e); - throw new IOException(message, e); - } + RaftUtils.processReply(reply, getPrintStream()::println, messageSupplier); } protected List getIds(String[] optionValues, BiConsumer consumer) { From 6cb0555a492a50e58a31a7106dff5e676729537f Mon Sep 17 00:00:00 2001 From: "Doroszlai, Attila" <6454655+adoroszlai@users.noreply.github.com> Date: Tue, 25 Jun 2024 10:03:37 +0200 Subject: [PATCH 062/397] RATIS-2110. Publish SBOM artifacts (#1110) --- pom.xml | 18 ++++++++++++++++++ ratis-assembly/src/main/assembly/bin.xml | 10 ++++++++++ 2 files changed, 28 insertions(+) diff --git a/pom.xml b/pom.xml index 60c145506a..2b61bf3d03 100644 --- a/pom.xml +++ b/pom.xml @@ -189,6 +189,7 @@ 0.6.1 1.0 + 2.8.0 4.2.1 4.2.0 @@ -724,6 +725,11 @@ jacoco-maven-plugin ${jacoco.version} + + org.cyclonedx + cyclonedx-maven-plugin + ${cyclonedx.version} + @@ -1055,6 +1061,18 @@ + + org.cyclonedx + cyclonedx-maven-plugin + + + package + + makeAggregateBom + + + + diff --git a/ratis-assembly/src/main/assembly/bin.xml b/ratis-assembly/src/main/assembly/bin.xml index 7be3d01d61..abf4d40aa8 100644 --- a/ratis-assembly/src/main/assembly/bin.xml +++ b/ratis-assembly/src/main/assembly/bin.xml @@ -63,6 +63,16 @@ 0644 + + + ${project.basedir}/../target + . + + bom.json + bom.xml + + 0644 + ${project.basedir}/../ratis-docs/target/classes/docs From 24a2712ad558efa645c87b0d5993ee57f65acacd Mon Sep 17 00:00:00 2001 From: "Doroszlai, Attila" <6454655+adoroszlai@users.noreply.github.com> Date: Fri, 28 Jun 2024 13:16:28 +0200 Subject: [PATCH 063/397] RATIS-2117. No need for manual assembly:single execution (#1115) --- .github/workflows/post-commit.yml | 2 +- dev-support/make_rc.sh | 4 ++-- ratis-assembly/pom.xml | 2 +- 3 files changed, 4 insertions(+), 4 deletions(-) diff --git a/.github/workflows/post-commit.yml b/.github/workflows/post-commit.yml index 7c286b29b2..2aafb90316 100644 --- a/.github/workflows/post-commit.yml +++ b/.github/workflows/post-commit.yml @@ -44,7 +44,7 @@ jobs: distribution: 'temurin' java-version: 8 - name: Run a full build - run: ./dev-support/checks/build.sh -Prelease assembly:single + run: ./dev-support/checks/build.sh -Prelease - name: Store binaries for tests uses: actions/upload-artifact@v4 with: diff --git a/dev-support/make_rc.sh b/dev-support/make_rc.sh index b5bec51dde..6394d1427b 100755 --- a/dev-support/make_rc.sh +++ b/dev-support/make_rc.sh @@ -106,7 +106,7 @@ prepare-src() { #grep -r SNAPSHOT --include=pom.xml - mvnFun clean install assembly:single -DskipTests=true -Prelease -Papache-release -Dgpg.keyname="${CODESIGNINGKEY}" + mvnFun clean install -DskipTests=true -Prelease -Papache-release -Dgpg.keyname="${CODESIGNINGKEY}" } prepare-bin() { @@ -118,7 +118,7 @@ prepare-bin() { mv "apache-ratis-${RATISVERSION}-src" "apache-ratis-${RATISVERSION}" cd "apache-ratis-${RATISVERSION}" - mvnFun clean install assembly:single -DskipTests=true -Prelease -Papache-release -Dgpg.keyname="${CODESIGNINGKEY}" + mvnFun clean install -DskipTests=true -Prelease -Papache-release -Dgpg.keyname="${CODESIGNINGKEY}" } assembly() { diff --git a/ratis-assembly/pom.xml b/ratis-assembly/pom.xml index 37fe569206..91a48c2901 100644 --- a/ratis-assembly/pom.xml +++ b/ratis-assembly/pom.xml @@ -142,7 +142,7 @@ - default-cli + bin package single From 4219c13b658e61af9a5d496c66714caf562d5956 Mon Sep 17 00:00:00 2001 From: "Doroszlai, Attila" <6454655+adoroszlai@users.noreply.github.com> Date: Mon, 1 Jul 2024 07:27:49 +0200 Subject: [PATCH 064/397] RATIS-2118. Improve assembly process (#1117) --- .github/workflows/post-commit.yml | 10 +-- dev-support/make_rc.sh | 6 +- ratis-assembly/pom.xml | 9 --- ratis-assembly/src/main/assembly/bin-pkg.xml | 43 ----------- ratis-assembly/src/main/assembly/bin.xml | 57 ++++++++++++++- .../src/main/assembly/examples-bin.xml | 73 ------------------- .../src/main/assembly/shell-bin.xml | 61 ---------------- ratis-assembly/src/main/assembly/src.xml | 1 + 8 files changed, 65 insertions(+), 195 deletions(-) delete mode 100644 ratis-assembly/src/main/assembly/bin-pkg.xml delete mode 100644 ratis-assembly/src/main/assembly/examples-bin.xml delete mode 100644 ratis-assembly/src/main/assembly/shell-bin.xml diff --git a/.github/workflows/post-commit.yml b/.github/workflows/post-commit.yml index 2aafb90316..4a09057544 100644 --- a/.github/workflows/post-commit.yml +++ b/.github/workflows/post-commit.yml @@ -50,14 +50,14 @@ jobs: with: name: ratis-bin path: | - ratis-assembly/target/apache-ratis-*.tar.gz - !ratis-assembly/target/apache-ratis-*-src.tar.gz + ratis-assembly/target/ratis-assembly-*.tar.gz + !ratis-assembly/target/ratis-assembly-*-src.tar.gz retention-days: 1 - name: Store source tarball for compilation uses: actions/upload-artifact@v4 with: name: ratis-src - path: ratis-assembly/target/apache-ratis-*-src.tar.gz + path: ratis-assembly/target/ratis-assembly-*-src.tar.gz retention-days: 1 compile: needs: @@ -75,7 +75,7 @@ jobs: name: ratis-src - name: Untar sources run: | - tar --strip-components 1 -xzvf apache-ratis-*-src.tar.gz + tar --strip-components 1 -xzvf ratis-assembly-*-src.tar.gz - name: Cache for maven dependencies uses: actions/cache/restore@v4 with: @@ -256,7 +256,7 @@ jobs: - name: Untar binaries run: | mkdir -p ratis-assembly/target - tar xzvf target/artifacts/ratis-bin/apache-ratis*.tar.gz -C ratis-assembly/target + tar xzvf target/artifacts/ratis-bin/ratis-assembly-*.tar.gz -C ratis-assembly/target - name: Calculate combined coverage run: ./dev-support/checks/coverage.sh - name: Upload coverage to Sonar diff --git a/dev-support/make_rc.sh b/dev-support/make_rc.sh index 6394d1427b..64a6d3d413 100755 --- a/dev-support/make_rc.sh +++ b/dev-support/make_rc.sh @@ -114,7 +114,7 @@ prepare-bin() { rm -rf "$WORKINGDIR" mkdir -p "$WORKINGDIR" cd "$WORKINGDIR" - tar zvxf "$projectdir/ratis-assembly/target/apache-ratis-${RATISVERSION}-src.tar.gz" + tar zvxf "$projectdir/ratis-assembly/target/ratis-assembly-${RATISVERSION}-src.tar.gz" mv "apache-ratis-${RATISVERSION}-src" "apache-ratis-${RATISVERSION}" cd "apache-ratis-${RATISVERSION}" @@ -126,8 +126,8 @@ assembly() { RCDIR="$SVNDISTDIR/${RATISVERSION}/${RC#-}" mkdir -p "$RCDIR" cd "$RCDIR" - cp "$WORKINGDIR/apache-ratis-${RATISVERSION}/ratis-assembly/target/apache-ratis-${RATISVERSION}-bin.tar.gz" "apache-ratis-${RATISVERSION}-bin.tar.gz" - cp "$projectdir/ratis-assembly/target/apache-ratis-${RATISVERSION}-src.tar.gz" "apache-ratis-${RATISVERSION}-src.tar.gz" + cp "$WORKINGDIR/apache-ratis-${RATISVERSION}/ratis-assembly/target/ratis-assembly-${RATISVERSION}-bin.tar.gz" "apache-ratis-${RATISVERSION}-bin.tar.gz" + cp "$projectdir/ratis-assembly/target/ratis-assembly-${RATISVERSION}-src.tar.gz" "apache-ratis-${RATISVERSION}-src.tar.gz" for i in *.tar.gz; do gpg -u "${CODESIGNINGKEY}" --armor --output "${i}.asc" --detach-sig "${i}"; done for i in *.tar.gz; do gpg --print-md SHA512 "${i}" > "${i}.sha512"; done for i in *.tar.gz; do gpg --print-mds "${i}" > "${i}.mds"; done diff --git a/ratis-assembly/pom.xml b/ratis-assembly/pom.xml index 91a48c2901..90ac0ac600 100644 --- a/ratis-assembly/pom.xml +++ b/ratis-assembly/pom.xml @@ -120,8 +120,6 @@ org.apache.maven.plugins maven-assembly-plugin - - apache-ratis-${project.version} false true gnu @@ -137,8 +135,6 @@ src/main/assembly/src.xml - apache-ratis-${project.version}-src - false @@ -149,13 +145,8 @@ - src/main/assembly/examples-bin.xml - src/main/assembly/shell-bin.xml src/main/assembly/bin.xml - src/main/assembly/bin-pkg.xml - apache-ratis-${project.version}-bin - false diff --git a/ratis-assembly/src/main/assembly/bin-pkg.xml b/ratis-assembly/src/main/assembly/bin-pkg.xml deleted file mode 100644 index 4d89869578..0000000000 --- a/ratis-assembly/src/main/assembly/bin-pkg.xml +++ /dev/null @@ -1,43 +0,0 @@ - - - - bin-pkg - - tar.gz - - - - ${project.basedir}/target/apache-ratis-${project.version}-bin/apache-ratis-${project.version}-bin - .. - - bin/ratis - libexec/*.sh - examples/bin/*.sh - - - - ${project.basedir}/target/apache-ratis-${project.version}-bin/apache-ratis-${project.version}-bin - .. - - bin/ratis - libexec/*.sh - examples/bin/*.sh - - 0755 - - - diff --git a/ratis-assembly/src/main/assembly/bin.xml b/ratis-assembly/src/main/assembly/bin.xml index abf4d40aa8..7fd3cc0df5 100644 --- a/ratis-assembly/src/main/assembly/bin.xml +++ b/ratis-assembly/src/main/assembly/bin.xml @@ -22,9 +22,18 @@ */ --> bin + apache-ratis-${project.version}-bin - dir + tar.gz + + + + org.apache.ratis:ratis-examples + + examples/lib + + true @@ -42,6 +51,7 @@ org.apache.ratis:ratis-metrics-api org.apache.ratis:ratis-metrics-default org.apache.ratis:ratis-metrics-dropwizard3 + org.apache.ratis:ratis-shell org.apache.ratis:ratis-tools org.apache.ratis:ratis-resource-bundle @@ -80,5 +90,50 @@ 0644 0755 + + ${project.basedir}/../ratis-shell/src/main/bin + bin + 0755 + + + ${project.basedir}/../ratis-shell/src/main/libexec + libexec + 0755 + 0755 + + + ${project.basedir}/../ratis-shell/src/main/conf + conf + 644 + + + ${project.basedir}/../ratis-shell/target/lib/ + jars + + + ${project.basedir}/../ratis-examples + examples + + README.md + + 0644 + + + ${project.basedir}/../ratis-examples/src/main/bin + examples/bin + + *.* + + 0755 + + + ${project.basedir}/../ratis-examples/src/main/resources + examples/conf + + conf.properties + log4j.properties + + 644 + diff --git a/ratis-assembly/src/main/assembly/examples-bin.xml b/ratis-assembly/src/main/assembly/examples-bin.xml deleted file mode 100644 index 21cc7eced2..0000000000 --- a/ratis-assembly/src/main/assembly/examples-bin.xml +++ /dev/null @@ -1,73 +0,0 @@ - - - - examples-bin - - dir - - - - - org.apache.ratis:ratis-examples - - examples/lib - - - - - ${project.basedir}/src/main/resources - . - - README.md - LICENSE - NOTICE - - 0644 - - - ${project.basedir}/../ratis-examples - examples - - README.md - - 0644 - - - ${project.basedir}/../ratis-examples/src/main/bin - examples/bin - - *.* - - 0755 - - - ${project.basedir}/../ratis-examples/src/main/resources - examples/conf - - conf.properties - log4j.properties - - 644 - - - diff --git a/ratis-assembly/src/main/assembly/shell-bin.xml b/ratis-assembly/src/main/assembly/shell-bin.xml deleted file mode 100644 index 470870f41c..0000000000 --- a/ratis-assembly/src/main/assembly/shell-bin.xml +++ /dev/null @@ -1,61 +0,0 @@ - - - - shell - - dir - - - - ${project.basedir}/../ratis-shell/target/ - jars - - ratis-shell-${project.version}.jar - - - - ${project.basedir}/../ratis-shell/target/lib/ - jars - - - ${project.basedir}/src/main/resources - . - - README.md - LICENSE - NOTICE - - 0644 - - - ${project.basedir}/../ratis-shell/src/main/bin - bin - 0755 - - - ${project.basedir}/../ratis-shell/src/main/libexec - libexec - 0755 - 0755 - - - ${project.basedir}/../ratis-shell/src/main/conf - conf - 644 - - - diff --git a/ratis-assembly/src/main/assembly/src.xml b/ratis-assembly/src/main/assembly/src.xml index 98e06c5739..8c025c9c66 100644 --- a/ratis-assembly/src/main/assembly/src.xml +++ b/ratis-assembly/src/main/assembly/src.xml @@ -22,6 +22,7 @@ */ --> src + apache-ratis-${project.version}-src tar.gz From a0208dcef2f07a5fea2cece2ffa1fe8cbe4fcd17 Mon Sep 17 00:00:00 2001 From: "Doroszlai, Attila" <6454655+adoroszlai@users.noreply.github.com> Date: Mon, 1 Jul 2024 07:41:26 +0200 Subject: [PATCH 065/397] RATIS-2121. Set javac --release when compiling with JDK 9+ (#1119) --- pom.xml | 24 +++++++++++++++++++++--- 1 file changed, 21 insertions(+), 3 deletions(-) diff --git a/pom.xml b/pom.xml index 2b61bf3d03..e52c34267b 100644 --- a/pom.xml +++ b/pom.xml @@ -204,7 +204,7 @@ bash - 1.8 + 8 ${javac.version} 3.3.9 @@ -522,6 +522,7 @@ org.apache.maven.plugins maven-enforcer-plugin + 3.2.1 @@ -574,8 +575,6 @@ org.apache.maven.plugins maven-compiler-plugin - ${javac.version} - ${javac.version} true 512m 2048m @@ -874,6 +873,25 @@ + + java8 + + [,8] + + + ${javac.version} + ${javac.version} + + + + java9-or-later + + [9,] + + + ${javac.version} + + experiments-build From 00f6a150ad1ab37ebaa1eb37513e7573156066d7 Mon Sep 17 00:00:00 2001 From: Haibo Sun <7675577+sunhaibotb@users.noreply.github.com> Date: Tue, 2 Jul 2024 14:42:21 +0800 Subject: [PATCH 066/397] RATIS-2116. Fix the issue where RaftServerImpl.appendEntries may be blocked indefinitely (#1116) --- .../apache/ratis/util/DataBlockingQueue.java | 7 ++ .../java/org/apache/ratis/util/DataQueue.java | 5 ++ .../segmented/SegmentedRaftLogWorker.java | 6 +- .../segmented/TestSegmentedRaftLog.java | 84 +++++++++++++++++++ 4 files changed, 101 insertions(+), 1 deletion(-) diff --git a/ratis-common/src/main/java/org/apache/ratis/util/DataBlockingQueue.java b/ratis-common/src/main/java/org/apache/ratis/util/DataBlockingQueue.java index 842b8f1549..e905893e5b 100644 --- a/ratis-common/src/main/java/org/apache/ratis/util/DataBlockingQueue.java +++ b/ratis-common/src/main/java/org/apache/ratis/util/DataBlockingQueue.java @@ -162,4 +162,11 @@ public List pollList(long timeoutM return results; } } + + @Override + public E peek() { + try(AutoCloseableLock auto = AutoCloseableLock.acquire(lock)) { + return super.peek(); + } + } } diff --git a/ratis-common/src/main/java/org/apache/ratis/util/DataQueue.java b/ratis-common/src/main/java/org/apache/ratis/util/DataQueue.java index 3db06f56e6..38762caa17 100644 --- a/ratis-common/src/main/java/org/apache/ratis/util/DataQueue.java +++ b/ratis-common/src/main/java/org/apache/ratis/util/DataQueue.java @@ -154,6 +154,11 @@ public E poll() { return polled; } + /** Peek the head element from this queue. */ + public E peek() { + return q.peek(); + } + /** The same as {@link java.util.Collection#remove(Object)}. */ public boolean remove(E e) { final boolean removed = q.remove(e); diff --git a/ratis-server/src/main/java/org/apache/ratis/server/raftlog/segmented/SegmentedRaftLogWorker.java b/ratis-server/src/main/java/org/apache/ratis/server/raftlog/segmented/SegmentedRaftLogWorker.java index bfeca7f87f..573270cca5 100644 --- a/ratis-server/src/main/java/org/apache/ratis/server/raftlog/segmented/SegmentedRaftLogWorker.java +++ b/ratis-server/src/main/java/org/apache/ratis/server/raftlog/segmented/SegmentedRaftLogWorker.java @@ -63,6 +63,9 @@ class SegmentedRaftLogWorker { static final TimeDuration ONE_SECOND = TimeDuration.valueOf(1, TimeUnit.SECONDS); + private static final String CLASS_NAME = JavaUtils.getClassSimpleName(SegmentedRaftLogWorker.class); + static final String RUN_WORKER = CLASS_NAME + ".runWorker"; + static class StateMachineDataPolicy { private final boolean sync; private final TimeDuration syncTimeout; @@ -297,6 +300,7 @@ private void run() { // if and when a log task encounters an exception RaftLogIOException logIOException = null; + CodeInjectionForTesting.execute(RUN_WORKER, server == null ? null : server.getId(), null, queue); while (running) { try { Task task = queue.poll(ONE_SECOND); @@ -355,7 +359,7 @@ private boolean shouldFlush() { } else if (pendingFlushNum >= forceSyncNum) { return true; } - return pendingFlushNum > 0 && queue.isEmpty(); + return pendingFlushNum > 0 && !(queue.peek() instanceof WriteLog); } private void flushIfNecessary() throws IOException { diff --git a/ratis-test/src/test/java/org/apache/ratis/server/raftlog/segmented/TestSegmentedRaftLog.java b/ratis-test/src/test/java/org/apache/ratis/server/raftlog/segmented/TestSegmentedRaftLog.java index b564501f36..93eb7db0e6 100644 --- a/ratis-test/src/test/java/org/apache/ratis/server/raftlog/segmented/TestSegmentedRaftLog.java +++ b/ratis-test/src/test/java/org/apache/ratis/server/raftlog/segmented/TestSegmentedRaftLog.java @@ -40,6 +40,8 @@ import org.apache.ratis.statemachine.impl.SimpleStateMachine4Testing; import org.apache.ratis.statemachine.StateMachine; import org.apache.ratis.statemachine.impl.BaseStateMachine; +import org.apache.ratis.util.CodeInjectionForTesting; +import org.apache.ratis.util.DataBlockingQueue; import org.apache.ratis.util.LifeCycle; import org.apache.ratis.util.Slf4jUtils; import org.apache.ratis.util.FileUtils; @@ -55,8 +57,13 @@ import java.util.Collections; import java.util.List; import java.util.concurrent.CompletableFuture; +import java.util.concurrent.ConcurrentLinkedQueue; +import java.util.concurrent.CountDownLatch; import java.util.concurrent.ExecutionException; import java.util.concurrent.TimeUnit; +import java.util.concurrent.TimeoutException; +import java.util.concurrent.atomic.AtomicInteger; +import java.util.concurrent.atomic.AtomicReference; import java.util.function.LongSupplier; import java.util.function.Supplier; import java.util.stream.Stream; @@ -72,6 +79,7 @@ import static java.lang.Boolean.FALSE; import static java.lang.Boolean.TRUE; +import static org.apache.ratis.server.raftlog.segmented.SegmentedRaftLogWorker.RUN_WORKER; import static org.junit.jupiter.api.Assertions.assertTrue; import static org.junit.jupiter.params.provider.Arguments.arguments; @@ -393,6 +401,82 @@ public void testAppendAndRoll(Boolean useAsyncFlush, Boolean smSyncFlush) throws } } + @ParameterizedTest + @MethodSource("data") + public void testPurgeAfterAppendEntry(Boolean useAsyncFlush, Boolean smSyncFlush) throws Exception { + RaftServerConfigKeys.Log.setAsyncFlushEnabled(properties, useAsyncFlush); + RaftServerConfigKeys.Log.StateMachineData.setSync(properties, smSyncFlush); + RaftServerConfigKeys.Log.setPurgeGap(properties, 1); + RaftServerConfigKeys.Log.setForceSyncNum(properties, 128); + + int startTerm = 0; + int endTerm = 2; + int segmentSize = 10; + long endIndexOfClosedSegment = segmentSize * (endTerm - startTerm - 1); + long nextStartIndex = segmentSize * (endTerm - startTerm); + + // append entries and roll logSegment for later purge operation + List ranges0 = prepareRanges(startTerm, endTerm, segmentSize, 0); + List entries0 = prepareLogEntries(ranges0, null); + try (SegmentedRaftLog raftLog = newSegmentedRaftLog()) { + raftLog.open(RaftLog.INVALID_LOG_INDEX, null); + entries0.stream().map(raftLog::appendEntry).forEach(CompletableFuture::join); + } + + // test the pattern in the task queue of SegmentedRaftLogWorker: (WriteLog, ..., PurgeLog) + List ranges = prepareRanges(endTerm - 1, endTerm, 1, nextStartIndex); + List entries = prepareLogEntries(ranges, null); + + try (SegmentedRaftLog raftLog = newSegmentedRaftLog()) { + final CountDownLatch raftLogOpened = new CountDownLatch(1); + final CountDownLatch tasksAdded = new CountDownLatch(1); + + // inject test code to make the pattern (WriteLog, PurgeLog) + final ConcurrentLinkedQueue> appendFutures = new ConcurrentLinkedQueue<>(); + final AtomicReference> purgeFuture = new AtomicReference<>(); + final AtomicInteger tasksCount = new AtomicInteger(0); + CodeInjectionForTesting.put(RUN_WORKER, (localId, remoteId, args) -> { + // wait for raftLog to be opened + try { + if(!raftLogOpened.await(FIVE_SECONDS.getDuration(), FIVE_SECONDS.getUnit())) { + throw new TimeoutException(); + } + } catch (InterruptedException | TimeoutException e) { + LOG.error("an exception occurred", e); + throw new RuntimeException(e); + } + + // add WriteLog and PurgeLog tasks + entries.stream().map(raftLog::appendEntry).forEach(appendFutures::add); + purgeFuture.set(raftLog.purge(endIndexOfClosedSegment)); + + tasksCount.set(((DataBlockingQueue) args[0]).getNumElements()); + tasksAdded.countDown(); + return true; + }); + + // open raftLog + raftLog.open(RaftLog.INVALID_LOG_INDEX, null); + raftLogOpened.countDown(); + + // wait for all tasks to be added + if(!tasksAdded.await(FIVE_SECONDS.getDuration(), FIVE_SECONDS.getUnit())) { + throw new TimeoutException(); + } + Assertions.assertEquals(entries.size() + 1, tasksCount.get()); + + // check if the purge task is executed + final Long purged = purgeFuture.get().get(); + LOG.info("purgeIndex = {}, purged = {}", endIndexOfClosedSegment, purged); + Assertions.assertEquals(endIndexOfClosedSegment, raftLog.getRaftLogCache().getStartIndex()); + + // check if the appendEntry futures are done + JavaUtils.allOf(appendFutures).get(FIVE_SECONDS.getDuration(), FIVE_SECONDS.getUnit()); + } finally { + CodeInjectionForTesting.put(RUN_WORKER, (localId, remoteId, args) -> false); + } + } + @ParameterizedTest @MethodSource("data") public void testTruncate(Boolean useAsyncFlush, Boolean smSyncFlush) throws Exception { From ad4c3d57c097b2ee411f821bb542473f22b42f8b Mon Sep 17 00:00:00 2001 From: "Doroszlai, Attila" <6454655+adoroszlai@users.noreply.github.com> Date: Tue, 2 Jul 2024 16:27:30 +0200 Subject: [PATCH 067/397] RATIS-1840. Avoid including build timestamp in artifacts (#1114) --- .github/workflows/post-commit.yml | 2 + dev-support/checks/repro.sh | 31 ++++++++++++ pom.xml | 55 ++++++++++++++------- src/main/resources/ratis-version.properties | 20 ++++++++ 4 files changed, 90 insertions(+), 18 deletions(-) create mode 100755 dev-support/checks/repro.sh create mode 100644 src/main/resources/ratis-version.properties diff --git a/.github/workflows/post-commit.yml b/.github/workflows/post-commit.yml index 4a09057544..5fee9462ae 100644 --- a/.github/workflows/post-commit.yml +++ b/.github/workflows/post-commit.yml @@ -92,6 +92,8 @@ jobs: java-version: ${{ matrix.java }} - name: Run a full build run: ./dev-support/checks/build.sh -Djavac.version=${{ matrix.java }} + - name: Test reproducibility + run: ./dev-support/checks/repro.sh -Djavac.version=${{ matrix.java }} rat: name: rat runs-on: ubuntu-20.04 diff --git a/dev-support/checks/repro.sh b/dev-support/checks/repro.sh new file mode 100755 index 0000000000..c87443ef21 --- /dev/null +++ b/dev-support/checks/repro.sh @@ -0,0 +1,31 @@ +#!/usr/bin/env bash +# 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. +DIR="$( cd "$( dirname "${BASH_SOURCE[0]}" )" >/dev/null 2>&1 && pwd )" +cd "$DIR/../.." || exit 1 + +source "${DIR}/../find_maven.sh" + +: ${WITH_COVERAGE:="false"} + +MAVEN_OPTIONS='-V -B -Dmaven.javadoc.skip=true -DskipTests --no-transfer-progress' + +if [[ "${WITH_COVERAGE}" != "true" ]]; then + MAVEN_OPTIONS="${MAVEN_OPTIONS} -Djacoco.skip" +fi + +export MAVEN_OPTS="-Xmx4096m" +${MVN} ${MAVEN_OPTIONS} clean verify artifact:compare "$@" +exit $? diff --git a/pom.xml b/pom.xml index e52c34267b..50712b3b9a 100644 --- a/pom.xml +++ b/pom.xml @@ -178,6 +178,7 @@ 1.6.1 3.0.0-M4 3.5.3 + 3.4.0 @@ -455,6 +456,11 @@ + + org.apache.hadoop + hadoop-maven-plugins + ${hadoop-maven-plugins.version} + org.xolstice.maven.plugins protobuf-maven-plugin @@ -786,6 +792,24 @@ true true + + org.apache.hadoop + hadoop-maven-plugins + + + version-info + generate-resources + + version-info + + + + ${project.basedir} + + + + + org.apache.maven.plugins maven-checkstyle-plugin @@ -807,24 +831,6 @@ false - - org.codehaus.mojo - buildnumber-maven-plugin - 1.4 - - - generate-resources - - create-metadata - - - target/classes - ratis-version.properties - Unknown - - - - org.codehaus.mojo build-helper-maven-plugin @@ -870,6 +876,19 @@ + + + ${project.basedir}/src/main/resources + false + + + ${project.basedir}/../src/main/resources + + ratis-version.properties + + true + + diff --git a/src/main/resources/ratis-version.properties b/src/main/resources/ratis-version.properties new file mode 100644 index 0000000000..f34dc73dc7 --- /dev/null +++ b/src/main/resources/ratis-version.properties @@ -0,0 +1,20 @@ +# +# 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. +# +name=${project.name} +version=${project.version} +revision=${version-info.scm.commit} From 3030f34029f37191f2a444e87498f8288e8dd0dd Mon Sep 17 00:00:00 2001 From: "Doroszlai, Attila" <6454655+adoroszlai@users.noreply.github.com> Date: Mon, 15 Jul 2024 13:06:47 +0200 Subject: [PATCH 068/397] RATIS-2125. javadoc jars should not include non-doc items (#1123) --- pom.xml | 3 --- 1 file changed, 3 deletions(-) diff --git a/pom.xml b/pom.xml index 50712b3b9a..dda19aec8d 100644 --- a/pom.xml +++ b/pom.xml @@ -977,9 +977,6 @@ jar - - ${project.build.directory} - From bcb76701ea4d87d79d6aff1142a9e1c95354e6ab Mon Sep 17 00:00:00 2001 From: Tsz-Wo Nicholas Sze Date: Wed, 17 Jul 2024 13:30:07 +0800 Subject: [PATCH 069/397] RATIS-1071. NettyClientRpc supports sendRequestAsync. Contributed by Rui Wang and Tsz-Wo Nicholas Sze. (#1122) --- .../org/apache/ratis/netty/NettyRpcProxy.java | 6 ++ .../ratis/netty/client/NettyClientRpc.java | 67 ++++++++++++++----- .../ratis/netty/TestRaftAsyncWithNetty.java | 25 +++++++ 3 files changed, 81 insertions(+), 17 deletions(-) create mode 100644 ratis-test/src/test/java/org/apache/ratis/netty/TestRaftAsyncWithNetty.java diff --git a/ratis-netty/src/main/java/org/apache/ratis/netty/NettyRpcProxy.java b/ratis-netty/src/main/java/org/apache/ratis/netty/NettyRpcProxy.java index b9788a8bb4..41269f76e3 100644 --- a/ratis-netty/src/main/java/org/apache/ratis/netty/NettyRpcProxy.java +++ b/ratis-netty/src/main/java/org/apache/ratis/netty/NettyRpcProxy.java @@ -176,6 +176,12 @@ public void close() { connection.close(); } + public CompletableFuture sendAsync(RaftNettyServerRequestProto proto) { + final CompletableFuture reply = new CompletableFuture<>(); + connection.offer(proto, reply); + return reply; + } + public RaftNettyServerReplyProto send( RaftRpcRequestProto request, RaftNettyServerRequestProto proto) throws IOException { diff --git a/ratis-netty/src/main/java/org/apache/ratis/netty/client/NettyClientRpc.java b/ratis-netty/src/main/java/org/apache/ratis/netty/client/NettyClientRpc.java index c816e29ee8..26ac41f7db 100644 --- a/ratis-netty/src/main/java/org/apache/ratis/netty/client/NettyClientRpc.java +++ b/ratis-netty/src/main/java/org/apache/ratis/netty/client/NettyClientRpc.java @@ -28,71 +28,104 @@ import org.apache.ratis.proto.RaftProtos.GroupManagementRequestProto; import org.apache.ratis.proto.RaftProtos.SetConfigurationRequestProto; import org.apache.ratis.proto.netty.NettyProtos.RaftNettyServerRequestProto; +import org.apache.ratis.util.JavaUtils; import java.io.IOException; +import java.util.concurrent.CompletableFuture; public class NettyClientRpc extends RaftClientRpcWithProxy { public NettyClientRpc(ClientId clientId, RaftProperties properties) { super(new NettyRpcProxy.PeerMap(clientId.toString(), properties)); } + @Override + public CompletableFuture sendRequestAsync(RaftClientRequest request) { + final RaftPeerId serverId = request.getServerId(); + try { + final NettyRpcProxy proxy = getProxies().getProxy(serverId); + final RaftNettyServerRequestProto serverRequestProto = buildRequestProto(request); + return proxy.sendAsync(serverRequestProto).thenApply(replyProto -> { + if (request instanceof GroupListRequest) { + return ClientProtoUtils.toGroupListReply(replyProto.getGroupListReply()); + } else if (request instanceof GroupInfoRequest) { + return ClientProtoUtils.toGroupInfoReply(replyProto.getGroupInfoReply()); + } else { + return ClientProtoUtils.toRaftClientReply(replyProto.getRaftClientReply()); + } + }); + } catch (Throwable e) { + return JavaUtils.completeExceptionally(e); + } + } + @Override public RaftClientReply sendRequest(RaftClientRequest request) throws IOException { final RaftPeerId serverId = request.getServerId(); final NettyRpcProxy proxy = getProxies().getProxy(serverId); + final RaftNettyServerRequestProto serverRequestProto = buildRequestProto(request); + final RaftRpcRequestProto rpcRequest = getRpcRequestProto(serverRequestProto); + if (request instanceof GroupListRequest) { + return ClientProtoUtils.toGroupListReply( + proxy.send(rpcRequest, serverRequestProto).getGroupListReply()); + } else if (request instanceof GroupInfoRequest) { + return ClientProtoUtils.toGroupInfoReply( + proxy.send(rpcRequest, serverRequestProto).getGroupInfoReply()); + } else { + return ClientProtoUtils.toRaftClientReply( + proxy.send(rpcRequest, serverRequestProto).getRaftClientReply()); + } + } + + private RaftNettyServerRequestProto buildRequestProto(RaftClientRequest request) { final RaftNettyServerRequestProto.Builder b = RaftNettyServerRequestProto.newBuilder(); - final RaftRpcRequestProto rpcRequest; if (request instanceof GroupManagementRequest) { final GroupManagementRequestProto proto = ClientProtoUtils.toGroupManagementRequestProto( (GroupManagementRequest)request); b.setGroupManagementRequest(proto); - rpcRequest = proto.getRpcRequest(); } else if (request instanceof SetConfigurationRequest) { final SetConfigurationRequestProto proto = ClientProtoUtils.toSetConfigurationRequestProto( (SetConfigurationRequest)request); b.setSetConfigurationRequest(proto); - rpcRequest = proto.getRpcRequest(); } else if (request instanceof GroupListRequest) { final RaftProtos.GroupListRequestProto proto = ClientProtoUtils.toGroupListRequestProto( (GroupListRequest)request); b.setGroupListRequest(proto); - rpcRequest = proto.getRpcRequest(); } else if (request instanceof GroupInfoRequest) { final RaftProtos.GroupInfoRequestProto proto = ClientProtoUtils.toGroupInfoRequestProto( (GroupInfoRequest)request); b.setGroupInfoRequest(proto); - rpcRequest = proto.getRpcRequest(); } else if (request instanceof TransferLeadershipRequest) { final RaftProtos.TransferLeadershipRequestProto proto = ClientProtoUtils.toTransferLeadershipRequestProto( (TransferLeadershipRequest)request); b.setTransferLeadershipRequest(proto); - rpcRequest = proto.getRpcRequest(); } else if (request instanceof SnapshotManagementRequest) { final RaftProtos.SnapshotManagementRequestProto proto = ClientProtoUtils.toSnapshotManagementRequestProto( (SnapshotManagementRequest) request); b.setSnapshotManagementRequest(proto); - rpcRequest = proto.getRpcRequest(); } else if (request instanceof LeaderElectionManagementRequest) { final RaftProtos.LeaderElectionManagementRequestProto proto = ClientProtoUtils.toLeaderElectionManagementRequestProto( (LeaderElectionManagementRequest) request); b.setLeaderElectionManagementRequest(proto); - rpcRequest = proto.getRpcRequest(); } else { final RaftClientRequestProto proto = ClientProtoUtils.toRaftClientRequestProto(request); b.setRaftClientRequest(proto); - rpcRequest = proto.getRpcRequest(); } - if (request instanceof GroupListRequest) { - return ClientProtoUtils.toGroupListReply( - proxy.send(rpcRequest, b.build()).getGroupListReply()); - } else if (request instanceof GroupInfoRequest) { - return ClientProtoUtils.toGroupInfoReply( - proxy.send(rpcRequest, b.build()).getGroupInfoReply()); + return b.build(); + } + + private RaftRpcRequestProto getRpcRequestProto(RaftNettyServerRequestProto serverRequestProto) { + if (serverRequestProto.hasGroupManagementRequest()) { + return serverRequestProto.getGroupManagementRequest().getRpcRequest(); + } else if (serverRequestProto.hasSetConfigurationRequest()) { + return serverRequestProto.getSetConfigurationRequest().getRpcRequest(); + } else if (serverRequestProto.hasGroupListRequest()) { + return serverRequestProto.getGroupListRequest().getRpcRequest(); + } else if (serverRequestProto.hasGroupInfoRequest()) { + return serverRequestProto.getGroupInfoRequest().getRpcRequest(); } else { - return ClientProtoUtils.toRaftClientReply( - proxy.send(rpcRequest, b.build()).getRaftClientReply()); + return serverRequestProto.getRaftClientRequest().getRpcRequest(); } } } diff --git a/ratis-test/src/test/java/org/apache/ratis/netty/TestRaftAsyncWithNetty.java b/ratis-test/src/test/java/org/apache/ratis/netty/TestRaftAsyncWithNetty.java new file mode 100644 index 0000000000..ebaa33d505 --- /dev/null +++ b/ratis-test/src/test/java/org/apache/ratis/netty/TestRaftAsyncWithNetty.java @@ -0,0 +1,25 @@ +/* + * 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.ratis.netty; + +import org.apache.ratis.RaftAsyncTests; + +public class TestRaftAsyncWithNetty + extends RaftAsyncTests + implements MiniRaftClusterWithNetty.FactoryGet { +} \ No newline at end of file From ca086b931a54bcf93eb948ab63f4774ea1e4d3cf Mon Sep 17 00:00:00 2001 From: Mridul Muralidharan <1591700+mridulm@users.noreply.github.com> Date: Thu, 25 Jul 2024 01:33:00 -0700 Subject: [PATCH 070/397] RATIS-2131. Surround with [] only if hostName is a IPv6 string (#1125) --- .../src/main/java/org/apache/ratis/util/NetUtils.java | 10 ++++++++-- 1 file changed, 8 insertions(+), 2 deletions(-) diff --git a/ratis-common/src/main/java/org/apache/ratis/util/NetUtils.java b/ratis-common/src/main/java/org/apache/ratis/util/NetUtils.java index a6ce5af792..baf6762aaa 100644 --- a/ratis-common/src/main/java/org/apache/ratis/util/NetUtils.java +++ b/ratis-common/src/main/java/org/apache/ratis/util/NetUtils.java @@ -17,6 +17,8 @@ */ package org.apache.ratis.util; +import org.apache.ratis.thirdparty.com.google.common.net.InetAddresses; + import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -145,8 +147,12 @@ static String address2String(InetSocketAddress address) { if (address == null) { return null; } - final StringBuilder b = new StringBuilder(address.getHostName()); - if (address.getAddress() instanceof Inet6Address) { + String hostName = address.getHostName(); + final StringBuilder b = new StringBuilder(hostName); + // Surround with '[', ']' only if it is a IPv6 ip - not for a IPv6 host + if (address.getAddress() instanceof Inet6Address && + InetAddresses.isInetAddress(hostName) && + InetAddresses.forString(hostName).getAddress().length == 16) { b.insert(0, '[').append(']'); } return b.append(':').append(address.getPort()).toString(); From aab923bb18822fb55fd6a4531e1b04187d1e3a12 Mon Sep 17 00:00:00 2001 From: Ming Wei <292619280@qq.com> Date: Tue, 30 Jul 2024 11:52:23 +0800 Subject: [PATCH 071/397] RATIS-2126. Optimize NotLeaderException message (#1124) --- .../apache/ratis/protocol/exceptions/NotLeaderException.java | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/ratis-common/src/main/java/org/apache/ratis/protocol/exceptions/NotLeaderException.java b/ratis-common/src/main/java/org/apache/ratis/protocol/exceptions/NotLeaderException.java index 8d5c2cb4e9..c7dc6a3961 100644 --- a/ratis-common/src/main/java/org/apache/ratis/protocol/exceptions/NotLeaderException.java +++ b/ratis-common/src/main/java/org/apache/ratis/protocol/exceptions/NotLeaderException.java @@ -30,7 +30,8 @@ public class NotLeaderException extends RaftException { private final Collection peers; public NotLeaderException(RaftGroupMemberId memberId, RaftPeer suggestedLeader, Collection peers) { - super("Server " + memberId + " is not the leader" + (suggestedLeader != null? " " + suggestedLeader: "")); + super("Server " + memberId + " is not the leader" + + (suggestedLeader != null ? ", suggested leader is: " + suggestedLeader : "")); this.suggestedLeader = suggestedLeader; this.peers = peers != null? Collections.unmodifiableCollection(peers): Collections.emptyList(); Preconditions.assertUnique(this.peers); From ceb18d14262b495cadda44994771d78130e6bccd Mon Sep 17 00:00:00 2001 From: Sadanand Shenoy Date: Thu, 1 Aug 2024 20:58:12 +0530 Subject: [PATCH 072/397] RATIS-2133. Ignore http proxy for ratis grpc client & server (#1128) --- .../org/apache/ratis/grpc/client/GrpcClientProtocolClient.java | 2 ++ .../org/apache/ratis/grpc/server/GrpcServerProtocolClient.java | 2 ++ 2 files changed, 4 insertions(+) diff --git a/ratis-grpc/src/main/java/org/apache/ratis/grpc/client/GrpcClientProtocolClient.java b/ratis-grpc/src/main/java/org/apache/ratis/grpc/client/GrpcClientProtocolClient.java index 08bacdb73b..a1f01a5129 100644 --- a/ratis-grpc/src/main/java/org/apache/ratis/grpc/client/GrpcClientProtocolClient.java +++ b/ratis-grpc/src/main/java/org/apache/ratis/grpc/client/GrpcClientProtocolClient.java @@ -128,6 +128,8 @@ private ManagedChannel buildChannel(String address, GrpcTlsConfig tlsConf, SizeInBytes flowControlWindow, SizeInBytes maxMessageSize) { NettyChannelBuilder channelBuilder = NettyChannelBuilder.forTarget(address); + // ignore any http proxy for grpc + channelBuilder.proxyDetector(uri -> null); if (tlsConf != null) { SslContextBuilder sslContextBuilder = GrpcSslContexts.forClient(); diff --git a/ratis-grpc/src/main/java/org/apache/ratis/grpc/server/GrpcServerProtocolClient.java b/ratis-grpc/src/main/java/org/apache/ratis/grpc/server/GrpcServerProtocolClient.java index 970134d02a..4a280ab335 100644 --- a/ratis-grpc/src/main/java/org/apache/ratis/grpc/server/GrpcServerProtocolClient.java +++ b/ratis-grpc/src/main/java/org/apache/ratis/grpc/server/GrpcServerProtocolClient.java @@ -78,6 +78,8 @@ private ManagedChannel buildChannel(RaftPeer target, int flowControlWindow, GrpcTlsConfig tlsConfig) { NettyChannelBuilder channelBuilder = NettyChannelBuilder.forTarget(target.getAddress()); + // ignore any http proxy for grpc + channelBuilder.proxyDetector(uri -> null); if (tlsConfig!= null) { SslContextBuilder sslContextBuilder = GrpcSslContexts.forClient(); From f82d55648439c113995b687998b675fbebeacb8e Mon Sep 17 00:00:00 2001 From: Duong Nguyen Date: Fri, 2 Aug 2024 09:32:22 -0700 Subject: [PATCH 073/397] RATIS-2104. Double shutdown in TestLeaderInstallSnapshot. (#1108) --- .../ratis/InstallSnapshotFromLeaderTests.java | 160 +++++++++--------- 1 file changed, 76 insertions(+), 84 deletions(-) diff --git a/ratis-server/src/test/java/org/apache/ratis/InstallSnapshotFromLeaderTests.java b/ratis-server/src/test/java/org/apache/ratis/InstallSnapshotFromLeaderTests.java index b83a7dfdd3..16a3353249 100644 --- a/ratis-server/src/test/java/org/apache/ratis/InstallSnapshotFromLeaderTests.java +++ b/ratis-server/src/test/java/org/apache/ratis/InstallSnapshotFromLeaderTests.java @@ -95,105 +95,97 @@ public void testInstallSnapshotLeaderSwitch() throws Exception { } private void testMultiFileInstallSnapshot(CLUSTER cluster) throws Exception { - try { - int i = 0; - RaftTestUtil.waitForLeader(cluster); - final RaftPeerId leaderId = cluster.getLeader().getId(); - - try (final RaftClient client = cluster.createClient(leaderId)) { - for (; i < SNAPSHOT_TRIGGER_THRESHOLD * 2 - 1; i++) { - RaftClientReply - reply = client.io().send(new RaftTestUtil.SimpleMessage("m" + i)); - Assertions.assertTrue(reply.isSuccess()); - } - - client.getSnapshotManagementApi(leaderId).create(3000); + int i = 0; + RaftTestUtil.waitForLeader(cluster); + final RaftPeerId leaderId = cluster.getLeader().getId(); + + try (final RaftClient client = cluster.createClient(leaderId)) { + for (; i < SNAPSHOT_TRIGGER_THRESHOLD * 2 - 1; i++) { + RaftClientReply + reply = client.io().send(new RaftTestUtil.SimpleMessage("m" + i)); + Assertions.assertTrue(reply.isSuccess()); } - final SnapshotInfo snapshot = cluster.getLeader().getStateMachine().getLatestSnapshot(); - Assertions.assertEquals(3, snapshot.getFiles().size()); - - // add two more peers - final MiniRaftCluster.PeerChanges change = cluster.addNewPeers(2, true, - true); - // trigger setConfiguration - cluster.setConfiguration(change.allPeersInNewConf); - - RaftServerTestUtil - .waitAndCheckNewConf(cluster, change.allPeersInNewConf, 0, null); - - // Check the installed snapshot file number on each Follower matches with the - // leader snapshot. - JavaUtils.attempt(() -> { - for (RaftServer.Division follower : cluster.getFollowers()) { - final SnapshotInfo info = follower.getStateMachine().getLatestSnapshot(); - Assertions.assertNotNull(info); - Assertions.assertEquals(3, info.getFiles().size()); - } - }, 10, ONE_SECOND, "check snapshot", LOG); - } finally { - cluster.shutdown(); + client.getSnapshotManagementApi(leaderId).create(3000); } + + final SnapshotInfo snapshot = cluster.getLeader().getStateMachine().getLatestSnapshot(); + Assertions.assertEquals(3, snapshot.getFiles().size()); + + // add two more peers + final MiniRaftCluster.PeerChanges change = cluster.addNewPeers(2, true, + true); + // trigger setConfiguration + cluster.setConfiguration(change.allPeersInNewConf); + + RaftServerTestUtil + .waitAndCheckNewConf(cluster, change.allPeersInNewConf, 0, null); + + // Check the installed snapshot file number on each Follower matches with the + // leader snapshot. + JavaUtils.attempt(() -> { + for (RaftServer.Division follower : cluster.getFollowers()) { + final SnapshotInfo info = follower.getStateMachine().getLatestSnapshot(); + Assertions.assertNotNull(info); + Assertions.assertEquals(3, info.getFiles().size()); + } + }, 10, ONE_SECOND, "check snapshot", LOG); } private void testInstallSnapshotDuringLeaderSwitch(CLUSTER cluster) throws Exception { - try { - RaftTestUtil.waitForLeader(cluster); - final RaftPeerId leaderId = cluster.getLeader().getId(); - - // perform operations and force all peers to take snapshot - try (final RaftClient client = cluster.createClient(leaderId)) { - for (int i = 0; i < SNAPSHOT_TRIGGER_THRESHOLD * 2; i++) { - final RaftClientReply - reply = client.io().send(new RaftTestUtil.SimpleMessage("m" + i)); - Assertions.assertTrue(reply.isSuccess()); - } + RaftTestUtil.waitForLeader(cluster); + final RaftPeerId leaderId = cluster.getLeader().getId(); - for (final RaftPeer peer: cluster.getPeers()) { - final RaftClientReply snapshotReply = client.getSnapshotManagementApi(leaderId).create(3000); - Assertions.assertTrue(snapshotReply.isSuccess()); - } + // perform operations and force all peers to take snapshot + try (final RaftClient client = cluster.createClient(leaderId)) { + for (int i = 0; i < SNAPSHOT_TRIGGER_THRESHOLD * 2; i++) { + final RaftClientReply + reply = client.io().send(new RaftTestUtil.SimpleMessage("m" + i)); + Assertions.assertTrue(reply.isSuccess()); } - final SnapshotInfo snapshot = cluster.getLeader().getStateMachine().getLatestSnapshot(); - Assertions.assertNotNull(snapshot); - // isolate two followers (majority) in old configuration - final List oldFollowers = cluster.getFollowers(); - for (RaftServer.Division f: oldFollowers) { - RaftTestUtil.isolate(cluster, f.getId()); + for (final RaftPeer peer: cluster.getPeers()) { + final RaftClientReply snapshotReply = client.getSnapshotManagementApi(leaderId).create(3000); + Assertions.assertTrue(snapshotReply.isSuccess()); } + } + final SnapshotInfo snapshot = cluster.getLeader().getStateMachine().getLatestSnapshot(); + Assertions.assertNotNull(snapshot); - // add two more peers and install snapshot from leaders - final MiniRaftCluster.PeerChanges change = cluster.addNewPeers(2, true, - true); - try (final RaftClient client = cluster.createClient(leaderId, RetryPolicies.noRetry())) { - Assertions.assertThrows(RaftRetryFailureException.class, - () -> client.admin().setConfiguration(change.allPeersInNewConf)); - } + // isolate two followers (majority) in old configuration + final List oldFollowers = cluster.getFollowers(); + for (RaftServer.Division f: oldFollowers) { + RaftTestUtil.isolate(cluster, f.getId()); + } - final SnapshotInfo snapshotInfo = cluster.getDivision(change.newPeers[0].getId()) - .getStateMachine().getLatestSnapshot(); - Assertions.assertNotNull(snapshotInfo); + // add two more peers and install snapshot from leaders + final MiniRaftCluster.PeerChanges change = cluster.addNewPeers(2, true, + true); + try (final RaftClient client = cluster.createClient(leaderId, RetryPolicies.noRetry())) { + Assertions.assertThrows(RaftRetryFailureException.class, + () -> client.admin().setConfiguration(change.allPeersInNewConf)); + } - // recover the old followers and isolate the leader to force leader switch - RaftTestUtil.isolate(cluster, leaderId); - for (RaftServer.Division f: oldFollowers) { - RaftTestUtil.deIsolate(cluster, f.getId()); - } - RaftTestUtil.waitForLeader(cluster); + final SnapshotInfo snapshotInfo = cluster.getDivision(change.newPeers[0].getId()) + .getStateMachine().getLatestSnapshot(); + Assertions.assertNotNull(snapshotInfo); - try (final RaftClient client = cluster.createClient(cluster.getLeader().getId())) { - // successfully setConfiguration during leader switch - final RaftClientReply setConf = client.admin().setConfiguration(change.allPeersInNewConf); - Assertions.assertTrue(setConf.isSuccess()); + // recover the old followers and isolate the leader to force leader switch + RaftTestUtil.isolate(cluster, leaderId); + for (RaftServer.Division f: oldFollowers) { + RaftTestUtil.deIsolate(cluster, f.getId()); + } + RaftTestUtil.waitForLeader(cluster); - RaftTestUtil.deIsolate(cluster, leaderId); - final RaftClientReply - reply = client.io().send(new RaftTestUtil.SimpleMessage("final")); - Assertions.assertTrue(reply.isSuccess()); - } - } finally { - cluster.shutdown(); + try (final RaftClient client = cluster.createClient(cluster.getLeader().getId())) { + // successfully setConfiguration during leader switch + final RaftClientReply setConf = client.admin().setConfiguration(change.allPeersInNewConf); + Assertions.assertTrue(setConf.isSuccess()); + + RaftTestUtil.deIsolate(cluster, leaderId); + final RaftClientReply + reply = client.io().send(new RaftTestUtil.SimpleMessage("final")); + Assertions.assertTrue(reply.isSuccess()); } } From dc8bf7b0d5ffd3773efc811efd76bc6754374285 Mon Sep 17 00:00:00 2001 From: Chung En Lee Date: Tue, 6 Aug 2024 22:01:31 +0800 Subject: [PATCH 074/397] RATIS-2134. `logMetadata` could miss appending the `metadataEntry` with the lastCommitIndex (#1130) --- .../ratis/server/impl/LeaderStateImpl.java | 17 +++++++++++------ .../ratis/server/raftlog/RaftLogBase.java | 17 +++-------------- 2 files changed, 14 insertions(+), 20 deletions(-) diff --git a/ratis-server/src/main/java/org/apache/ratis/server/impl/LeaderStateImpl.java b/ratis-server/src/main/java/org/apache/ratis/server/impl/LeaderStateImpl.java index eacf506154..32f9dbeed2 100644 --- a/ratis-server/src/main/java/org/apache/ratis/server/impl/LeaderStateImpl.java +++ b/ratis-server/src/main/java/org/apache/ratis/server/impl/LeaderStateImpl.java @@ -946,10 +946,7 @@ private boolean hasMajority(Predicate isAcked) { private void updateCommit(LogEntryHeader[] entriesToCommit) { final long newCommitIndex = raftLog.getLastCommittedIndex(); - if (logMetadataEnabled) { - logMetadata(newCommitIndex); - } - commitIndexChanged(); + long lastCommitIndex = RaftLog.INVALID_LOG_INDEX; boolean hasConfiguration = false; for (LogEntryHeader entry : entriesToCommit) { @@ -958,7 +955,14 @@ private void updateCommit(LogEntryHeader[] entriesToCommit) { } hasConfiguration |= entry.getLogEntryBodyCase() == LogEntryBodyCase.CONFIGURATIONENTRY; raftLog.getRaftLogMetrics().onLogEntryCommitted(entry); + if (entry.getLogEntryBodyCase() != LogEntryBodyCase.METADATAENTRY) { + lastCommitIndex = entry.getIndex(); + } + } + if (logMetadataEnabled && lastCommitIndex != RaftLog.INVALID_LOG_INDEX) { + logMetadata(lastCommitIndex); } + commitIndexChanged(); if (hasConfiguration) { checkAndUpdateConfiguration(); } @@ -978,8 +982,9 @@ private void updateCommit(long majority, long min) { } private void logMetadata(long commitIndex) { - raftLog.appendMetadata(currentTerm, commitIndex); - notifySenders(); + if (raftLog.appendMetadata(currentTerm, commitIndex) != RaftLog.INVALID_LOG_INDEX) { + notifySenders(); + } } private void checkAndUpdateConfiguration() { diff --git a/ratis-server/src/main/java/org/apache/ratis/server/raftlog/RaftLogBase.java b/ratis-server/src/main/java/org/apache/ratis/server/raftlog/RaftLogBase.java index 14851b1a4e..d241bcd205 100644 --- a/ratis-server/src/main/java/org/apache/ratis/server/raftlog/RaftLogBase.java +++ b/ratis-server/src/main/java/org/apache/ratis/server/raftlog/RaftLogBase.java @@ -237,21 +237,10 @@ private boolean shouldAppendMetadata(long newCommitIndex) { if (newCommitIndex <= 0) { // do not log the first conf entry return false; - } else if (Optional.ofNullable(lastMetadataEntry.get()) - .filter(e -> e.getIndex() == newCommitIndex || e.getMetadataEntry().getCommitIndex() >= newCommitIndex) - .isPresent()) { - //log neither lastMetadataEntry, nor entries with a smaller commit index. - return false; - } - try { - if (get(newCommitIndex).hasMetadataEntry()) { - // do not log the metadata entry - return false; - } - } catch(RaftLogIOException e) { - LOG.error("Failed to get log entry for index " + newCommitIndex, e); } - return true; + final LogEntryProto last = lastMetadataEntry.get(); + // do not log entries with a smaller commit index. + return last == null || newCommitIndex > last.getMetadataEntry().getCommitIndex(); } @Override From 1beac435645f428bf64ee691ed555f93e25ec066 Mon Sep 17 00:00:00 2001 From: Potato Date: Sat, 10 Aug 2024 00:13:59 +0800 Subject: [PATCH 075/397] RATIS-2136 Fixed the issue that RatisServer could not start after frequent restarts (#1131) --- .../main/java/org/apache/ratis/server/raftlog/RaftLogBase.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/ratis-server/src/main/java/org/apache/ratis/server/raftlog/RaftLogBase.java b/ratis-server/src/main/java/org/apache/ratis/server/raftlog/RaftLogBase.java index d241bcd205..1e22e07bd0 100644 --- a/ratis-server/src/main/java/org/apache/ratis/server/raftlog/RaftLogBase.java +++ b/ratis-server/src/main/java/org/apache/ratis/server/raftlog/RaftLogBase.java @@ -272,7 +272,7 @@ public final void open(long lastIndexInSnapshot, Consumer consume final long startIndex = getStartIndex(); if (startIndex > LEAST_VALID_LOG_INDEX) { - purgeIndex.updateIncreasingly(startIndex - 1, infoIndexChange); + purgeIndex.updateToMax(startIndex - 1, infoIndexChange); } } From 295ca145fd81899bab009c6d82652c18590c22f6 Mon Sep 17 00:00:00 2001 From: Tsz-Wo Nicholas Sze Date: Mon, 12 Aug 2024 09:21:29 -0700 Subject: [PATCH 076/397] RATIS-2115. Fix flaky RetryCacheTests and InstallSnapshotFromLeaderTests. (#1129) --- .../org/apache/ratis/InstallSnapshotFromLeaderTests.java | 7 ++++++- .../src/test/java/org/apache/ratis/RetryCacheTests.java | 6 +++--- ...napshot.java => TestLeaderInstallSnapshotWithGrpc.java} | 4 ++-- 3 files changed, 11 insertions(+), 6 deletions(-) rename ratis-test/src/test/java/org/apache/ratis/grpc/{TestLeaderInstallSnapshot.java => TestLeaderInstallSnapshotWithGrpc.java} (97%) diff --git a/ratis-server/src/test/java/org/apache/ratis/InstallSnapshotFromLeaderTests.java b/ratis-server/src/test/java/org/apache/ratis/InstallSnapshotFromLeaderTests.java index 16a3353249..ba6bca5b5d 100644 --- a/ratis-server/src/test/java/org/apache/ratis/InstallSnapshotFromLeaderTests.java +++ b/ratis-server/src/test/java/org/apache/ratis/InstallSnapshotFromLeaderTests.java @@ -23,7 +23,9 @@ import org.apache.ratis.protocol.RaftGroupId; import org.apache.ratis.protocol.RaftPeer; import org.apache.ratis.protocol.RaftPeerId; +import org.apache.ratis.protocol.exceptions.RaftException; import org.apache.ratis.protocol.exceptions.RaftRetryFailureException; +import org.apache.ratis.protocol.exceptions.ReconfigurationTimeoutException; import org.apache.ratis.retry.RetryPolicies; import org.apache.ratis.server.RaftServer; import org.apache.ratis.server.RaftServerConfigKeys; @@ -162,8 +164,11 @@ private void testInstallSnapshotDuringLeaderSwitch(CLUSTER cluster) throws Excep final MiniRaftCluster.PeerChanges change = cluster.addNewPeers(2, true, true); try (final RaftClient client = cluster.createClient(leaderId, RetryPolicies.noRetry())) { - Assertions.assertThrows(RaftRetryFailureException.class, + final RaftException e = Assertions.assertThrows(RaftException.class, () -> client.admin().setConfiguration(change.allPeersInNewConf)); + Assertions.assertTrue( e instanceof RaftRetryFailureException + || e instanceof ReconfigurationTimeoutException, + () -> "Unexpected exception: " + e); } final SnapshotInfo snapshotInfo = cluster.getDivision(change.newPeers[0].getId()) diff --git a/ratis-server/src/test/java/org/apache/ratis/RetryCacheTests.java b/ratis-server/src/test/java/org/apache/ratis/RetryCacheTests.java index 8929dfcc7f..5a363bf081 100644 --- a/ratis-server/src/test/java/org/apache/ratis/RetryCacheTests.java +++ b/ratis-server/src/test/java/org/apache/ratis/RetryCacheTests.java @@ -40,6 +40,7 @@ import org.slf4j.event.Level; import java.util.Arrays; +import java.util.Collections; import java.util.concurrent.TimeUnit; import static java.util.Arrays.asList; @@ -137,9 +138,8 @@ void runTestRetryOnNewLeader(CLUSTER cluster) throws Exception { final long oldLastApplied = cluster.getLeader().getInfo().getLastAppliedIndex(); // trigger the reconfiguration, make sure the original leader is kicked out - PeerChanges change = cluster.addNewPeers(2, true); - RaftPeer[] allPeers = cluster.removePeers(2, true, - asList(change.newPeers)).allPeersInNewConf; + final PeerChanges change = cluster.removePeers(2, true, Collections.emptyList()); + final RaftPeer[] allPeers = change.allPeersInNewConf; // trigger setConfiguration RaftServerTestUtil.runWithMinorityPeers(cluster, Arrays.asList(allPeers), peers -> cluster.setConfiguration(peers.toArray(RaftPeer.emptyArray()))); diff --git a/ratis-test/src/test/java/org/apache/ratis/grpc/TestLeaderInstallSnapshot.java b/ratis-test/src/test/java/org/apache/ratis/grpc/TestLeaderInstallSnapshotWithGrpc.java similarity index 97% rename from ratis-test/src/test/java/org/apache/ratis/grpc/TestLeaderInstallSnapshot.java rename to ratis-test/src/test/java/org/apache/ratis/grpc/TestLeaderInstallSnapshotWithGrpc.java index b85cd13535..9b2c903849 100644 --- a/ratis-test/src/test/java/org/apache/ratis/grpc/TestLeaderInstallSnapshot.java +++ b/ratis-test/src/test/java/org/apache/ratis/grpc/TestLeaderInstallSnapshotWithGrpc.java @@ -1,4 +1,4 @@ -/** +/* * 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 @@ -24,7 +24,7 @@ import java.util.Arrays; import java.util.Collection; -public class TestLeaderInstallSnapshot +public class TestLeaderInstallSnapshotWithGrpc extends InstallSnapshotFromLeaderTests implements MiniRaftClusterWithGrpc.FactoryGet { From 76a157210347df87bc6461abf7f7c52b85b98ad7 Mon Sep 17 00:00:00 2001 From: Tsz-Wo Nicholas Sze Date: Mon, 12 Aug 2024 09:22:51 -0700 Subject: [PATCH 077/397] RATIS-2135. The leader keeps sending inconsistent entries repeatedly to followers. (#1132) --- .../apache/ratis/grpc/server/GrpcService.java | 23 +++++++------------ 1 file changed, 8 insertions(+), 15 deletions(-) diff --git a/ratis-grpc/src/main/java/org/apache/ratis/grpc/server/GrpcService.java b/ratis-grpc/src/main/java/org/apache/ratis/grpc/server/GrpcService.java index 097900a0fb..5997c54290 100644 --- a/ratis-grpc/src/main/java/org/apache/ratis/grpc/server/GrpcService.java +++ b/ratis-grpc/src/main/java/org/apache/ratis/grpc/server/GrpcService.java @@ -63,8 +63,7 @@ public final class GrpcService extends RaftServerRpcWithProxy appendEntriesAsync(AppendEntriesRequestProto request) - throws IOException { + public CompletableFuture appendEntriesAsync(AppendEntriesRequestProto request) { throw new UnsupportedOperationException("This method is not supported"); } @@ -97,7 +96,6 @@ public void onCompleted() { public static final class Builder { private RaftServer server; - private GrpcTlsConfig tlsConfig; private GrpcTlsConfig adminTlsConfig; private GrpcTlsConfig clientTlsConfig; private GrpcTlsConfig serverTlsConfig; @@ -113,11 +111,6 @@ public GrpcService build() { return new GrpcService(server, adminTlsConfig, clientTlsConfig, serverTlsConfig); } - public Builder setTlsConfig(GrpcTlsConfig tlsConfig) { - this.tlsConfig = tlsConfig; - return this; - } - public Builder setAdminTlsConfig(GrpcTlsConfig config) { this.adminTlsConfig = config; return this; @@ -132,10 +125,6 @@ public Builder setServerTlsConfig(GrpcTlsConfig config) { this.serverTlsConfig = config; return this; } - - public GrpcTlsConfig getTlsConfig() { - return tlsConfig; - } } public static Builder newBuilder() { @@ -188,10 +177,14 @@ private GrpcService(RaftServer raftServer, Supplier idSupplier, super(idSupplier, id -> new PeerProxyMap<>(id.toString(), p -> new GrpcServerProtocolClient(p, flowControlWindow.getSizeInt(), requestTimeoutDuration, serverTlsConfig, useSeparateHBChannel))); - if (appenderBufferSize.getSize() > grpcMessageSizeMax.getSize()) { + + final SizeInBytes gap = SizeInBytes.ONE_MB; + final long diff = grpcMessageSizeMax.getSize() - appenderBufferSize.getSize(); + if (diff < gap.getSize()) { throw new IllegalArgumentException("Illegal configuration: " - + RaftServerConfigKeys.Log.Appender.BUFFER_BYTE_LIMIT_KEY + " = " + appenderBufferSize - + " > " + GrpcConfigKeys.MESSAGE_SIZE_MAX_KEY + " = " + grpcMessageSizeMax); + + GrpcConfigKeys.MESSAGE_SIZE_MAX_KEY + "(= " + grpcMessageSizeMax + + ") must be " + gap + " larger than " + + RaftServerConfigKeys.Log.Appender.BUFFER_BYTE_LIMIT_KEY + "(= " + appenderBufferSize + ")."); } final RaftProperties properties = raftServer.getProperties(); From 3236f5fce6d91ac3ba4092566e5485d7e75b0d53 Mon Sep 17 00:00:00 2001 From: Potato Date: Tue, 13 Aug 2024 12:17:47 +0800 Subject: [PATCH 078/397] RATIS-2138 Remove uncessary error log in JVMPauseMonitor When GC is serious (#1133) --- .../java/org/apache/ratis/server/impl/LeaderStateImpl.java | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/ratis-server/src/main/java/org/apache/ratis/server/impl/LeaderStateImpl.java b/ratis-server/src/main/java/org/apache/ratis/server/impl/LeaderStateImpl.java index 32f9dbeed2..68121cef6a 100644 --- a/ratis-server/src/main/java/org/apache/ratis/server/impl/LeaderStateImpl.java +++ b/ratis-server/src/main/java/org/apache/ratis/server/impl/LeaderStateImpl.java @@ -618,7 +618,7 @@ public AppendEntriesRequestProto newAppendEntriesRequestProto(FollowerInfo follo List entries, TermIndex previous, long callId) { final boolean initializing = !isCaughtUp(follower); final RaftPeerId targetId = follower.getId(); - return ServerProtoUtils.toAppendEntriesRequestProto(server.getMemberId(), targetId, currentTerm, entries, + return ServerProtoUtils.toAppendEntriesRequestProto(server.getMemberId(), targetId, getCurrentTerm(), entries, ServerImplUtils.effectiveCommitIndex(raftLog.getLastCommittedIndex(), previous, entries.size()), initializing, previous, server.getCommitInfos(), callId); } @@ -698,7 +698,7 @@ private void updateSenders(RaftConfigurationImpl conf) { } void submitStepDownEvent(StepDownReason reason) { - submitStepDownEvent(getCurrentTerm(), reason); + submitStepDownEvent(currentTerm, reason); } void submitStepDownEvent(long term, StepDownReason reason) { From 7ea7582c209b18f94b2146672f34f79b6491279c Mon Sep 17 00:00:00 2001 From: Tsz-Wo Nicholas Sze Date: Wed, 14 Aug 2024 11:09:29 -0700 Subject: [PATCH 079/397] RATIS-2139. Fix checkstyle:ParameterNumber in GrpcService. (#1134) --- .../apache/ratis/grpc/server/GrpcService.java | 256 ++++++++++-------- 1 file changed, 147 insertions(+), 109 deletions(-) diff --git a/ratis-grpc/src/main/java/org/apache/ratis/grpc/server/GrpcService.java b/ratis-grpc/src/main/java/org/apache/ratis/grpc/server/GrpcService.java index 5997c54290..33f23315d2 100644 --- a/ratis-grpc/src/main/java/org/apache/ratis/grpc/server/GrpcService.java +++ b/ratis-grpc/src/main/java/org/apache/ratis/grpc/server/GrpcService.java @@ -22,13 +22,16 @@ import org.apache.ratis.grpc.GrpcTlsConfig; import org.apache.ratis.grpc.GrpcUtil; import org.apache.ratis.grpc.metrics.intercept.server.MetricServerInterceptor; +import org.apache.ratis.protocol.AdminAsynchronousProtocol; import org.apache.ratis.protocol.RaftGroupId; +import org.apache.ratis.protocol.RaftPeer; import org.apache.ratis.protocol.RaftPeerId; import org.apache.ratis.rpc.SupportedRpcType; import org.apache.ratis.server.RaftServer; import org.apache.ratis.server.RaftServerConfigKeys; import org.apache.ratis.server.RaftServerRpcWithProxy; import org.apache.ratis.server.protocol.RaftServerAsynchronousProtocol; +import org.apache.ratis.thirdparty.io.grpc.ServerInterceptor; import org.apache.ratis.thirdparty.io.grpc.ServerInterceptors; import org.apache.ratis.thirdparty.io.grpc.netty.GrpcSslContexts; import org.apache.ratis.thirdparty.io.grpc.netty.NettyServerBuilder; @@ -96,19 +99,139 @@ public void onCompleted() { public static final class Builder { private RaftServer server; + + private String adminHost; + private int adminPort; private GrpcTlsConfig adminTlsConfig; + private String clientHost; + private int clientPort; private GrpcTlsConfig clientTlsConfig; + private String serverHost; + private int serverPort; private GrpcTlsConfig serverTlsConfig; + private SizeInBytes messageSizeMax; + private SizeInBytes flowControlWindow; + private TimeDuration requestTimeoutDuration; + private boolean separateHeartbeatChannel; + private Builder() {} public Builder setServer(RaftServer raftServer) { this.server = raftServer; + + final RaftProperties properties = server.getProperties(); + this.adminHost = GrpcConfigKeys.Admin.host(properties); + this.adminPort = GrpcConfigKeys.Admin.port(properties); + this.clientHost = GrpcConfigKeys.Client.host(properties); + this.clientPort = GrpcConfigKeys.Client.port(properties); + this.serverHost = GrpcConfigKeys.Server.host(properties); + this.serverPort = GrpcConfigKeys.Server.port(properties); + this.messageSizeMax = GrpcConfigKeys.messageSizeMax(properties, LOG::info); + this.flowControlWindow = GrpcConfigKeys.flowControlWindow(properties, LOG::info); + this.requestTimeoutDuration = RaftServerConfigKeys.Rpc.requestTimeout(properties); + this.separateHeartbeatChannel = GrpcConfigKeys.Server.heartbeatChannel(properties); + + final SizeInBytes appenderBufferSize = RaftServerConfigKeys.Log.Appender.bufferByteLimit(properties); + final SizeInBytes gap = SizeInBytes.ONE_MB; + final long diff = messageSizeMax.getSize() - appenderBufferSize.getSize(); + if (diff < gap.getSize()) { + throw new IllegalArgumentException("Illegal configuration: " + + GrpcConfigKeys.MESSAGE_SIZE_MAX_KEY + "(= " + messageSizeMax + + ") must be " + gap + " larger than " + + RaftServerConfigKeys.Log.Appender.BUFFER_BYTE_LIMIT_KEY + "(= " + appenderBufferSize + ")."); + } + return this; } + private GrpcServerProtocolClient newGrpcServerProtocolClient(RaftPeer target) { + return new GrpcServerProtocolClient(target, flowControlWindow.getSizeInt(), + requestTimeoutDuration, serverTlsConfig, separateHeartbeatChannel); + } + + private ExecutorService newExecutor() { + final RaftProperties properties = server.getProperties(); + return ConcurrentUtils.newThreadPoolWithMax( + GrpcConfigKeys.Server.asyncRequestThreadPoolCached(properties), + GrpcConfigKeys.Server.asyncRequestThreadPoolSize(properties), + server.getId() + "-request-"); + } + + private GrpcClientProtocolService newGrpcClientProtocolService( + ExecutorService executor) { + return new GrpcClientProtocolService(server::getId, server, executor); + } + + private GrpcServerProtocolService newGrpcServerProtocolService() { + return new GrpcServerProtocolService(server::getId, server); + } + + private MetricServerInterceptor newMetricServerInterceptor() { + return new MetricServerInterceptor(server::getId, + JavaUtils.getClassSimpleName(getClass()) + "_" + serverPort); + } + + private NettyServerBuilder newNettyServerBuilderForServer() { + return newNettyServerBuilder(serverHost, serverPort, serverTlsConfig); + } + + private NettyServerBuilder newNettyServerBuilderForAdmin() { + return newNettyServerBuilder(adminHost, adminPort, adminTlsConfig); + } + + private NettyServerBuilder newNettyServerBuilderForClient() { + return newNettyServerBuilder(clientHost, clientPort, clientTlsConfig); + } + + private NettyServerBuilder newNettyServerBuilder(String hostname, int port, GrpcTlsConfig tlsConfig) { + final InetSocketAddress address = hostname == null || hostname.isEmpty() ? + new InetSocketAddress(port) : new InetSocketAddress(hostname, port); + final NettyServerBuilder nettyServerBuilder = NettyServerBuilder.forAddress(address) + .withChildOption(ChannelOption.SO_REUSEADDR, true) + .maxInboundMessageSize(messageSizeMax.getSizeInt()) + .flowControlWindow(flowControlWindow.getSizeInt()); + + if (tlsConfig != null) { + SslContextBuilder sslContextBuilder = GrpcUtil.initSslContextBuilderForServer(tlsConfig.getKeyManager()); + if (tlsConfig.getMtlsEnabled()) { + sslContextBuilder.clientAuth(ClientAuth.REQUIRE); + GrpcUtil.setTrustManager(sslContextBuilder, tlsConfig.getTrustManager()); + } + sslContextBuilder = GrpcSslContexts.configure(sslContextBuilder, OPENSSL); + try { + nettyServerBuilder.sslContext(sslContextBuilder.build()); + } catch (Exception ex) { + throw new IllegalArgumentException("Failed to build SslContext, tlsConfig=" + tlsConfig, ex); + } + } + return nettyServerBuilder; + } + + private boolean separateAdminServer() { + return adminPort > 0 && adminPort != serverPort; + } + + private boolean separateClientServer() { + return clientPort > 0 && clientPort != serverPort; + } + + Server newServer(GrpcClientProtocolService client, ServerInterceptor interceptor) { + final NettyServerBuilder serverBuilder = newNettyServerBuilderForServer(); + final GrpcServerProtocolService service = newGrpcServerProtocolService(); + serverBuilder.addService(ServerInterceptors.intercept(service, interceptor)); + + if (!separateAdminServer()) { + addAdminService(serverBuilder, server, interceptor); + } + if (!separateClientServer()) { + addClientService(serverBuilder, client, interceptor); + } + return serverBuilder.build(); + } + public GrpcService build() { - return new GrpcService(server, adminTlsConfig, clientTlsConfig, serverTlsConfig); + return new GrpcService(this); } public Builder setAdminTlsConfig(GrpcTlsConfig config) { @@ -147,136 +270,51 @@ public MetricServerInterceptor getServerInterceptor() { return serverInterceptor; } - private GrpcService(RaftServer server, - GrpcTlsConfig adminTlsConfig, GrpcTlsConfig clientTlsConfig, GrpcTlsConfig serverTlsConfig) { - this(server, server::getId, - GrpcConfigKeys.Admin.host(server.getProperties()), - GrpcConfigKeys.Admin.port(server.getProperties()), - adminTlsConfig, - GrpcConfigKeys.Client.host(server.getProperties()), - GrpcConfigKeys.Client.port(server.getProperties()), - clientTlsConfig, - GrpcConfigKeys.Server.host(server.getProperties()), - GrpcConfigKeys.Server.port(server.getProperties()), - serverTlsConfig, - GrpcConfigKeys.messageSizeMax(server.getProperties(), LOG::info), - RaftServerConfigKeys.Log.Appender.bufferByteLimit(server.getProperties()), - GrpcConfigKeys.flowControlWindow(server.getProperties(), LOG::info), - RaftServerConfigKeys.Rpc.requestTimeout(server.getProperties()), - GrpcConfigKeys.Server.heartbeatChannel(server.getProperties())); - } - - @SuppressWarnings("checkstyle:ParameterNumber") // private constructor - private GrpcService(RaftServer raftServer, Supplier idSupplier, - String adminHost, int adminPort, GrpcTlsConfig adminTlsConfig, - String clientHost, int clientPort, GrpcTlsConfig clientTlsConfig, - String serverHost, int serverPort, GrpcTlsConfig serverTlsConfig, - SizeInBytes grpcMessageSizeMax, SizeInBytes appenderBufferSize, - SizeInBytes flowControlWindow,TimeDuration requestTimeoutDuration, - boolean useSeparateHBChannel) { - super(idSupplier, id -> new PeerProxyMap<>(id.toString(), - p -> new GrpcServerProtocolClient(p, flowControlWindow.getSizeInt(), - requestTimeoutDuration, serverTlsConfig, useSeparateHBChannel))); - - final SizeInBytes gap = SizeInBytes.ONE_MB; - final long diff = grpcMessageSizeMax.getSize() - appenderBufferSize.getSize(); - if (diff < gap.getSize()) { - throw new IllegalArgumentException("Illegal configuration: " - + GrpcConfigKeys.MESSAGE_SIZE_MAX_KEY + "(= " + grpcMessageSizeMax - + ") must be " + gap + " larger than " - + RaftServerConfigKeys.Log.Appender.BUFFER_BYTE_LIMIT_KEY + "(= " + appenderBufferSize + ")."); - } + private GrpcService(Builder b) { + super(b.server::getId, id -> new PeerProxyMap<>(id.toString(), b::newGrpcServerProtocolClient)); - final RaftProperties properties = raftServer.getProperties(); - this.executor = ConcurrentUtils.newThreadPoolWithMax( - GrpcConfigKeys.Server.asyncRequestThreadPoolCached(properties), - GrpcConfigKeys.Server.asyncRequestThreadPoolSize(properties), - getId() + "-request-"); - this.clientProtocolService = new GrpcClientProtocolService(idSupplier, raftServer, executor); - - this.serverInterceptor = new MetricServerInterceptor( - idSupplier, - JavaUtils.getClassSimpleName(getClass()) + "_" + serverPort - ); - - final boolean separateAdminServer = adminPort != serverPort && adminPort > 0; - final boolean separateClientServer = clientPort != serverPort && clientPort > 0; - - final NettyServerBuilder serverBuilder = - startBuildingNettyServer(serverHost, serverPort, serverTlsConfig, grpcMessageSizeMax, flowControlWindow); - serverBuilder.addService(ServerInterceptors.intercept( - new GrpcServerProtocolService(idSupplier, raftServer), serverInterceptor)); - if (!separateAdminServer) { - addAdminService(raftServer, serverBuilder); - } - if (!separateClientServer) { - addClientService(serverBuilder); - } + this.executor = b.newExecutor(); + this.clientProtocolService = b.newGrpcClientProtocolService(executor); + this.serverInterceptor = b.newMetricServerInterceptor(); + final Server server = b.newServer(clientProtocolService, serverInterceptor); - final Server server = serverBuilder.build(); servers.put(GrpcServerProtocolService.class.getSimpleName(), server); - addressSupplier = newAddressSupplier(serverPort, server); + addressSupplier = newAddressSupplier(b.serverPort, server); - if (separateAdminServer) { - final NettyServerBuilder builder = - startBuildingNettyServer(adminHost, adminPort, adminTlsConfig, grpcMessageSizeMax, flowControlWindow); - addAdminService(raftServer, builder); + if (b.separateAdminServer()) { + final NettyServerBuilder builder = b.newNettyServerBuilderForAdmin(); + addAdminService(builder, b.server, serverInterceptor); final Server adminServer = builder.build(); servers.put(GrpcAdminProtocolService.class.getName(), adminServer); - adminServerAddressSupplier = newAddressSupplier(adminPort, adminServer); + adminServerAddressSupplier = newAddressSupplier(b.adminPort, adminServer); } else { adminServerAddressSupplier = addressSupplier; } - if (separateClientServer) { - final NettyServerBuilder builder = - startBuildingNettyServer(clientHost, clientPort, clientTlsConfig, grpcMessageSizeMax, flowControlWindow); - addClientService(builder); + if (b.separateClientServer()) { + final NettyServerBuilder builder = b.newNettyServerBuilderForClient(); + addClientService(builder, clientProtocolService, serverInterceptor); final Server clientServer = builder.build(); servers.put(GrpcClientProtocolService.class.getName(), clientServer); - clientServerAddressSupplier = newAddressSupplier(clientPort, clientServer); + clientServerAddressSupplier = newAddressSupplier(b.clientPort, clientServer); } else { clientServerAddressSupplier = addressSupplier; } } - private MemoizedSupplier newAddressSupplier(int port, Server server) { + static MemoizedSupplier newAddressSupplier(int port, Server server) { return JavaUtils.memoize(() -> new InetSocketAddress(port != 0 ? port : server.getPort())); } - private void addClientService(NettyServerBuilder builder) { - builder.addService(ServerInterceptors.intercept(clientProtocolService, serverInterceptor)); + static void addClientService(NettyServerBuilder builder, GrpcClientProtocolService client, + ServerInterceptor interceptor) { + builder.addService(ServerInterceptors.intercept(client, interceptor)); } - private void addAdminService(RaftServer raftServer, NettyServerBuilder nettyServerBuilder) { - nettyServerBuilder.addService(ServerInterceptors.intercept( - new GrpcAdminProtocolService(raftServer), - serverInterceptor)); - } - - private static NettyServerBuilder startBuildingNettyServer(String hostname, int port, GrpcTlsConfig tlsConfig, - SizeInBytes grpcMessageSizeMax, SizeInBytes flowControlWindow) { - InetSocketAddress address = hostname == null || hostname.isEmpty() ? - new InetSocketAddress(port) : new InetSocketAddress(hostname, port); - NettyServerBuilder nettyServerBuilder = NettyServerBuilder.forAddress(address) - .withChildOption(ChannelOption.SO_REUSEADDR, true) - .maxInboundMessageSize(grpcMessageSizeMax.getSizeInt()) - .flowControlWindow(flowControlWindow.getSizeInt()); - - if (tlsConfig != null) { - SslContextBuilder sslContextBuilder = GrpcUtil.initSslContextBuilderForServer(tlsConfig.getKeyManager()); - if (tlsConfig.getMtlsEnabled()) { - sslContextBuilder.clientAuth(ClientAuth.REQUIRE); - GrpcUtil.setTrustManager(sslContextBuilder, tlsConfig.getTrustManager()); - } - sslContextBuilder = GrpcSslContexts.configure(sslContextBuilder, OPENSSL); - try { - nettyServerBuilder.sslContext(sslContextBuilder.build()); - } catch (Exception ex) { - throw new IllegalArgumentException("Failed to build SslContext, tlsConfig=" + tlsConfig, ex); - } - } - return nettyServerBuilder; + static void addAdminService(NettyServerBuilder builder, AdminAsynchronousProtocol admin, + ServerInterceptor interceptor) { + final GrpcAdminProtocolService service = new GrpcAdminProtocolService(admin); + builder.addService(ServerInterceptors.intercept(service, interceptor)); } @Override From 973f6a7768a7742a843a711011125e1e79af2728 Mon Sep 17 00:00:00 2001 From: Flyangz Date: Thu, 22 Aug 2024 00:10:04 +0800 Subject: [PATCH 080/397] RATIS-2137. Fix LogAppenderDefault in handling of INCONSISTENCY. (#1136) --- .../server/leader/LogAppenderDefault.java | 23 ++++++++++++------- 1 file changed, 15 insertions(+), 8 deletions(-) diff --git a/ratis-server/src/main/java/org/apache/ratis/server/leader/LogAppenderDefault.java b/ratis-server/src/main/java/org/apache/ratis/server/leader/LogAppenderDefault.java index 21ef70d4df..1b21bb7e48 100644 --- a/ratis-server/src/main/java/org/apache/ratis/server/leader/LogAppenderDefault.java +++ b/ratis-server/src/main/java/org/apache/ratis/server/leader/LogAppenderDefault.java @@ -23,6 +23,7 @@ import org.apache.ratis.proto.RaftProtos.InstallSnapshotRequestProto; import org.apache.ratis.rpc.CallId; import org.apache.ratis.server.RaftServer; +import org.apache.ratis.server.raftlog.RaftLog; import org.apache.ratis.server.raftlog.RaftLogIOException; import org.apache.ratis.server.util.ServerStringUtils; import org.apache.ratis.statemachine.SnapshotInfo; @@ -33,6 +34,7 @@ import java.util.Comparator; import java.util.UUID; import java.util.concurrent.TimeUnit; +import java.util.concurrent.atomic.AtomicLong; /** * The default implementation of {@link LogAppender} @@ -54,7 +56,7 @@ public Comparator getCallIdComparator() { } /** Send an appendEntries RPC; retry indefinitely. */ - private AppendEntriesReplyProto sendAppendEntriesWithRetries() + private AppendEntriesReplyProto sendAppendEntriesWithRetries(AtomicLong requestFirstIndex) throws InterruptedException, InterruptedIOException, RaftLogIOException { int retry = 0; @@ -76,12 +78,15 @@ private AppendEntriesReplyProto sendAppendEntriesWithRetries() resetHeartbeatTrigger(); final Timestamp sendTime = Timestamp.currentTime(); getFollower().updateLastRpcSendTime(request.getEntriesCount() == 0); - final AppendEntriesReplyProto r = getServerRpc().appendEntries(request); + final AppendEntriesRequestProto proto = request; + final AppendEntriesReplyProto reply = getServerRpc().appendEntries(proto); + final long first = proto.getEntriesCount() > 0 ? proto.getEntries(0).getIndex() : RaftLog.INVALID_LOG_INDEX; + requestFirstIndex.set(first); getFollower().updateLastRpcResponseTime(); getFollower().updateLastRespondedAppendEntriesSendTime(sendTime); - getLeaderState().onFollowerCommitIndex(getFollower(), r.getFollowerCommit()); - return r; + getLeaderState().onFollowerCommitIndex(getFollower(), reply.getFollowerCommit()); + return reply; } catch (InterruptedIOException | RaftLogIOException e) { throw e; } catch (IOException ioe) { @@ -153,9 +158,10 @@ public void run() throws InterruptedException, IOException { } // otherwise if r is null, retry the snapshot installation } else { - final AppendEntriesReplyProto r = sendAppendEntriesWithRetries(); + final AtomicLong requestFirstIndex = new AtomicLong(RaftLog.INVALID_LOG_INDEX); + final AppendEntriesReplyProto r = sendAppendEntriesWithRetries(requestFirstIndex); if (r != null) { - handleReply(r); + handleReply(r, requestFirstIndex.get()); } } } @@ -166,7 +172,8 @@ public void run() throws InterruptedException, IOException { } } - private void handleReply(AppendEntriesReplyProto reply) throws IllegalArgumentException { + private void handleReply(AppendEntriesReplyProto reply, long requestFirstIndex) + throws IllegalArgumentException { if (reply != null) { switch (reply.getResult()) { case SUCCESS: @@ -189,7 +196,7 @@ private void handleReply(AppendEntriesReplyProto reply) throws IllegalArgumentEx onFollowerTerm(reply.getTerm()); break; case INCONSISTENCY: - getFollower().decreaseNextIndex(reply.getNextIndex()); + getFollower().setNextIndex(getNextIndexForInconsistency(requestFirstIndex, reply.getNextIndex())); break; case UNRECOGNIZED: LOG.warn("{}: received {}", this, reply.getResult()); From e5f2133f1d02738e7a5952327a568d208d6e0272 Mon Sep 17 00:00:00 2001 From: Potato Date: Thu, 22 Aug 2024 10:42:39 +0800 Subject: [PATCH 081/397] RATIS-2144. SegmentedRaftLogWorker should close the stream before releasing the buffer. (#1138) --- .../ratis/server/raftlog/segmented/SegmentedRaftLogWorker.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/ratis-server/src/main/java/org/apache/ratis/server/raftlog/segmented/SegmentedRaftLogWorker.java b/ratis-server/src/main/java/org/apache/ratis/server/raftlog/segmented/SegmentedRaftLogWorker.java index 573270cca5..5b0470d4fa 100644 --- a/ratis-server/src/main/java/org/apache/ratis/server/raftlog/segmented/SegmentedRaftLogWorker.java +++ b/ratis-server/src/main/java/org/apache/ratis/server/raftlog/segmented/SegmentedRaftLogWorker.java @@ -246,8 +246,8 @@ void close() { Optional.ofNullable(flushExecutor).ifPresent(ExecutorService::shutdown); ConcurrentUtils.shutdownAndWait(TimeDuration.ONE_SECOND.multiply(3), workerThreadExecutor, timeout -> LOG.warn("{}: shutdown timeout in " + timeout, name)); - PlatformDependent.freeDirectBuffer(writeBuffer); IOUtils.cleanup(LOG, out); + PlatformDependent.freeDirectBuffer(writeBuffer); LOG.info("{} close()", name); } From 47c3a829ed6863fdd74636cf65ae1c26503ee571 Mon Sep 17 00:00:00 2001 From: z-bb <35120306+z-bb@users.noreply.github.com> Date: Thu, 22 Aug 2024 23:37:09 +0800 Subject: [PATCH 082/397] RATIS-2140. Thread wait when installing snapshot. (#1137) --- .../java/org/apache/ratis/grpc/server/GrpcLogAppender.java | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/ratis-grpc/src/main/java/org/apache/ratis/grpc/server/GrpcLogAppender.java b/ratis-grpc/src/main/java/org/apache/ratis/grpc/server/GrpcLogAppender.java index 88e28dc48b..8580c42396 100644 --- a/ratis-grpc/src/main/java/org/apache/ratis/grpc/server/GrpcLogAppender.java +++ b/ratis-grpc/src/main/java/org/apache/ratis/grpc/server/GrpcLogAppender.java @@ -770,7 +770,7 @@ private void installSnapshot(SnapshotInfo snapshot) { while (isRunning() && !responseHandler.isDone()) { try { - getEventAwaitForSignal().await(); + getEventAwaitForSignal().await(getWaitTimeMs(), TimeUnit.MILLISECONDS); } catch (InterruptedException ignored) { Thread.currentThread().interrupt(); } @@ -815,7 +815,7 @@ private void notifyInstallSnapshot(TermIndex firstAvailable) { while (isRunning() && !responseHandler.isDone()) { try { - getEventAwaitForSignal().await(); + getEventAwaitForSignal().await(getWaitTimeMs(), TimeUnit.MILLISECONDS); } catch (InterruptedException ignored) { Thread.currentThread().interrupt(); } From 189c3fd2bd8bf67b7ec7cc6f25ed9673c230ebb6 Mon Sep 17 00:00:00 2001 From: Tsz-Wo Nicholas Sze Date: Thu, 22 Aug 2024 09:44:24 -0700 Subject: [PATCH 083/397] RATIS-2113. Use consistent method names and parameter types in RaftUtils. (#1135) --- .../cli/{RaftUtils.java => CliUtils.java} | 129 +++++++++--------- .../shell/cli/sh/command/AbstractCommand.java | 13 -- .../cli/sh/command/AbstractRatisCommand.java | 25 ++-- .../shell/cli/sh/election/PauseCommand.java | 4 +- .../shell/cli/sh/election/ResumeCommand.java | 4 +- .../cli/sh/election/StepDownCommand.java | 4 +- .../cli/sh/election/TransferCommand.java | 4 +- .../shell/cli/sh/group/GroupListCommand.java | 8 +- .../cli/sh/local/RaftMetaConfCommand.java | 6 +- .../ratis/shell/cli/sh/peer/AddCommand.java | 13 +- .../shell/cli/sh/peer/RemoveCommand.java | 4 +- .../shell/cli/sh/peer/SetPriorityCommand.java | 4 +- .../cli/sh/snapshot/TakeSnapshotCommand.java | 4 +- 13 files changed, 101 insertions(+), 121 deletions(-) rename ratis-shell/src/main/java/org/apache/ratis/shell/cli/{RaftUtils.java => CliUtils.java} (62%) diff --git a/ratis-shell/src/main/java/org/apache/ratis/shell/cli/RaftUtils.java b/ratis-shell/src/main/java/org/apache/ratis/shell/cli/CliUtils.java similarity index 62% rename from ratis-shell/src/main/java/org/apache/ratis/shell/cli/RaftUtils.java rename to ratis-shell/src/main/java/org/apache/ratis/shell/cli/CliUtils.java index 9c5d90e4bf..98e21be2a8 100644 --- a/ratis-shell/src/main/java/org/apache/ratis/shell/cli/RaftUtils.java +++ b/ratis-shell/src/main/java/org/apache/ratis/shell/cli/CliUtils.java @@ -37,7 +37,6 @@ import java.util.ArrayList; import java.util.Collection; import java.util.concurrent.TimeUnit; -import java.util.function.Consumer; import java.util.List; import java.util.Optional; import java.util.Properties; @@ -46,43 +45,31 @@ import java.util.UUID; /** - * Helper class for raft operations. + * Utilities for command line interface. */ -public final class RaftUtils { - - public static final RaftGroupId DEFAULT_RAFT_GROUP_ID = RaftGroupId.randomId(); - - private RaftUtils() { +public final class CliUtils { + private static final ExponentialBackoffRetry RETRY_POLICY = ExponentialBackoffRetry.newBuilder() + .setBaseSleepTime(TimeDuration.valueOf(1000, TimeUnit.MILLISECONDS)) + .setMaxAttempts(10) + .setMaxSleepTime(TimeDuration.valueOf(100_000, TimeUnit.MILLISECONDS)) + .build(); + + private CliUtils() { // prevent instantiation } - /** - * Gets the raft peer id. - * - * @param address the address of the server - * @return the raft peer id - */ + /** @return {@link RaftPeerId} from the given address. */ public static RaftPeerId getPeerId(InetSocketAddress address) { return getPeerId(address.getHostString(), address.getPort()); } - /** - * Gets the raft peer id. - * - * @param host the hostname of the server - * @param port the port of the server - * @return the raft peer id - */ + /** @return {@link RaftPeerId} from the given host and port. */ public static RaftPeerId getPeerId(String host, int port) { return RaftPeerId.getRaftPeerId(host + "_" + port); } - /** - * Create a raft client to communicate to ratis server. - * @param raftGroup the raft group - * @return return a raft client - */ - public static RaftClient createClient(RaftGroup raftGroup) { + /** Create a new {@link RaftClient} from the given group. */ + public static RaftClient newRaftClient(RaftGroup group) { RaftProperties properties = new RaftProperties(); RaftClientConfigKeys.Rpc.setRequestTimeout(properties, TimeDuration.valueOf(15, TimeUnit.SECONDS)); @@ -92,16 +79,10 @@ public static RaftClient createClient(RaftGroup raftGroup) { final Properties sys = System.getProperties(); sys.stringPropertyNames().forEach(key -> properties.set(key, sys.getProperty(key))); - ExponentialBackoffRetry retryPolicy = ExponentialBackoffRetry.newBuilder() - .setBaseSleepTime(TimeDuration.valueOf(1000, TimeUnit.MILLISECONDS)) - .setMaxAttempts(10) - .setMaxSleepTime( - TimeDuration.valueOf(100_000, TimeUnit.MILLISECONDS)) - .build(); return RaftClient.newBuilder() - .setRaftGroup(raftGroup) + .setRaftGroup(group) .setProperties(properties) - .setRetryPolicy(retryPolicy) + .setRetryPolicy(RETRY_POLICY) .build(); } @@ -116,7 +97,7 @@ public static RaftClient createClient(RaftGroup raftGroup) { * @return the first non-null value returned by the given function applied to the given list. */ private static RETURN applyFunctionReturnFirstNonNull( - Collection list, CheckedFunction function) { + Collection list, CheckedFunction function, PrintStream out) { for (PARAMETER parameter : list) { try { RETURN ret = function.apply(parameter); @@ -124,13 +105,14 @@ private static RETURN applyFunc return ret; } } catch (Throwable e) { - e.printStackTrace(); + e.printStackTrace(out); } } return null; } - public static List buildRaftPeersFromStr(String peers) { + /** Parse the given string as a list of {@link RaftPeer}. */ + public static List parseRaftPeers(String peers) { List addresses = new ArrayList<>(); String[] peersArray = peers.split(","); for (String peer : peersArray) { @@ -138,64 +120,79 @@ public static List buildRaftPeersFromStr(String peers) { } return addresses.stream() - .map(addr -> RaftPeer.newBuilder() - .setId(RaftUtils.getPeerId(addr)) - .setAddress(addr) - .build() - ).collect(Collectors.toList()); + .map(addr -> RaftPeer.newBuilder().setId(getPeerId(addr)).setAddress(addr).build()) + .collect(Collectors.toList()); } - public static RaftGroupId buildRaftGroupIdFromStr(String groupId) { - return groupId != null && groupId.isEmpty() ? RaftGroupId.valueOf(UUID.fromString(groupId)) - : DEFAULT_RAFT_GROUP_ID; + /** Parse the given string as a {@link RaftGroupId}. */ + public static RaftGroupId parseRaftGroupId(String groupId) { + return groupId != null && groupId.isEmpty() ? RaftGroupId.valueOf(UUID.fromString(groupId)) : null; } - public static RaftGroupId retrieveRemoteGroupId(RaftGroupId raftGroupIdFromConfig, - List peers, - RaftClient client, PrintStream printStream) throws IOException { - if (!DEFAULT_RAFT_GROUP_ID .equals(raftGroupIdFromConfig)) { - return raftGroupIdFromConfig; + /** + * Get the group id from the given peers if the given group id is null. + * + * @param client for communicating to the peers. + * @param peers the peers of the group. + * @param groupId the given group id, if there is any. + * @param err for printing error messages. + * @return the group id from the given peers if the given group id is null; + * otherwise, return the given group id. + */ + public static RaftGroupId getGroupId(RaftClient client, List peers, RaftGroupId groupId, + PrintStream err) throws IOException { + if (groupId != null) { + return groupId; } - final RaftGroupId remoteGroupId; final List groupIds = applyFunctionReturnFirstNonNull(peers, - p -> client.getGroupManagementApi((p.getId())).list().getGroupIds()); + p -> client.getGroupManagementApi(p.getId()).list().getGroupIds(), err); if (groupIds == null) { - printStream.println("Failed to get group ID from " + peers); - throw new IOException("Failed to get group ID from " + peers); + final String message = "Failed to get group ID from " + peers; + err.println("Failed to get group ID from " + peers); + throw new IOException(message); } else if (groupIds.size() == 1) { - remoteGroupId = groupIds.get(0); + return groupIds.get(0); } else { String message = "Unexpected multiple group IDs " + groupIds + ". In such case, the target group ID must be specified."; - printStream.println(message); + err.println(message); throw new IOException(message); } - return remoteGroupId; } - public static GroupInfoReply retrieveGroupInfoByGroupId(RaftGroupId remoteGroupId, List peers, - RaftClient client, PrintStream printStream) - throws IOException { + /** + * Get the group info from the given peers. + * + * @param client for communicating to the peers. + * @param peers the peers of the group. + * @param groupId the target group + * @param err for printing error messages. + * @return the group info + */ + public static GroupInfoReply getGroupInfo(RaftClient client, List peers, RaftGroupId groupId, + PrintStream err) throws IOException { GroupInfoReply groupInfoReply = applyFunctionReturnFirstNonNull(peers, - p -> client.getGroupManagementApi((p.getId())).info(remoteGroupId)); - processReply(groupInfoReply, printStream::println, - () -> "Failed to get group info for group id " + remoteGroupId.getUuid() + " from " + peers); + p -> client.getGroupManagementApi((p.getId())).info(groupId), err); + checkReply(groupInfoReply, () -> "Failed to get group info for " + groupId.getUuid() + + " from " + peers, err); return groupInfoReply; } - public static void processReply(RaftClientReply reply, Consumer printer, Supplier message) + /** Check if the given reply is success. */ + public static void checkReply(RaftClientReply reply, Supplier message, PrintStream printStream) throws IOException { if (reply == null || !reply.isSuccess()) { final RaftException e = Optional.ofNullable(reply) .map(RaftClientReply::getException) .orElseGet(() -> new RaftException("Reply: " + reply)); - printer.accept(message.get()); - throw new IOException(e.getMessage(), e); + printStream.println(message.get()); + throw new IOException(message.get(), e); } } + /** Parse the given string as a {@link InetSocketAddress}. */ public static InetSocketAddress parseInetSocketAddress(String address) { try { final String[] hostPortPair = address.split(":"); diff --git a/ratis-shell/src/main/java/org/apache/ratis/shell/cli/sh/command/AbstractCommand.java b/ratis-shell/src/main/java/org/apache/ratis/shell/cli/sh/command/AbstractCommand.java index 20a52a80f8..f02761de4d 100644 --- a/ratis-shell/src/main/java/org/apache/ratis/shell/cli/sh/command/AbstractCommand.java +++ b/ratis-shell/src/main/java/org/apache/ratis/shell/cli/sh/command/AbstractCommand.java @@ -20,7 +20,6 @@ import org.apache.ratis.shell.cli.Command; import java.io.PrintStream; -import java.net.InetSocketAddress; /** * The base class for all the ratis shell {@link Command} classes. @@ -33,18 +32,6 @@ protected AbstractCommand(Context context) { printStream = context.getPrintStream(); } - public static InetSocketAddress parseInetSocketAddress(String address) { - try { - final String[] hostPortPair = address.split(":"); - if (hostPortPair.length < 2) { - throw new IllegalArgumentException("Unexpected address format ."); - } - return new InetSocketAddress(hostPortPair[0], Integer.parseInt(hostPortPair[1])); - } catch (Exception e) { - throw new IllegalArgumentException("Failed to parse the server address parameter \"" + address + "\".", e); - } - } - protected PrintStream getPrintStream() { return printStream; } diff --git a/ratis-shell/src/main/java/org/apache/ratis/shell/cli/sh/command/AbstractRatisCommand.java b/ratis-shell/src/main/java/org/apache/ratis/shell/cli/sh/command/AbstractRatisCommand.java index 91bdc873b7..a9d391f866 100644 --- a/ratis-shell/src/main/java/org/apache/ratis/shell/cli/sh/command/AbstractRatisCommand.java +++ b/ratis-shell/src/main/java/org/apache/ratis/shell/cli/sh/command/AbstractRatisCommand.java @@ -24,7 +24,7 @@ import org.apache.ratis.protocol.RaftPeer; import org.apache.ratis.protocol.RaftPeerId; import org.apache.ratis.protocol.GroupInfoReply; -import org.apache.ratis.shell.cli.RaftUtils; +import org.apache.ratis.shell.cli.CliUtils; import org.apache.commons.cli.CommandLine; import org.apache.commons.cli.Options; import org.apache.ratis.client.RaftClient; @@ -47,11 +47,6 @@ import java.util.stream.Collectors; import java.util.stream.Stream; -import static org.apache.ratis.shell.cli.RaftUtils.buildRaftGroupIdFromStr; -import static org.apache.ratis.shell.cli.RaftUtils.buildRaftPeersFromStr; -import static org.apache.ratis.shell.cli.RaftUtils.retrieveGroupInfoByGroupId; -import static org.apache.ratis.shell.cli.RaftUtils.retrieveRemoteGroupId; - /** * The base class for the ratis shell which need to connect to server. */ @@ -67,13 +62,13 @@ protected AbstractRatisCommand(Context context) { @Override public int run(CommandLine cl) throws IOException { - List peers = buildRaftPeersFromStr(cl.getOptionValue(PEER_OPTION_NAME)); - RaftGroupId raftGroupIdFromConfig = buildRaftGroupIdFromStr(cl.getOptionValue(GROUPID_OPTION_NAME)); - raftGroup = RaftGroup.valueOf(raftGroupIdFromConfig, peers); + final List peers = CliUtils.parseRaftPeers(cl.getOptionValue(PEER_OPTION_NAME)); + final RaftGroupId groupIdSpecified = CliUtils.parseRaftGroupId(cl.getOptionValue(GROUPID_OPTION_NAME)); + raftGroup = RaftGroup.valueOf(groupIdSpecified != null? groupIdSpecified: RaftGroupId.randomId(), peers); PrintStream printStream = getPrintStream(); - try (final RaftClient client = RaftUtils.createClient(raftGroup)) { - final RaftGroupId remoteGroupId = retrieveRemoteGroupId(raftGroupIdFromConfig, peers, client, printStream); - groupInfoReply = retrieveGroupInfoByGroupId(remoteGroupId, peers, client, printStream); + try (final RaftClient client = CliUtils.newRaftClient(raftGroup)) { + final RaftGroupId remoteGroupId = CliUtils.getGroupId(client, peers, groupIdSpecified, printStream); + groupInfoReply = CliUtils.getGroupInfo(client, peers, remoteGroupId, printStream); raftGroup = groupInfoReply.getGroup(); } return 0; @@ -121,7 +116,7 @@ protected RaftPeerProto getLeader(RoleInfoProto roleInfo) { } protected void processReply(RaftClientReply reply, Supplier messageSupplier) throws IOException { - RaftUtils.processReply(reply, getPrintStream()::println, messageSupplier); + CliUtils.checkReply(reply, messageSupplier, getPrintStream()); } protected List getIds(String[] optionValues, BiConsumer consumer) { @@ -130,8 +125,8 @@ protected List getIds(String[] optionValues, BiConsumer ids = new ArrayList<>(); for (String address : optionValues) { - final InetSocketAddress serverAddress = parseInetSocketAddress(address); - final RaftPeerId peerId = RaftUtils.getPeerId(serverAddress); + final InetSocketAddress serverAddress = CliUtils.parseInetSocketAddress(address); + final RaftPeerId peerId = CliUtils.getPeerId(serverAddress); consumer.accept(peerId, serverAddress); ids.add(peerId); } diff --git a/ratis-shell/src/main/java/org/apache/ratis/shell/cli/sh/election/PauseCommand.java b/ratis-shell/src/main/java/org/apache/ratis/shell/cli/sh/election/PauseCommand.java index 4ea2969bac..242e1886a2 100644 --- a/ratis-shell/src/main/java/org/apache/ratis/shell/cli/sh/election/PauseCommand.java +++ b/ratis-shell/src/main/java/org/apache/ratis/shell/cli/sh/election/PauseCommand.java @@ -24,7 +24,7 @@ import org.apache.ratis.protocol.RaftClientReply; import org.apache.ratis.protocol.RaftPeer; import org.apache.ratis.protocol.RaftPeerId; -import org.apache.ratis.shell.cli.RaftUtils; +import org.apache.ratis.shell.cli.CliUtils; import org.apache.ratis.shell.cli.sh.command.AbstractRatisCommand; import org.apache.ratis.shell.cli.sh.command.Context; @@ -61,7 +61,7 @@ public int run(CommandLine cl) throws IOException { printf("Peer not found: %s", strAddr); return -1; } - try(final RaftClient raftClient = RaftUtils.createClient(getRaftGroup())) { + try(final RaftClient raftClient = CliUtils.newRaftClient(getRaftGroup())) { RaftClientReply reply = raftClient.getLeaderElectionManagementApi(peerId).pause(); processReply(reply, () -> String.format("Failed to pause leader election on peer %s", strAddr)); printf(String.format("Successful pause leader election on peer %s", strAddr)); diff --git a/ratis-shell/src/main/java/org/apache/ratis/shell/cli/sh/election/ResumeCommand.java b/ratis-shell/src/main/java/org/apache/ratis/shell/cli/sh/election/ResumeCommand.java index 4b4dc225a0..dbcee7bd3f 100644 --- a/ratis-shell/src/main/java/org/apache/ratis/shell/cli/sh/election/ResumeCommand.java +++ b/ratis-shell/src/main/java/org/apache/ratis/shell/cli/sh/election/ResumeCommand.java @@ -24,7 +24,7 @@ import org.apache.ratis.protocol.RaftClientReply; import org.apache.ratis.protocol.RaftPeer; import org.apache.ratis.protocol.RaftPeerId; -import org.apache.ratis.shell.cli.RaftUtils; +import org.apache.ratis.shell.cli.CliUtils; import org.apache.ratis.shell.cli.sh.command.AbstractRatisCommand; import org.apache.ratis.shell.cli.sh.command.Context; @@ -61,7 +61,7 @@ public int run(CommandLine cl) throws IOException { printf("Can't find a sever with the address:%s", strAddr); return -1; } - try(final RaftClient raftClient = RaftUtils.createClient(getRaftGroup())) { + try(final RaftClient raftClient = CliUtils.newRaftClient(getRaftGroup())) { RaftClientReply reply = raftClient.getLeaderElectionManagementApi(peerId).resume(); processReply(reply, () -> String.format("Failed to resume leader election on peer %s", strAddr)); printf(String.format("Successful pause leader election on peer %s", strAddr)); diff --git a/ratis-shell/src/main/java/org/apache/ratis/shell/cli/sh/election/StepDownCommand.java b/ratis-shell/src/main/java/org/apache/ratis/shell/cli/sh/election/StepDownCommand.java index 911a2bb26a..da641a07ba 100644 --- a/ratis-shell/src/main/java/org/apache/ratis/shell/cli/sh/election/StepDownCommand.java +++ b/ratis-shell/src/main/java/org/apache/ratis/shell/cli/sh/election/StepDownCommand.java @@ -21,7 +21,7 @@ import org.apache.ratis.client.RaftClient; import org.apache.ratis.protocol.RaftClientReply; import org.apache.ratis.protocol.RaftPeerId; -import org.apache.ratis.shell.cli.RaftUtils; +import org.apache.ratis.shell.cli.CliUtils; import org.apache.ratis.shell.cli.sh.command.AbstractRatisCommand; import org.apache.ratis.shell.cli.sh.command.Context; @@ -48,7 +48,7 @@ public String getCommandName() { public int run(CommandLine cl) throws IOException { super.run(cl); - try (RaftClient client = RaftUtils.createClient(getRaftGroup())) { + try (RaftClient client = CliUtils.newRaftClient(getRaftGroup())) { RaftPeerId leaderId = RaftPeerId.valueOf(getLeader(getGroupInfoReply().getRoleInfoProto()).getId()); final RaftClientReply transferLeadershipReply = client.admin().transferLeadership(null, leaderId, 60_000); processReply(transferLeadershipReply, () -> "Failed to step down leader"); diff --git a/ratis-shell/src/main/java/org/apache/ratis/shell/cli/sh/election/TransferCommand.java b/ratis-shell/src/main/java/org/apache/ratis/shell/cli/sh/election/TransferCommand.java index c71d7f89f6..24eae7353a 100644 --- a/ratis-shell/src/main/java/org/apache/ratis/shell/cli/sh/election/TransferCommand.java +++ b/ratis-shell/src/main/java/org/apache/ratis/shell/cli/sh/election/TransferCommand.java @@ -25,7 +25,7 @@ import org.apache.ratis.protocol.RaftClientReply; import org.apache.ratis.protocol.RaftPeer; import org.apache.ratis.protocol.exceptions.TransferLeadershipException; -import org.apache.ratis.shell.cli.RaftUtils; +import org.apache.ratis.shell.cli.CliUtils; import org.apache.ratis.shell.cli.sh.command.AbstractRatisCommand; import org.apache.ratis.shell.cli.sh.command.Context; import org.apache.ratis.util.TimeDuration; @@ -74,7 +74,7 @@ public int run(CommandLine cl) throws IOException { printf("Peer with address %s not found.", strAddr); return -2; } - try (RaftClient client = RaftUtils.createClient(getRaftGroup())) { + try (RaftClient client = CliUtils.newRaftClient(getRaftGroup())) { // transfer leadership if (!tryTransfer(client, newLeader, highestPriority, timeout.orElse(timeoutDefault))) { // legacy mode, transfer leadership by setting priority. diff --git a/ratis-shell/src/main/java/org/apache/ratis/shell/cli/sh/group/GroupListCommand.java b/ratis-shell/src/main/java/org/apache/ratis/shell/cli/sh/group/GroupListCommand.java index 5bbd1939ad..5ee89c53cb 100644 --- a/ratis-shell/src/main/java/org/apache/ratis/shell/cli/sh/group/GroupListCommand.java +++ b/ratis-shell/src/main/java/org/apache/ratis/shell/cli/sh/group/GroupListCommand.java @@ -24,7 +24,7 @@ import org.apache.ratis.client.RaftClient; import org.apache.ratis.protocol.GroupListReply; import org.apache.ratis.protocol.RaftPeerId; -import org.apache.ratis.shell.cli.RaftUtils; +import org.apache.ratis.shell.cli.CliUtils; import org.apache.ratis.shell.cli.sh.command.AbstractRatisCommand; import org.apache.ratis.shell.cli.sh.command.Context; @@ -61,15 +61,15 @@ public int run(CommandLine cl) throws IOException { address = getRaftGroup().getPeer(peerId).getAddress(); } else if (cl.hasOption(SERVER_ADDRESS_OPTION_NAME)) { address = cl.getOptionValue(SERVER_ADDRESS_OPTION_NAME); - final InetSocketAddress serverAddress = parseInetSocketAddress(address); - peerId = RaftUtils.getPeerId(serverAddress); + final InetSocketAddress serverAddress = CliUtils.parseInetSocketAddress(address); + peerId = CliUtils.getPeerId(serverAddress); } else { throw new IllegalArgumentException( "Both " + PEER_ID_OPTION_NAME + " and " + SERVER_ADDRESS_OPTION_NAME + " options are missing."); } - try(final RaftClient raftClient = RaftUtils.createClient(getRaftGroup())) { + try(final RaftClient raftClient = CliUtils.newRaftClient(getRaftGroup())) { GroupListReply reply = raftClient.getGroupManagementApi(peerId).list(); processReply(reply, () -> String.format("Failed to get group information of peerId %s (server %s)", peerId, address)); diff --git a/ratis-shell/src/main/java/org/apache/ratis/shell/cli/sh/local/RaftMetaConfCommand.java b/ratis-shell/src/main/java/org/apache/ratis/shell/cli/sh/local/RaftMetaConfCommand.java index e258d863b8..a63b659375 100644 --- a/ratis-shell/src/main/java/org/apache/ratis/shell/cli/sh/local/RaftMetaConfCommand.java +++ b/ratis-shell/src/main/java/org/apache/ratis/shell/cli/sh/local/RaftMetaConfCommand.java @@ -25,7 +25,7 @@ import org.apache.ratis.proto.RaftProtos.RaftPeerProto; import org.apache.ratis.proto.RaftProtos.RaftPeerRole; import org.apache.ratis.protocol.RaftPeerId; -import org.apache.ratis.shell.cli.RaftUtils; +import org.apache.ratis.shell.cli.CliUtils; import org.apache.ratis.shell.cli.sh.command.AbstractCommand; import org.apache.ratis.shell.cli.sh.command.Context; import org.apache.ratis.thirdparty.com.google.protobuf.ByteString; @@ -89,7 +89,7 @@ public int run(CommandLine cl) throws IOException { printf(message, idWithAddress, peersStr); return -1; } - InetSocketAddress inetSocketAddress = parseInetSocketAddress( + InetSocketAddress inetSocketAddress = CliUtils.parseInetSocketAddress( peerIdWithAddressArray[peerIdWithAddressArray.length - 1]); String addressString = inetSocketAddress.getHostString() + ":" + inetSocketAddress.getPort(); if (addresses.contains(addressString)) { @@ -111,7 +111,7 @@ public int run(CommandLine cl) throws IOException { ids.add(peerId); } else { // If peer ID is not provided, use host address as peerId value - peerId = RaftUtils.getPeerId(inetSocketAddress).toString(); + peerId = CliUtils.getPeerId(inetSocketAddress).toString(); } raftPeerProtos.add(RaftPeerProto.newBuilder() diff --git a/ratis-shell/src/main/java/org/apache/ratis/shell/cli/sh/peer/AddCommand.java b/ratis-shell/src/main/java/org/apache/ratis/shell/cli/sh/peer/AddCommand.java index 3c65bb12de..62c6c57934 100644 --- a/ratis-shell/src/main/java/org/apache/ratis/shell/cli/sh/peer/AddCommand.java +++ b/ratis-shell/src/main/java/org/apache/ratis/shell/cli/sh/peer/AddCommand.java @@ -25,9 +25,10 @@ import org.apache.ratis.protocol.RaftClientReply; import org.apache.ratis.protocol.RaftPeer; import org.apache.ratis.protocol.RaftPeerId; -import org.apache.ratis.shell.cli.RaftUtils; +import org.apache.ratis.shell.cli.CliUtils; import org.apache.ratis.shell.cli.sh.command.AbstractRatisCommand; import org.apache.ratis.shell.cli.sh.command.Context; +import org.apache.ratis.util.Preconditions; import java.io.IOException; import java.net.InetSocketAddress; @@ -66,10 +67,10 @@ public int run(CommandLine cl) throws IOException { if (cl.hasOption(ADDRESS_OPTION_NAME) && cl.hasOption(PEER_ID_OPTION_NAME)) { ids = Arrays.stream(cl.getOptionValue(PEER_ID_OPTION_NAME).split(",")) .map(RaftPeerId::getRaftPeerId).collect(Collectors.toList()); - List addresses = - Arrays.stream(cl.getOptionValue(ADDRESS_OPTION_NAME).split(",")) - .map(s -> parseInetSocketAddress(s)).collect(Collectors.toList()); - assert ids.size() == addresses.size(); + final List addresses = Arrays.stream(cl.getOptionValue(ADDRESS_OPTION_NAME).split(",")) + .map(CliUtils::parseInetSocketAddress) + .collect(Collectors.toList()); + Preconditions.assertSame(ids.size(), addresses.size(), "size"); for (int i = 0; i < ids.size(); i++) { peersInfo.put(ids.get(i), addresses.get(i)); } @@ -80,7 +81,7 @@ public int run(CommandLine cl) throws IOException { "Both " + PEER_ID_OPTION_NAME + " and " + ADDRESS_OPTION_NAME + " options are missing."); } - try (RaftClient client = RaftUtils.createClient(getRaftGroup())) { + try (RaftClient client = CliUtils.newRaftClient(getRaftGroup())) { final Stream remaining = getPeerStream(RaftPeerRole.FOLLOWER); final Stream adding = ids.stream().map(raftPeerId -> RaftPeer.newBuilder() .setId(raftPeerId) diff --git a/ratis-shell/src/main/java/org/apache/ratis/shell/cli/sh/peer/RemoveCommand.java b/ratis-shell/src/main/java/org/apache/ratis/shell/cli/sh/peer/RemoveCommand.java index 5918516070..e2aa786b39 100644 --- a/ratis-shell/src/main/java/org/apache/ratis/shell/cli/sh/peer/RemoveCommand.java +++ b/ratis-shell/src/main/java/org/apache/ratis/shell/cli/sh/peer/RemoveCommand.java @@ -25,7 +25,7 @@ import org.apache.ratis.protocol.RaftClientReply; import org.apache.ratis.protocol.RaftPeer; import org.apache.ratis.protocol.RaftPeerId; -import org.apache.ratis.shell.cli.RaftUtils; +import org.apache.ratis.shell.cli.CliUtils; import org.apache.ratis.shell.cli.sh.command.AbstractRatisCommand; import org.apache.ratis.shell.cli.sh.command.Context; @@ -66,7 +66,7 @@ public int run(CommandLine cl) throws IOException { throw new IllegalArgumentException( "Both " + PEER_ID_OPTION_NAME + " and " + ADDRESS_OPTION_NAME + " options are missing."); } - try (RaftClient client = RaftUtils.createClient(getRaftGroup())) { + try (RaftClient client = CliUtils.newRaftClient(getRaftGroup())) { final List peers = getPeerStream(RaftPeerRole.FOLLOWER) .filter(raftPeer -> !ids.contains(raftPeer.getId())).collect(Collectors.toList()); final List listeners = getPeerStream(RaftPeerRole.LISTENER) diff --git a/ratis-shell/src/main/java/org/apache/ratis/shell/cli/sh/peer/SetPriorityCommand.java b/ratis-shell/src/main/java/org/apache/ratis/shell/cli/sh/peer/SetPriorityCommand.java index 01e81f3c34..e2d4d1a539 100644 --- a/ratis-shell/src/main/java/org/apache/ratis/shell/cli/sh/peer/SetPriorityCommand.java +++ b/ratis-shell/src/main/java/org/apache/ratis/shell/cli/sh/peer/SetPriorityCommand.java @@ -24,7 +24,7 @@ import org.apache.ratis.proto.RaftProtos.RaftPeerRole; import org.apache.ratis.protocol.RaftClientReply; import org.apache.ratis.protocol.RaftPeer; -import org.apache.ratis.shell.cli.RaftUtils; +import org.apache.ratis.shell.cli.CliUtils; import org.apache.ratis.shell.cli.sh.command.AbstractRatisCommand; import org.apache.ratis.shell.cli.sh.command.Context; @@ -63,7 +63,7 @@ public int run(CommandLine cl) throws IOException { addressPriorityMap.put(str[0], Integer.parseInt(str[1])); } - try (RaftClient client = RaftUtils.createClient(getRaftGroup())) { + try (RaftClient client = CliUtils.newRaftClient(getRaftGroup())) { final List peers = getPeerStream(RaftPeerRole.FOLLOWER).map(peer -> { final Integer newPriority = addressPriorityMap.get(peer.getAddress()); final int priority = newPriority != null ? newPriority : peer.getPriority(); diff --git a/ratis-shell/src/main/java/org/apache/ratis/shell/cli/sh/snapshot/TakeSnapshotCommand.java b/ratis-shell/src/main/java/org/apache/ratis/shell/cli/sh/snapshot/TakeSnapshotCommand.java index 10bac34975..e76f215f45 100644 --- a/ratis-shell/src/main/java/org/apache/ratis/shell/cli/sh/snapshot/TakeSnapshotCommand.java +++ b/ratis-shell/src/main/java/org/apache/ratis/shell/cli/sh/snapshot/TakeSnapshotCommand.java @@ -23,7 +23,7 @@ import org.apache.ratis.client.RaftClient; import org.apache.ratis.protocol.RaftClientReply; import org.apache.ratis.protocol.RaftPeerId; -import org.apache.ratis.shell.cli.RaftUtils; +import org.apache.ratis.shell.cli.CliUtils; import org.apache.ratis.shell.cli.sh.command.AbstractRatisCommand; import org.apache.ratis.shell.cli.sh.command.Context; @@ -58,7 +58,7 @@ public int run(CommandLine cl) throws IOException { } else { timeout = 3000; } - try(final RaftClient raftClient = RaftUtils.createClient(getRaftGroup())) { + try(final RaftClient raftClient = CliUtils.newRaftClient(getRaftGroup())) { if (cl.hasOption(PEER_ID_OPTION_NAME)) { peerId = RaftPeerId.getRaftPeerId(cl.getOptionValue(PEER_ID_OPTION_NAME)); } else { From 16ff0f0543952fd72d8c84c829bc8cf92f7ec46f Mon Sep 17 00:00:00 2001 From: Tsz-Wo Nicholas Sze Date: Wed, 21 Aug 2024 08:57:20 -0700 Subject: [PATCH 084/397] Revert "RATIS-1983. Refactor client request processing to support reference count. (#998)" This reverts commit 43a02109d1f0b34dd80b1e36f1d023f86f24e3ab. --- .../RaftClientAsynchronousProtocol.java | 34 +---- .../ratis/util/ReferenceCountedObject.java | 24 ---- .../filestore/FileStoreStateMachine.java | 18 +-- .../statemachine/TransactionContext.java | 8 -- .../ratis/server/impl/RaftServerImpl.java | 86 ++++-------- .../ratis/server/impl/RaftServerProxy.java | 13 +- .../server/raftlog/segmented/LogSegment.java | 132 ++++++------------ .../raftlog/segmented/SegmentedRaftLog.java | 19 ++- .../segmented/SegmentedRaftLogCache.java | 5 +- .../segmented/SegmentedRaftLogWorker.java | 8 +- .../impl/TransactionContextImpl.java | 18 --- .../raftlog/segmented/TestLogSegment.java | 24 ++-- .../segmented/TestSegmentedRaftLogCache.java | 12 +- .../org/apache/ratis/tools/ParseRatisLog.java | 9 +- 14 files changed, 112 insertions(+), 298 deletions(-) diff --git a/ratis-common/src/main/java/org/apache/ratis/protocol/RaftClientAsynchronousProtocol.java b/ratis-common/src/main/java/org/apache/ratis/protocol/RaftClientAsynchronousProtocol.java index 1985bbe667..1a9f83c823 100644 --- a/ratis-common/src/main/java/org/apache/ratis/protocol/RaftClientAsynchronousProtocol.java +++ b/ratis-common/src/main/java/org/apache/ratis/protocol/RaftClientAsynchronousProtocol.java @@ -1,4 +1,4 @@ -/* +/** * 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 @@ -17,40 +17,12 @@ */ package org.apache.ratis.protocol; -import org.apache.ratis.util.JavaUtils; -import org.apache.ratis.util.ReferenceCountedObject; - import java.io.IOException; import java.util.concurrent.CompletableFuture; /** Asynchronous version of {@link RaftClientProtocol}. */ public interface RaftClientAsynchronousProtocol { - /** - * It is recommended to override {@link #submitClientRequestAsync(ReferenceCountedObject)} instead. - * Then, it does not have to override this method. - */ - default CompletableFuture submitClientRequestAsync( - RaftClientRequest request) throws IOException { - return submitClientRequestAsync(ReferenceCountedObject.wrap(request)); - } + CompletableFuture submitClientRequestAsync( + RaftClientRequest request) throws IOException; - /** - * A referenced counted request is submitted from a client for processing. - * Implementations of this method should retain the request, process it and then release it. - * The request may be retained even after the future returned by this method has completed. - * - * @return a future of the reply - * @see ReferenceCountedObject - */ - default CompletableFuture submitClientRequestAsync( - ReferenceCountedObject requestRef) { - try { - // for backward compatibility - return submitClientRequestAsync(requestRef.retain()) - .whenComplete((r, e) -> requestRef.release()); - } catch (Exception e) { - requestRef.release(); - return JavaUtils.completeExceptionally(e); - } - } } \ No newline at end of file diff --git a/ratis-common/src/main/java/org/apache/ratis/util/ReferenceCountedObject.java b/ratis-common/src/main/java/org/apache/ratis/util/ReferenceCountedObject.java index 3f72f5ffe8..0dd378dc01 100644 --- a/ratis-common/src/main/java/org/apache/ratis/util/ReferenceCountedObject.java +++ b/ratis-common/src/main/java/org/apache/ratis/util/ReferenceCountedObject.java @@ -101,30 +101,6 @@ static ReferenceCountedObject wrap(V value) { return wrap(value, () -> {}, ignored -> {}); } - /** - * @return a {@link ReferenceCountedObject} of the given value by delegating to this object. - */ - default ReferenceCountedObject delegate(V value) { - final ReferenceCountedObject delegated = this; - return new ReferenceCountedObject() { - @Override - public V get() { - return value; - } - - @Override - public V retain() { - delegated.retain(); - return value; - } - - @Override - public boolean release() { - return delegated.release(); - } - }; - } - /** * Wrap the given value as a {@link ReferenceCountedObject}. * diff --git a/ratis-examples/src/main/java/org/apache/ratis/examples/filestore/FileStoreStateMachine.java b/ratis-examples/src/main/java/org/apache/ratis/examples/filestore/FileStoreStateMachine.java index 858e300ec9..5f258ee3b7 100644 --- a/ratis-examples/src/main/java/org/apache/ratis/examples/filestore/FileStoreStateMachine.java +++ b/ratis-examples/src/main/java/org/apache/ratis/examples/filestore/FileStoreStateMachine.java @@ -32,7 +32,6 @@ import org.apache.ratis.protocol.RaftClientRequest; import org.apache.ratis.protocol.RaftGroupId; import org.apache.ratis.server.RaftServer; -import org.apache.ratis.server.raftlog.LogProtoUtils; import org.apache.ratis.server.storage.RaftStorage; import org.apache.ratis.statemachine.StateMachineStorage; import org.apache.ratis.statemachine.TransactionContext; @@ -41,7 +40,6 @@ import org.apache.ratis.thirdparty.com.google.protobuf.ByteString; import org.apache.ratis.thirdparty.com.google.protobuf.InvalidProtocolBufferException; import org.apache.ratis.util.FileUtils; -import org.apache.ratis.util.JavaUtils; import java.io.IOException; import java.nio.file.Path; @@ -170,11 +168,9 @@ public CompletableFuture read(LogEntryProto entry, TransactionContex } static class LocalStream implements DataStream { - private final String name; private final DataChannel dataChannel; - LocalStream(String name, DataChannel dataChannel) { - this.name = JavaUtils.getClassSimpleName(getClass()) + "[" + name + "]"; + LocalStream(DataChannel dataChannel) { this.dataChannel = dataChannel; } @@ -194,11 +190,6 @@ public CompletableFuture cleanUp() { } }); } - - @Override - public String toString() { - return name; - } } @Override @@ -211,14 +202,13 @@ public CompletableFuture stream(RaftClientRequest request) { return FileStoreCommon.completeExceptionally( "Failed to parse stream header", e); } - final String file = proto.getStream().getPath().toStringUtf8(); - return files.createDataChannel(file) - .thenApply(channel -> new LocalStream(file, channel)); + return files.createDataChannel(proto.getStream().getPath().toStringUtf8()) + .thenApply(LocalStream::new); } @Override public CompletableFuture link(DataStream stream, LogEntryProto entry) { - LOG.info("linking {} to {}", stream, LogProtoUtils.toLogEntryString(entry)); + LOG.info("linking {}", stream); return files.streamLink(stream); } diff --git a/ratis-server-api/src/main/java/org/apache/ratis/statemachine/TransactionContext.java b/ratis-server-api/src/main/java/org/apache/ratis/statemachine/TransactionContext.java index e0190747fc..3821b058c9 100644 --- a/ratis-server-api/src/main/java/org/apache/ratis/statemachine/TransactionContext.java +++ b/ratis-server-api/src/main/java/org/apache/ratis/statemachine/TransactionContext.java @@ -23,7 +23,6 @@ import org.apache.ratis.protocol.RaftClientRequest; import org.apache.ratis.thirdparty.com.google.protobuf.ByteString; import org.apache.ratis.util.Preconditions; -import org.apache.ratis.util.ReferenceCountedObject; import org.apache.ratis.util.ReflectionUtils; import java.io.IOException; @@ -99,13 +98,6 @@ public interface TransactionContext { */ LogEntryProto getLogEntry(); - /** Wrap the given log entry as a {@link ReferenceCountedObject} for retaining it for later use. */ - default ReferenceCountedObject wrap(LogEntryProto entry) { - Preconditions.assertSame(getLogEntry().getTerm(), entry.getTerm(), "entry.term"); - Preconditions.assertSame(getLogEntry().getIndex(), entry.getIndex(), "entry.index"); - return ReferenceCountedObject.wrap(entry); - } - /** * Sets whether to commit the transaction to the RAFT log or not * @param shouldCommit true if the transaction is supposed to be committed to the RAFT log diff --git a/ratis-server/src/main/java/org/apache/ratis/server/impl/RaftServerImpl.java b/ratis-server/src/main/java/org/apache/ratis/server/impl/RaftServerImpl.java index c1a716bd0c..8d7246fcf7 100644 --- a/ratis-server/src/main/java/org/apache/ratis/server/impl/RaftServerImpl.java +++ b/ratis-server/src/main/java/org/apache/ratis/server/impl/RaftServerImpl.java @@ -109,7 +109,6 @@ import org.apache.ratis.util.MemoizedSupplier; import org.apache.ratis.util.Preconditions; import org.apache.ratis.util.ProtoUtils; -import org.apache.ratis.util.ReferenceCountedObject; import org.apache.ratis.util.TimeDuration; import org.apache.ratis.util.function.CheckedSupplier; @@ -772,21 +771,15 @@ void assertLifeCycleState(Set expected) throws ServerNotReadyEx } /** - * Append a transaction to the log for processing a client request. - * Note that the given request could be different from {@link TransactionContext#getClientRequest()} - * since the request could be converted; see {@link #convertRaftClientRequest(RaftClientRequest)}. - * - * @param request The client request. - * @param context The context of the transaction. - * @param cacheEntry the entry in the retry cache. - * @return a future of the reply. + * Handle a normal update request from client. */ private CompletableFuture appendTransaction( - RaftClientRequest request, TransactionContextImpl context, CacheEntry cacheEntry) { - Objects.requireNonNull(request, "request == null"); + RaftClientRequest request, TransactionContextImpl context, CacheEntry cacheEntry) throws IOException { CodeInjectionForTesting.execute(APPEND_TRANSACTION, getId(), request.getClientId(), request, context, cacheEntry); + assertLifeCycleState(LifeCycle.States.RUNNING); + final PendingRequest pending; synchronized (this) { final CompletableFuture reply = checkLeaderState(request, cacheEntry); @@ -805,7 +798,6 @@ private CompletableFuture appendTransaction( return cacheEntry.getReplyFuture(); } try { - assertLifeCycleState(LifeCycle.States.RUNNING); state.appendLog(context); } catch (StateMachineException e) { // the StateMachineException is thrown by the SM in the preAppend stage. @@ -817,9 +809,6 @@ private CompletableFuture appendTransaction( leaderState.submitStepDownEvent(LeaderState.StepDownReason.STATE_MACHINE_EXCEPTION); } return CompletableFuture.completedFuture(exceptionReply); - } catch (ServerNotReadyException e) { - final RaftClientReply exceptionReply = newExceptionReply(request, e); - return CompletableFuture.completedFuture(exceptionReply); } // put the request into the pending queue @@ -872,13 +861,11 @@ void stepDownOnJvmPause() { role.getLeaderState().ifPresent(leader -> leader.submitStepDownEvent(LeaderState.StepDownReason.JVM_PAUSE)); } - /** If the given request is {@link TypeCase#FORWARD}, convert it. */ - static RaftClientRequest convertRaftClientRequest(RaftClientRequest request) throws InvalidProtocolBufferException { - if (!request.is(TypeCase.FORWARD)) { - return request; - } - return ClientProtoUtils.toRaftClientRequest(RaftClientRequestProto.parseFrom( - request.getMessage().getContent().asReadOnlyByteBuffer())); + private RaftClientRequest filterDataStreamRaftClientRequest(RaftClientRequest request) + throws InvalidProtocolBufferException { + return !request.is(TypeCase.FORWARD) ? request : ClientProtoUtils.toRaftClientRequest( + RaftClientRequestProto.parseFrom( + request.getMessage().getContent().asReadOnlyByteBuffer())); } CompletableFuture executeSubmitServerRequestAsync( @@ -888,29 +875,20 @@ CompletableFuture executeSubmitServerRequestAsync( serverExecutor).join(); } - CompletableFuture executeSubmitClientRequestAsync( - ReferenceCountedObject request) { - return CompletableFuture.supplyAsync(() -> submitClientRequestAsync(request), clientExecutor).join(); + CompletableFuture executeSubmitClientRequestAsync(RaftClientRequest request) { + return CompletableFuture.supplyAsync( + () -> JavaUtils.callAsUnchecked(() -> submitClientRequestAsync(request), CompletionException::new), + clientExecutor).join(); } @Override public CompletableFuture submitClientRequestAsync( - ReferenceCountedObject requestRef) { - final RaftClientRequest request = requestRef.retain(); + RaftClientRequest request) throws IOException { + assertLifeCycleState(LifeCycle.States.RUNNING); LOG.debug("{}: receive client request({})", getMemberId(), request); - - try { - assertLifeCycleState(LifeCycle.States.RUNNING); - } catch (ServerNotReadyException e) { - final RaftClientReply reply = newExceptionReply(request, e); - requestRef.release(); - return CompletableFuture.completedFuture(reply); - } - final Timekeeper timer = raftServerMetrics.getClientRequestTimer(request.getType()); final Optional timerContext = Optional.ofNullable(timer).map(Timekeeper::time); - return replyFuture(requestRef).whenComplete((clientReply, exception) -> { - requestRef.release(); + return replyFuture(request).whenComplete((clientReply, exception) -> { timerContext.ifPresent(Timekeeper.Context::stop); if (exception != null || clientReply.getException() != null) { raftServerMetrics.incFailedRequestCount(request.getType()); @@ -918,8 +896,7 @@ public CompletableFuture submitClientRequestAsync( }); } - private CompletableFuture replyFuture(ReferenceCountedObject requestRef) { - final RaftClientRequest request = requestRef.get(); + private CompletableFuture replyFuture(RaftClientRequest request) throws IOException { retryCache.invalidateRepliedRequests(request); final TypeCase type = request.getType().getTypeCase(); @@ -931,18 +908,17 @@ private CompletableFuture replyFuture(ReferenceCountedObject writeAsyncImpl(ReferenceCountedObject return CompletableFuture.completedFuture(exceptionReply); } - context.setDelegatedRef(requestRef); return appendTransaction(request, context, cacheEntry); } @@ -1089,8 +1056,7 @@ private RaftClientReply readException2Reply(RaftClientRequest request, Throwable } } - private CompletableFuture messageStreamAsync(ReferenceCountedObject requestRef) { - final RaftClientRequest request = requestRef.get(); + private CompletableFuture messageStreamAsync(RaftClientRequest request) throws IOException { final CompletableFuture reply = checkLeaderState(request); if (reply != null) { return reply; @@ -1102,7 +1068,7 @@ private CompletableFuture messageStreamAsync(ReferenceCountedOb return f.thenApply(r -> null); } // the message stream has ended and the request become a WRITE request - return replyFuture(requestRef.delegate(f.join())); + return replyFuture(f.join()); } return role.getLeaderState() diff --git a/ratis-server/src/main/java/org/apache/ratis/server/impl/RaftServerProxy.java b/ratis-server/src/main/java/org/apache/ratis/server/impl/RaftServerProxy.java index 9834d62ab4..84221cfcfe 100644 --- a/ratis-server/src/main/java/org/apache/ratis/server/impl/RaftServerProxy.java +++ b/ratis-server/src/main/java/org/apache/ratis/server/impl/RaftServerProxy.java @@ -52,7 +52,6 @@ import org.apache.ratis.util.MemoizedSupplier; import org.apache.ratis.util.Preconditions; import org.apache.ratis.util.ProtoUtils; -import org.apache.ratis.util.ReferenceCountedObject; import org.apache.ratis.util.TimeDuration; import java.io.Closeable; @@ -446,15 +445,9 @@ public void close() { } @Override - public CompletableFuture submitClientRequestAsync( - ReferenceCountedObject requestRef) { - final RaftClientRequest request = requestRef.retain(); - try { - return getImplFuture(request.getRaftGroupId()) - .thenCompose(impl -> impl.executeSubmitClientRequestAsync(requestRef)); - } finally { - requestRef.release(); - } + public CompletableFuture submitClientRequestAsync(RaftClientRequest request) { + return getImplFuture(request.getRaftGroupId()) + .thenCompose(impl -> impl.executeSubmitClientRequestAsync(request)); } @Override diff --git a/ratis-server/src/main/java/org/apache/ratis/server/raftlog/segmented/LogSegment.java b/ratis-server/src/main/java/org/apache/ratis/server/raftlog/segmented/LogSegment.java index 12e7c4f1d1..89a6e20501 100644 --- a/ratis-server/src/main/java/org/apache/ratis/server/raftlog/segmented/LogSegment.java +++ b/ratis-server/src/main/java/org/apache/ratis/server/raftlog/segmented/LogSegment.java @@ -30,7 +30,6 @@ import org.apache.ratis.thirdparty.com.google.protobuf.CodedOutputStream; import org.apache.ratis.util.FileUtils; import org.apache.ratis.util.Preconditions; -import org.apache.ratis.util.ReferenceCountedObject; import org.apache.ratis.util.SizeInBytes; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -42,7 +41,6 @@ import java.util.List; import java.util.Map; import java.util.Objects; -import java.util.Optional; import java.util.concurrent.ConcurrentHashMap; import java.util.concurrent.atomic.AtomicInteger; import java.util.concurrent.atomic.AtomicLong; @@ -69,20 +67,17 @@ enum Op { } static long getEntrySize(LogEntryProto entry, Op op) { - switch (op) { - case CHECK_SEGMENT_FILE_FULL: - case LOAD_SEGMENT_FILE: - case WRITE_CACHE_WITH_STATE_MACHINE_CACHE: - Preconditions.assertTrue(entry == LogProtoUtils.removeStateMachineData(entry), - () -> "Unexpected LogEntryProto with StateMachine data: op=" + op + ", entry=" + entry); - break; - case WRITE_CACHE_WITHOUT_STATE_MACHINE_CACHE: - case REMOVE_CACHE: - break; - default: - throw new IllegalStateException("Unexpected op " + op + ", entry=" + entry); + LogEntryProto e = entry; + if (op == Op.CHECK_SEGMENT_FILE_FULL) { + e = LogProtoUtils.removeStateMachineData(entry); + } else if (op == Op.LOAD_SEGMENT_FILE || op == Op.WRITE_CACHE_WITH_STATE_MACHINE_CACHE) { + Preconditions.assertTrue(entry == LogProtoUtils.removeStateMachineData(entry), + () -> "Unexpected LogEntryProto with StateMachine data: op=" + op + ", entry=" + entry); + } else { + Preconditions.assertTrue(op == Op.WRITE_CACHE_WITHOUT_STATE_MACHINE_CACHE || op == Op.REMOVE_CACHE, + () -> "Unexpected op " + op + ", entry=" + entry); } - final int serialized = entry.getSerializedSize(); + final int serialized = e.getSerializedSize(); return serialized + CodedOutputStream.computeUInt32SizeNoTag(serialized) + 4L; } @@ -129,8 +124,7 @@ static LogSegment newLogSegment(RaftStorage storage, LogSegmentStartEnd startEnd } public static int readSegmentFile(File file, LogSegmentStartEnd startEnd, SizeInBytes maxOpSize, - CorruptionPolicy corruptionPolicy, SegmentedRaftLogMetrics raftLogMetrics, - Consumer> entryConsumer) + CorruptionPolicy corruptionPolicy, SegmentedRaftLogMetrics raftLogMetrics, Consumer entryConsumer) throws IOException { int count = 0; try (SegmentedRaftLogInputStream in = new SegmentedRaftLogInputStream( @@ -142,8 +136,7 @@ public static int readSegmentFile(File file, LogSegmentStartEnd startEnd, SizeIn } if (entryConsumer != null) { - // TODO: use reference count to support zero buffer copying for readSegmentFile - entryConsumer.accept(ReferenceCountedObject.wrap(next)); + entryConsumer.accept(next); } count++; } @@ -170,7 +163,10 @@ static LogSegment loadSegment(RaftStorage storage, File file, LogSegmentStartEnd final CorruptionPolicy corruptionPolicy = CorruptionPolicy.get(storage, RaftStorage::getLogCorruptionPolicy); final boolean isOpen = startEnd.isOpen(); final int entryCount = readSegmentFile(file, startEnd, maxOpSize, corruptionPolicy, raftLogMetrics, entry -> { - segment.append(Op.LOAD_SEGMENT_FILE, entry, keepEntryInCache || isOpen, logConsumer); + segment.append(keepEntryInCache || isOpen, entry, Op.LOAD_SEGMENT_FILE); + if (logConsumer != null) { + logConsumer.accept(entry); + } }); LOG.info("Successfully read {} entries from segment file {}", entryCount, file); @@ -238,10 +234,10 @@ public LogEntryProto load(LogRecord key) throws IOException { // the on-disk log file should be truncated but has not been done yet. final AtomicReference toReturn = new AtomicReference<>(); final LogSegmentStartEnd startEnd = LogSegmentStartEnd.valueOf(startIndex, endIndex, isOpen); - readSegmentFile(file, startEnd, maxOpSize, getLogCorruptionPolicy(), raftLogMetrics, entryRef -> { - final LogEntryProto entry = entryRef.retain(); + readSegmentFile(file, startEnd, maxOpSize, + getLogCorruptionPolicy(), raftLogMetrics, entry -> { final TermIndex ti = TermIndex.valueOf(entry); - putEntryCache(ti, entryRef, Op.LOAD_SEGMENT_FILE); + putEntryCache(ti, entry, Op.LOAD_SEGMENT_FILE); if (ti.equals(key.getTermIndex())) { toReturn.set(entry); } @@ -251,48 +247,13 @@ public LogEntryProto load(LogRecord key) throws IOException { } } - static class EntryCache { - private final Map> map = new ConcurrentHashMap<>(); - private final AtomicLong size = new AtomicLong(); - - long size() { - return size.get(); - } - - LogEntryProto get(TermIndex ti) { - return Optional.ofNullable(map.get(ti)) - .map(ReferenceCountedObject::get) - .orElse(null); - } - - void clear() { - map.values().forEach(ReferenceCountedObject::release); - map.clear(); - size.set(0); - } - - void put(TermIndex key, ReferenceCountedObject valueRef, Op op) { - valueRef.retain(); - Optional.ofNullable(map.put(key, valueRef)).ifPresent(this::release); - size.getAndAdd(getEntrySize(valueRef.get(), op)); - } - - private void release(ReferenceCountedObject entry) { - size.getAndAdd(-getEntrySize(entry.get(), Op.REMOVE_CACHE)); - entry.release(); - } - - void remove(TermIndex key) { - Optional.ofNullable(map.remove(key)).ifPresent(this::release); - } - } - File getFile() { return LogSegmentStartEnd.valueOf(startIndex, endIndex, isOpen).getFile(storage); } private volatile boolean isOpen; private long totalFileSize = SegmentedRaftLogFormat.getHeaderLength(); + private AtomicLong totalCacheSize = new AtomicLong(0); /** Segment start index, inclusive. */ private final long startIndex; /** Segment end index, inclusive. */ @@ -310,7 +271,7 @@ File getFile() { /** * the entryCache caches the content of log entries. */ - private final EntryCache entryCache = new EntryCache(); + private final Map entryCache = new ConcurrentHashMap<>(); private LogSegment(RaftStorage storage, boolean isOpen, long start, long end, SizeInBytes maxOpSize, SegmentedRaftLogMetrics raftLogMetrics) { @@ -342,29 +303,12 @@ CorruptionPolicy getLogCorruptionPolicy() { return CorruptionPolicy.get(storage, RaftStorage::getLogCorruptionPolicy); } - void appendToOpenSegment(Op op, ReferenceCountedObject entryRef) { + void appendToOpenSegment(LogEntryProto entry, Op op) { Preconditions.assertTrue(isOpen(), "The log segment %s is not open for append", this); - append(op, entryRef, true, null); - } - - private void append(Op op, ReferenceCountedObject entryRef, - boolean keepEntryInCache, Consumer logConsumer) { - final LogEntryProto entry = entryRef.retain(); - try { - final LogRecord record = appendLogRecord(op, entry); - if (keepEntryInCache) { - putEntryCache(record.getTermIndex(), entryRef, op); - } - if (logConsumer != null) { - logConsumer.accept(entry); - } - } finally { - entryRef.release(); - } + append(true, entry, op); } - - private LogRecord appendLogRecord(Op op, LogEntryProto entry) { + private void append(boolean keepEntryInCache, LogEntryProto entry, Op op) { Objects.requireNonNull(entry, "entry == null"); if (records.isEmpty()) { Preconditions.assertTrue(entry.getIndex() == startIndex, @@ -380,9 +324,11 @@ private LogRecord appendLogRecord(Op op, LogEntryProto entry) { final LogRecord record = new LogRecord(totalFileSize, entry); records.add(record); + if (keepEntryInCache) { + putEntryCache(record.getTermIndex(), entry, op); + } totalFileSize += getEntrySize(entry, op); endIndex = entry.getIndex(); - return record; } LogEntryProto getEntryFromCache(TermIndex ti) { @@ -425,7 +371,7 @@ long getTotalFileSize() { } long getTotalCacheSize() { - return entryCache.size(); + return totalCacheSize.get(); } /** @@ -435,7 +381,7 @@ synchronized void truncate(long fromIndex) { Preconditions.assertTrue(fromIndex >= startIndex && fromIndex <= endIndex); for (long index = endIndex; index >= fromIndex; index--) { LogRecord removed = records.remove(Math.toIntExact(index - startIndex)); - removeEntryCache(removed.getTermIndex()); + removeEntryCache(removed.getTermIndex(), Op.REMOVE_CACHE); totalFileSize = removed.offset; } isOpen = false; @@ -480,18 +426,28 @@ int getLoadingTimes() { void evictCache() { entryCache.clear(); + totalCacheSize.set(0); } - void putEntryCache(TermIndex key, ReferenceCountedObject valueRef, Op op) { - entryCache.put(key, valueRef, op); + void putEntryCache(TermIndex key, LogEntryProto value, Op op) { + final LogEntryProto previous = entryCache.put(key, value); + long previousSize = 0; + if (previous != null) { + // Different threads maybe load LogSegment file into cache at the same time, so duplicate maybe happen + previousSize = getEntrySize(value, Op.REMOVE_CACHE); + } + totalCacheSize.getAndAdd(getEntrySize(value, op) - previousSize); } - void removeEntryCache(TermIndex key) { - entryCache.remove(key); + void removeEntryCache(TermIndex key, Op op) { + LogEntryProto value = entryCache.remove(key); + if (value != null) { + totalCacheSize.getAndAdd(-getEntrySize(value, op)); + } } boolean hasCache() { - return isOpen || entryCache.size() > 0; // open segment always has cache. + return isOpen || !entryCache.isEmpty(); // open segment always has cache. } boolean containsIndex(long index) { diff --git a/ratis-server/src/main/java/org/apache/ratis/server/raftlog/segmented/SegmentedRaftLog.java b/ratis-server/src/main/java/org/apache/ratis/server/raftlog/segmented/SegmentedRaftLog.java index def472a607..f49900f16a 100644 --- a/ratis-server/src/main/java/org/apache/ratis/server/raftlog/segmented/SegmentedRaftLog.java +++ b/ratis-server/src/main/java/org/apache/ratis/server/raftlog/segmented/SegmentedRaftLog.java @@ -41,7 +41,6 @@ import org.apache.ratis.util.AwaitToRun; import org.apache.ratis.util.JavaUtils; import org.apache.ratis.util.Preconditions; -import org.apache.ratis.util.ReferenceCountedObject; import org.apache.ratis.util.StringUtils; import java.io.File; @@ -54,7 +53,6 @@ import java.util.concurrent.CompletionException; import java.util.function.BiFunction; import java.util.function.Consumer; -import java.util.function.Function; import java.util.function.LongSupplier; import org.apache.ratis.util.UncheckedAutoCloseable; @@ -395,7 +393,6 @@ protected CompletableFuture appendEntryImpl(LogEntryProto entry, Transacti if (LOG.isTraceEnabled()) { LOG.trace("{}: appendEntry {}", getName(), LogProtoUtils.toLogEntryString(entry)); } - final LogEntryProto removedStateMachineData = LogProtoUtils.removeStateMachineData(entry); try(AutoCloseableLock writeLock = writeLock()) { final Timekeeper.Context appendEntryTimerContext = getRaftLogMetrics().startAppendEntryTimer(); validateLogEntry(entry); @@ -404,7 +401,7 @@ protected CompletableFuture appendEntryImpl(LogEntryProto entry, Transacti if (currentOpenSegment == null) { cache.addOpenSegment(entry.getIndex()); fileLogWorker.startLogSegment(entry.getIndex()); - } else if (isSegmentFull(currentOpenSegment, removedStateMachineData)) { + } else if (isSegmentFull(currentOpenSegment, entry)) { rollOpenSegment = true; } else { final TermIndex last = currentOpenSegment.getLastTermIndex(); @@ -426,17 +423,17 @@ protected CompletableFuture appendEntryImpl(LogEntryProto entry, Transacti // If the entry has state machine data, then the entry should be inserted // to statemachine first and then to the cache. Not following the order // will leave a spurious entry in the cache. - final Task write = fileLogWorker.writeLogEntry(entry, removedStateMachineData, context); - final Function> wrap = context != null ? - context::wrap : ReferenceCountedObject::wrap; + CompletableFuture writeFuture = + fileLogWorker.writeLogEntry(entry, context).getFuture(); if (stateMachineCachingEnabled) { // The stateMachineData will be cached inside the StateMachine itself. - cache.appendEntry(LogSegment.Op.WRITE_CACHE_WITH_STATE_MACHINE_CACHE, wrap.apply(removedStateMachineData)); + cache.appendEntry(LogProtoUtils.removeStateMachineData(entry), + LogSegment.Op.WRITE_CACHE_WITH_STATE_MACHINE_CACHE); } else { - cache.appendEntry(LogSegment.Op.WRITE_CACHE_WITHOUT_STATE_MACHINE_CACHE, wrap.apply(entry) - ); + cache.appendEntry(entry, LogSegment.Op.WRITE_CACHE_WITHOUT_STATE_MACHINE_CACHE); } - return write.getFuture().whenComplete((clientReply, exception) -> appendEntryTimerContext.stop()); + writeFuture.whenComplete((clientReply, exception) -> appendEntryTimerContext.stop()); + return writeFuture; } catch (Exception e) { LOG.error("{}: Failed to append {}", getName(), toLogEntryString(entry), e); throw e; diff --git a/ratis-server/src/main/java/org/apache/ratis/server/raftlog/segmented/SegmentedRaftLogCache.java b/ratis-server/src/main/java/org/apache/ratis/server/raftlog/segmented/SegmentedRaftLogCache.java index 0b05b14e5c..58c70c4af9 100644 --- a/ratis-server/src/main/java/org/apache/ratis/server/raftlog/segmented/SegmentedRaftLogCache.java +++ b/ratis-server/src/main/java/org/apache/ratis/server/raftlog/segmented/SegmentedRaftLogCache.java @@ -32,7 +32,6 @@ import org.apache.ratis.util.AutoCloseableReadWriteLock; import org.apache.ratis.util.JavaUtils; import org.apache.ratis.util.Preconditions; -import org.apache.ratis.util.ReferenceCountedObject; import org.apache.ratis.util.SizeInBytes; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -598,11 +597,11 @@ TermIndex getLastTermIndex() { } } - void appendEntry(LogSegment.Op op, ReferenceCountedObject entry) { + void appendEntry(LogEntryProto entry, LogSegment.Op op) { // SegmentedRaftLog does the segment creation/rolling work. Here we just // simply append the entry into the open segment. Preconditions.assertNotNull(openSegment, "openSegment"); - openSegment.appendToOpenSegment(op, entry); + openSegment.appendToOpenSegment(entry, op); } /** diff --git a/ratis-server/src/main/java/org/apache/ratis/server/raftlog/segmented/SegmentedRaftLogWorker.java b/ratis-server/src/main/java/org/apache/ratis/server/raftlog/segmented/SegmentedRaftLogWorker.java index 5b0470d4fa..68266b4175 100644 --- a/ratis-server/src/main/java/org/apache/ratis/server/raftlog/segmented/SegmentedRaftLogWorker.java +++ b/ratis-server/src/main/java/org/apache/ratis/server/raftlog/segmented/SegmentedRaftLogWorker.java @@ -445,8 +445,8 @@ void rollLogSegment(LogSegment segmentToClose) { addIOTask(new StartLogSegment(segmentToClose.getEndIndex() + 1)); } - Task writeLogEntry(LogEntryProto entry, LogEntryProto removedStateMachineData, TransactionContext context) { - return addIOTask(new WriteLog(entry, removedStateMachineData, context)); + Task writeLogEntry(LogEntryProto entry, TransactionContext context) { + return addIOTask(new WriteLog(entry, context)); } Task truncate(TruncationSegments ts, long index) { @@ -493,8 +493,8 @@ private class WriteLog extends Task { private final CompletableFuture stateMachineFuture; private final CompletableFuture combined; - WriteLog(LogEntryProto entry, LogEntryProto removedStateMachineData, TransactionContext context) { - this.entry = removedStateMachineData; + WriteLog(LogEntryProto entry, TransactionContext context) { + this.entry = LogProtoUtils.removeStateMachineData(entry); if (this.entry == entry) { final StateMachineLogEntryProto proto = entry.hasStateMachineLogEntry()? entry.getStateMachineLogEntry(): null; if (stateMachine != null && proto != null && proto.getType() == StateMachineLogEntryProto.Type.DATASTREAM) { diff --git a/ratis-server/src/main/java/org/apache/ratis/statemachine/impl/TransactionContextImpl.java b/ratis-server/src/main/java/org/apache/ratis/statemachine/impl/TransactionContextImpl.java index 6006257166..d92f3a1c82 100644 --- a/ratis-server/src/main/java/org/apache/ratis/statemachine/impl/TransactionContextImpl.java +++ b/ratis-server/src/main/java/org/apache/ratis/statemachine/impl/TransactionContextImpl.java @@ -26,7 +26,6 @@ import org.apache.ratis.statemachine.TransactionContext; import org.apache.ratis.thirdparty.com.google.protobuf.ByteString; import org.apache.ratis.util.Preconditions; -import org.apache.ratis.util.ReferenceCountedObject; import java.io.IOException; import java.util.Objects; @@ -72,9 +71,6 @@ public class TransactionContextImpl implements TransactionContext { /** Committed LogEntry. */ @SuppressWarnings({"squid:S3077"}) // Suppress volatile for generic type private volatile LogEntryProto logEntry; - /** For wrapping {@link #logEntry} in order to release the underlying buffer. */ - @SuppressWarnings({"squid:S3077"}) // Suppress volatile for generic type - private volatile ReferenceCountedObject delegatedRef; private final CompletableFuture logIndexFuture = new CompletableFuture<>(); @@ -130,20 +126,6 @@ public RaftClientRequest getClientRequest() { return clientRequest; } - public void setDelegatedRef(ReferenceCountedObject ref) { - this.delegatedRef = ref; - } - - @Override - public ReferenceCountedObject wrap(LogEntryProto entry) { - if (delegatedRef == null) { - return TransactionContext.super.wrap(entry); - } - Preconditions.assertSame(getLogEntry().getTerm(), entry.getTerm(), "entry.term"); - Preconditions.assertSame(getLogEntry().getIndex(), entry.getIndex(), "entry.index"); - return delegatedRef.delegate(entry); - } - @Override public StateMachineLogEntryProto getStateMachineLogEntry() { return stateMachineLogEntry; diff --git a/ratis-test/src/test/java/org/apache/ratis/server/raftlog/segmented/TestLogSegment.java b/ratis-test/src/test/java/org/apache/ratis/server/raftlog/segmented/TestLogSegment.java index 4e04e9e62c..50f9d2382b 100644 --- a/ratis-test/src/test/java/org/apache/ratis/server/raftlog/segmented/TestLogSegment.java +++ b/ratis-test/src/test/java/org/apache/ratis/server/raftlog/segmented/TestLogSegment.java @@ -21,20 +21,18 @@ import org.apache.ratis.RaftTestUtil.SimpleOperation; import org.apache.ratis.conf.RaftProperties; import org.apache.ratis.metrics.impl.DefaultTimekeeperImpl; -import org.apache.ratis.proto.RaftProtos.LogEntryProto; -import org.apache.ratis.proto.RaftProtos.StateMachineLogEntryProto; import org.apache.ratis.server.RaftServerConfigKeys; import org.apache.ratis.server.impl.RaftServerTestUtil; import org.apache.ratis.server.metrics.SegmentedRaftLogMetrics; import org.apache.ratis.server.protocol.TermIndex; import org.apache.ratis.server.raftlog.LogProtoUtils; -import org.apache.ratis.server.raftlog.segmented.LogSegment.Op; import org.apache.ratis.server.storage.RaftStorage; +import org.apache.ratis.proto.RaftProtos.LogEntryProto; +import org.apache.ratis.proto.RaftProtos.StateMachineLogEntryProto; import org.apache.ratis.server.storage.RaftStorageTestUtils; import org.apache.ratis.thirdparty.com.google.protobuf.CodedOutputStream; import org.apache.ratis.util.FileUtils; import org.apache.ratis.util.Preconditions; -import org.apache.ratis.util.ReferenceCountedObject; import org.apache.ratis.util.SizeInBytes; import org.apache.ratis.util.TraditionalBinaryPrefix; import org.junit.jupiter.api.AfterEach; @@ -146,7 +144,7 @@ static void checkLogSegment(LogSegment segment, long start, long end, if (entry == null) { entry = segment.loadCache(record); } - offset += getEntrySize(entry, Op.WRITE_CACHE_WITHOUT_STATE_MACHINE_CACHE); + offset += getEntrySize(entry, LogSegment.Op.WRITE_CACHE_WITHOUT_STATE_MACHINE_CACHE); } } @@ -205,8 +203,8 @@ public void testAppendEntries() throws Exception { while (size < max) { SimpleOperation op = new SimpleOperation("m" + i); LogEntryProto entry = LogProtoUtils.toLogEntryProto(op.getLogEntryContent(), term, i++ + start); - size += getEntrySize(entry, Op.WRITE_CACHE_WITHOUT_STATE_MACHINE_CACHE); - segment.appendToOpenSegment(Op.WRITE_CACHE_WITHOUT_STATE_MACHINE_CACHE, ReferenceCountedObject.wrap(entry)); + size += getEntrySize(entry, LogSegment.Op.WRITE_CACHE_WITHOUT_STATE_MACHINE_CACHE); + segment.appendToOpenSegment(entry, LogSegment.Op.WRITE_CACHE_WITHOUT_STATE_MACHINE_CACHE); } Assertions.assertTrue(segment.getTotalFileSize() >= max); @@ -238,18 +236,18 @@ public void testAppendWithGap() throws Exception { final StateMachineLogEntryProto m = op.getLogEntryContent(); try { LogEntryProto entry = LogProtoUtils.toLogEntryProto(m, 0, 1001); - segment.appendToOpenSegment(Op.WRITE_CACHE_WITHOUT_STATE_MACHINE_CACHE, ReferenceCountedObject.wrap(entry)); + segment.appendToOpenSegment(entry, LogSegment.Op.WRITE_CACHE_WITHOUT_STATE_MACHINE_CACHE); Assertions.fail("should fail since the entry's index needs to be 1000"); } catch (IllegalStateException e) { // the exception is expected. } LogEntryProto entry = LogProtoUtils.toLogEntryProto(m, 0, 1000); - segment.appendToOpenSegment(Op.WRITE_CACHE_WITHOUT_STATE_MACHINE_CACHE, ReferenceCountedObject.wrap(entry)); + segment.appendToOpenSegment(entry, LogSegment.Op.WRITE_CACHE_WITHOUT_STATE_MACHINE_CACHE); try { entry = LogProtoUtils.toLogEntryProto(m, 0, 1002); - segment.appendToOpenSegment(Op.WRITE_CACHE_WITHOUT_STATE_MACHINE_CACHE, ReferenceCountedObject.wrap(entry)); + segment.appendToOpenSegment(entry, LogSegment.Op.WRITE_CACHE_WITHOUT_STATE_MACHINE_CACHE); Assertions.fail("should fail since the entry's index needs to be 1001"); } catch (IllegalStateException e) { // the exception is expected. @@ -264,7 +262,7 @@ public void testTruncate() throws Exception { for (int i = 0; i < 100; i++) { LogEntryProto entry = LogProtoUtils.toLogEntryProto( new SimpleOperation("m" + i).getLogEntryContent(), term, i + start); - segment.appendToOpenSegment(Op.WRITE_CACHE_WITHOUT_STATE_MACHINE_CACHE, ReferenceCountedObject.wrap(entry)); + segment.appendToOpenSegment(entry, LogSegment.Op.WRITE_CACHE_WITHOUT_STATE_MACHINE_CACHE); } // truncate an open segment (remove 1080~1099) @@ -319,7 +317,7 @@ public void testPreallocateSegment() throws Exception { 1024, 1024, ByteBuffer.allocateDirect(bufferSize))) { SimpleOperation op = new SimpleOperation(new String(content)); LogEntryProto entry = LogProtoUtils.toLogEntryProto(op.getLogEntryContent(), 0, 0); - size = LogSegment.getEntrySize(entry, Op.WRITE_CACHE_WITHOUT_STATE_MACHINE_CACHE); + size = LogSegment.getEntrySize(entry, LogSegment.Op.WRITE_CACHE_WITHOUT_STATE_MACHINE_CACHE); out.write(entry); } Assertions.assertEquals(file.length(), @@ -346,7 +344,7 @@ public void testPreallocationAndAppend() throws Exception { Arrays.fill(content, (byte) 1); SimpleOperation op = new SimpleOperation(new String(content)); LogEntryProto entry = LogProtoUtils.toLogEntryProto(op.getLogEntryContent(), 0, 0); - final long entrySize = LogSegment.getEntrySize(entry, Op.WRITE_CACHE_WITHOUT_STATE_MACHINE_CACHE); + final long entrySize = LogSegment.getEntrySize(entry, LogSegment.Op.WRITE_CACHE_WITHOUT_STATE_MACHINE_CACHE); long totalSize = SegmentedRaftLogFormat.getHeaderLength(); long preallocated = 16 * 1024; diff --git a/ratis-test/src/test/java/org/apache/ratis/server/raftlog/segmented/TestSegmentedRaftLogCache.java b/ratis-test/src/test/java/org/apache/ratis/server/raftlog/segmented/TestSegmentedRaftLogCache.java index efcb90580f..8015f18274 100644 --- a/ratis-test/src/test/java/org/apache/ratis/server/raftlog/segmented/TestSegmentedRaftLogCache.java +++ b/ratis-test/src/test/java/org/apache/ratis/server/raftlog/segmented/TestSegmentedRaftLogCache.java @@ -34,9 +34,7 @@ import org.apache.ratis.server.raftlog.LogProtoUtils; import org.apache.ratis.server.raftlog.segmented.SegmentedRaftLogCache.TruncationSegments; import org.apache.ratis.server.raftlog.segmented.LogSegment.LogRecord; -import org.apache.ratis.server.raftlog.segmented.LogSegment.Op; import org.apache.ratis.proto.RaftProtos.LogEntryProto; -import org.apache.ratis.util.ReferenceCountedObject; import org.junit.jupiter.api.AfterEach; import org.junit.jupiter.api.Assertions; import org.junit.jupiter.api.BeforeEach; @@ -66,7 +64,7 @@ private LogSegment prepareLogSegment(long start, long end, boolean isOpen) { for (long i = start; i <= end; i++) { SimpleOperation m = new SimpleOperation("m" + i); LogEntryProto entry = LogProtoUtils.toLogEntryProto(m.getLogEntryContent(), 0, i); - s.appendToOpenSegment(Op.WRITE_CACHE_WITHOUT_STATE_MACHINE_CACHE, ReferenceCountedObject.wrap(entry)); + s.appendToOpenSegment(entry, LogSegment.Op.WRITE_CACHE_WITHOUT_STATE_MACHINE_CACHE); } if (!isOpen) { s.close(); @@ -150,15 +148,14 @@ public void testAddSegments() throws Exception { } @Test - public void testAppendEntry() { + public void testAppendEntry() throws Exception { LogSegment closedSegment = prepareLogSegment(0, 99, false); cache.addSegment(closedSegment); final SimpleOperation m = new SimpleOperation("m"); try { LogEntryProto entry = LogProtoUtils.toLogEntryProto(m.getLogEntryContent(), 0, 0); - cache.appendEntry(Op.WRITE_CACHE_WITHOUT_STATE_MACHINE_CACHE, ReferenceCountedObject.wrap(entry) - ); + cache.appendEntry(entry, LogSegment.Op.WRITE_CACHE_WITHOUT_STATE_MACHINE_CACHE); Assertions.fail("the open segment is null"); } catch (IllegalStateException ignored) { } @@ -167,8 +164,7 @@ public void testAppendEntry() { cache.addSegment(openSegment); for (long index = 101; index < 200; index++) { LogEntryProto entry = LogProtoUtils.toLogEntryProto(m.getLogEntryContent(), 0, index); - cache.appendEntry(Op.WRITE_CACHE_WITHOUT_STATE_MACHINE_CACHE, ReferenceCountedObject.wrap(entry) - ); + cache.appendEntry(entry, LogSegment.Op.WRITE_CACHE_WITHOUT_STATE_MACHINE_CACHE); } Assertions.assertNotNull(cache.getOpenSegment()); diff --git a/ratis-tools/src/main/java/org/apache/ratis/tools/ParseRatisLog.java b/ratis-tools/src/main/java/org/apache/ratis/tools/ParseRatisLog.java index ea512fa70c..7107977fbb 100644 --- a/ratis-tools/src/main/java/org/apache/ratis/tools/ParseRatisLog.java +++ b/ratis-tools/src/main/java/org/apache/ratis/tools/ParseRatisLog.java @@ -24,7 +24,6 @@ import org.apache.ratis.server.raftlog.LogProtoUtils; import org.apache.ratis.server.raftlog.segmented.LogSegmentPath; import org.apache.ratis.server.raftlog.segmented.LogSegment; -import org.apache.ratis.util.ReferenceCountedObject; import org.apache.ratis.util.SizeInBytes; import java.io.File; @@ -61,7 +60,7 @@ public void dumpSegmentFile() throws IOException { System.out.println("Processing Raft Log file: " + file.getAbsolutePath() + " size:" + file.length()); final int entryCount = LogSegment.readSegmentFile(file, pi.getStartEnd(), maxOpSize, - RaftServerConfigKeys.Log.CorruptionPolicy.EXCEPTION, null, this::processLogEntry); + RaftServerConfigKeys.Log.CorruptionPolicy.EXCEPTION, null, entry -> processLogEntry(entry.get())); System.out.println("Num Total Entries: " + entryCount); System.out.println("Num Conf Entries: " + numConfEntries); System.out.println("Num Metadata Entries: " + numMetadataEntries); @@ -70,8 +69,7 @@ public void dumpSegmentFile() throws IOException { } - private void processLogEntry(ReferenceCountedObject ref) { - final LogEntryProto proto = ref.retain(); + private void processLogEntry(LogEntryProto proto) { if (proto.hasConfigurationEntry()) { numConfEntries++; } else if (proto.hasMetadataEntry()) { @@ -79,13 +77,12 @@ private void processLogEntry(ReferenceCountedObject ref) { } else if (proto.hasStateMachineLogEntry()) { numStateMachineEntries++; } else { - System.out.println("Found an invalid entry: " + proto); + System.out.println("Found invalid entry" + proto.toString()); numInvalidEntries++; } String str = LogProtoUtils.toLogEntryString(proto, smLogToString); System.out.println(str); - ref.release(); } public static class Builder { From dd76b7e3bcbb9b280ba555511a6a29557942af89 Mon Sep 17 00:00:00 2001 From: Tsz-Wo Nicholas Sze Date: Mon, 26 Aug 2024 11:10:36 -0700 Subject: [PATCH 085/397] Revert "RATIS-2101. Move TermIndex.PRIVATE_CACHE to Util.CACHE (#1103)" This reverts commit 93eb32a8620fdd4e5119592ef32bc50590810c7b. --- .../apache/ratis/server/protocol/TermIndex.java | 15 ++++++--------- 1 file changed, 6 insertions(+), 9 deletions(-) diff --git a/ratis-server-api/src/main/java/org/apache/ratis/server/protocol/TermIndex.java b/ratis-server-api/src/main/java/org/apache/ratis/server/protocol/TermIndex.java index a8aa670613..6a9bd1cfb7 100644 --- a/ratis-server-api/src/main/java/org/apache/ratis/server/protocol/TermIndex.java +++ b/ratis-server-api/src/main/java/org/apache/ratis/server/protocol/TermIndex.java @@ -29,14 +29,11 @@ /** The term and the log index defined in the Raft consensus algorithm. */ public interface TermIndex extends Comparable { - class Util { - /** An LRU Cache for {@link TermIndex} instances */ - private static final Cache CACHE = CacheBuilder.newBuilder() - .maximumSize(1 << 16) - .expireAfterAccess(1, TimeUnit.MINUTES) - .build(); - } - TermIndex[] EMPTY_ARRAY = {}; + /** An LRU Cache for {@link TermIndex} instances */ + Cache PRIVATE_CACHE = CacheBuilder.newBuilder() + .maximumSize(1 << 16) + .expireAfterAccess(1, TimeUnit.MINUTES) + .build(); /** @return the term. */ long getTerm(); @@ -110,7 +107,7 @@ public String toString() { } }; try { - return Util.CACHE.get(key, () -> key); + return PRIVATE_CACHE.get(key, () -> key); } catch (ExecutionException e) { throw new IllegalStateException("Failed to valueOf(" + term + ", " + index + "), key=" + key, e); } From dceb1f6e32d7afd7335d614904b70457e91c856a Mon Sep 17 00:00:00 2001 From: Tsz-Wo Nicholas Sze Date: Mon, 26 Aug 2024 11:11:03 -0700 Subject: [PATCH 086/397] Revert "RATIS-2099. Cache TermIndexImpl instead of using anonymous class (#1100)" This reverts commit 428ce4ae3d5a0349f3425cb85ef1a3d38dea24b1. --- .../apache/ratis/server/protocol/TermIndex.java | 17 ++--------------- 1 file changed, 2 insertions(+), 15 deletions(-) diff --git a/ratis-server-api/src/main/java/org/apache/ratis/server/protocol/TermIndex.java b/ratis-server-api/src/main/java/org/apache/ratis/server/protocol/TermIndex.java index 6a9bd1cfb7..7def686bb5 100644 --- a/ratis-server-api/src/main/java/org/apache/ratis/server/protocol/TermIndex.java +++ b/ratis-server-api/src/main/java/org/apache/ratis/server/protocol/TermIndex.java @@ -19,21 +19,13 @@ import org.apache.ratis.proto.RaftProtos.LogEntryProto; import org.apache.ratis.proto.RaftProtos.TermIndexProto; -import org.apache.ratis.thirdparty.com.google.common.cache.Cache; -import org.apache.ratis.thirdparty.com.google.common.cache.CacheBuilder; import java.util.Comparator; import java.util.Optional; -import java.util.concurrent.ExecutionException; -import java.util.concurrent.TimeUnit; /** The term and the log index defined in the Raft consensus algorithm. */ public interface TermIndex extends Comparable { - /** An LRU Cache for {@link TermIndex} instances */ - Cache PRIVATE_CACHE = CacheBuilder.newBuilder() - .maximumSize(1 << 16) - .expireAfterAccess(1, TimeUnit.MINUTES) - .build(); + TermIndex[] EMPTY_ARRAY = {}; /** @return the term. */ long getTerm(); @@ -68,7 +60,7 @@ static TermIndex valueOf(LogEntryProto proto) { /** @return a {@link TermIndex} object. */ static TermIndex valueOf(long term, long index) { - final TermIndex key = new TermIndex() { + return new TermIndex() { @Override public long getTerm() { return term; @@ -106,10 +98,5 @@ public String toString() { return String.format("(t:%s, i:%s)", longToString(term), longToString(index)); } }; - try { - return PRIVATE_CACHE.get(key, () -> key); - } catch (ExecutionException e) { - throw new IllegalStateException("Failed to valueOf(" + term + ", " + index + "), key=" + key, e); - } } } \ No newline at end of file From 35bb472b6f4fd6262f4ca71185abc482e4038335 Mon Sep 17 00:00:00 2001 From: z-bb <35120306+z-bb@users.noreply.github.com> Date: Fri, 30 Aug 2024 23:40:11 +0800 Subject: [PATCH 087/397] RATIS-2145. Follower hangs until the next trigger to take a snapshot. (#1139) --- .../java/org/apache/ratis/grpc/server/GrpcLogAppender.java | 6 +----- 1 file changed, 1 insertion(+), 5 deletions(-) diff --git a/ratis-grpc/src/main/java/org/apache/ratis/grpc/server/GrpcLogAppender.java b/ratis-grpc/src/main/java/org/apache/ratis/grpc/server/GrpcLogAppender.java index 8580c42396..0229f8e848 100644 --- a/ratis-grpc/src/main/java/org/apache/ratis/grpc/server/GrpcLogAppender.java +++ b/ratis-grpc/src/main/java/org/apache/ratis/grpc/server/GrpcLogAppender.java @@ -435,7 +435,7 @@ private void sendRequest(AppendEntriesRequest request, } private void timeoutAppendRequest(long cid, boolean heartbeat) { - final AppendEntriesRequest pending = pendingRequests.handleTimeout(cid, heartbeat); + final AppendEntriesRequest pending = pendingRequests.remove(cid, heartbeat); if (pending != null) { final int errorCount = replyState.process(Event.TIMEOUT); LOG.warn("{}: Timed out {}appendEntries, errorCount={}, request={}", @@ -955,9 +955,5 @@ AppendEntriesRequest remove(AppendEntriesReplyProto reply) { AppendEntriesRequest remove(long cid, boolean isHeartbeat) { return isHeartbeat ? heartbeats.remove(cid): logRequests.remove(cid); } - - public AppendEntriesRequest handleTimeout(long callId, boolean heartbeat) { - return heartbeat ? heartbeats.remove(callId) : logRequests.get(callId); - } } } From dde2582b29fe369bbaca972ff5ad100c18566bbe Mon Sep 17 00:00:00 2001 From: Potato Date: Tue, 3 Sep 2024 11:42:14 +0800 Subject: [PATCH 088/397] RATIS-2146. Fixed possible issues caused by concurrent deletion and election when member changes (#1140) --- .../org/apache/ratis/server/impl/RaftServerImpl.java | 10 ++++++++++ 1 file changed, 10 insertions(+) diff --git a/ratis-server/src/main/java/org/apache/ratis/server/impl/RaftServerImpl.java b/ratis-server/src/main/java/org/apache/ratis/server/impl/RaftServerImpl.java index 8d7246fcf7..c682de8bfb 100644 --- a/ratis-server/src/main/java/org/apache/ratis/server/impl/RaftServerImpl.java +++ b/ratis-server/src/main/java/org/apache/ratis/server/impl/RaftServerImpl.java @@ -17,6 +17,7 @@ */ package org.apache.ratis.server.impl; +import java.util.concurrent.CountDownLatch; import org.apache.ratis.client.impl.ClientProtoUtils; import org.apache.ratis.conf.RaftProperties; import org.apache.ratis.metrics.Timekeeper; @@ -228,6 +229,7 @@ public long[] getFollowerNextIndices() { private final RaftServerJmxAdapter jmxAdapter = new RaftServerJmxAdapter(this); private final LeaderElectionMetrics leaderElectionMetrics; private final RaftServerMetricsImpl raftServerMetrics; + private final CountDownLatch closeFinishedLatch = new CountDownLatch(1); // To avoid append entry before complete start() method // For example, if thread1 start(), but before thread1 startAsFollower(), thread2 receive append entry @@ -462,6 +464,13 @@ void groupRemove(boolean deleteDirectory, boolean renameDirectory) { /* Shutdown is triggered here inorder to avoid any locked files. */ state.getStateMachineUpdater().setRemoving(); close(); + try { + closeFinishedLatch.await(); + } catch (InterruptedException e) { + Thread.currentThread().interrupt(); + LOG.warn("{}: Waiting closing interrupted, will not continue to remove group locally", getMemberId()); + return; + } getStateMachine().event().notifyGroupRemove(); if (deleteDirectory) { for (int i = 0; i < FileUtils.NUM_ATTEMPTS; i ++) { @@ -540,6 +549,7 @@ public void close() { } catch (Exception e) { LOG.warn(getMemberId() + ": Failed to shutdown serverExecutor", e); } + closeFinishedLatch.countDown(); }); } From 2cd3cca0e542c5ad53bde68bffafa5d1af6db5d7 Mon Sep 17 00:00:00 2001 From: OneSizeFitQuorum Date: Wed, 4 Sep 2024 10:00:06 +0800 Subject: [PATCH 089/397] Change version for the version 3.1.1 -rc2 --- pom.xml | 4 ++-- ratis-assembly/pom.xml | 2 +- ratis-client/pom.xml | 2 +- ratis-common/pom.xml | 2 +- ratis-docs/pom.xml | 2 +- ratis-examples/pom.xml | 2 +- ratis-experiments/pom.xml | 2 +- ratis-grpc/pom.xml | 2 +- ratis-metrics-api/pom.xml | 2 +- ratis-metrics-default/pom.xml | 2 +- ratis-metrics-dropwizard3/pom.xml | 2 +- ratis-netty/pom.xml | 2 +- ratis-proto/pom.xml | 2 +- ratis-replicated-map/pom.xml | 2 +- ratis-resource-bundle/pom.xml | 2 +- ratis-server-api/pom.xml | 2 +- ratis-server/pom.xml | 2 +- ratis-shell/pom.xml | 2 +- ratis-test/pom.xml | 2 +- ratis-tools/pom.xml | 2 +- 20 files changed, 21 insertions(+), 21 deletions(-) diff --git a/pom.xml b/pom.xml index dda19aec8d..00aee4949f 100644 --- a/pom.xml +++ b/pom.xml @@ -24,7 +24,7 @@ ratis org.apache.ratis - 3.1.1-SNAPSHOT + 3.1.1 Apache Ratis pom @@ -162,7 +162,7 @@ - 2024-08-22T17:50:05Z + 2024-09-04T02:00:06Z UTF-8 UTF-8 diff --git a/ratis-assembly/pom.xml b/ratis-assembly/pom.xml index 90ac0ac600..60f04f7695 100644 --- a/ratis-assembly/pom.xml +++ b/ratis-assembly/pom.xml @@ -17,7 +17,7 @@ ratis org.apache.ratis - 3.1.1-SNAPSHOT + 3.1.1 ratis-assembly diff --git a/ratis-client/pom.xml b/ratis-client/pom.xml index 3597fdddbe..1c4cac0f8e 100644 --- a/ratis-client/pom.xml +++ b/ratis-client/pom.xml @@ -17,7 +17,7 @@ ratis org.apache.ratis - 3.1.1-SNAPSHOT + 3.1.1 ratis-client diff --git a/ratis-common/pom.xml b/ratis-common/pom.xml index 066aa57ecf..1b25ae818a 100644 --- a/ratis-common/pom.xml +++ b/ratis-common/pom.xml @@ -17,7 +17,7 @@ ratis org.apache.ratis - 3.1.1-SNAPSHOT + 3.1.1 ratis-common diff --git a/ratis-docs/pom.xml b/ratis-docs/pom.xml index b44781db58..c794dd310e 100644 --- a/ratis-docs/pom.xml +++ b/ratis-docs/pom.xml @@ -20,7 +20,7 @@ https://maven.apache.org/xsd/maven-4.0.0.xsd"> ratis org.apache.ratis - 3.1.1-SNAPSHOT + 3.1.1 ratis-docs diff --git a/ratis-examples/pom.xml b/ratis-examples/pom.xml index fddbdc94be..58ce273c18 100644 --- a/ratis-examples/pom.xml +++ b/ratis-examples/pom.xml @@ -17,7 +17,7 @@ ratis org.apache.ratis - 3.1.1-SNAPSHOT + 3.1.1 ratis-examples diff --git a/ratis-experiments/pom.xml b/ratis-experiments/pom.xml index a8c43bf814..7addeb7bf5 100644 --- a/ratis-experiments/pom.xml +++ b/ratis-experiments/pom.xml @@ -17,7 +17,7 @@ ratis org.apache.ratis - 3.1.1-SNAPSHOT + 3.1.1 ratis-experiments diff --git a/ratis-grpc/pom.xml b/ratis-grpc/pom.xml index aa3d76ff51..b1379bfc42 100644 --- a/ratis-grpc/pom.xml +++ b/ratis-grpc/pom.xml @@ -17,7 +17,7 @@ ratis org.apache.ratis - 3.1.1-SNAPSHOT + 3.1.1 ratis-grpc diff --git a/ratis-metrics-api/pom.xml b/ratis-metrics-api/pom.xml index 4606f41d68..660f8d1f77 100644 --- a/ratis-metrics-api/pom.xml +++ b/ratis-metrics-api/pom.xml @@ -18,7 +18,7 @@ ratis org.apache.ratis - 3.1.1-SNAPSHOT + 3.1.1 ratis-metrics-api diff --git a/ratis-metrics-default/pom.xml b/ratis-metrics-default/pom.xml index f59b94fd7e..91bd4e1e3d 100644 --- a/ratis-metrics-default/pom.xml +++ b/ratis-metrics-default/pom.xml @@ -18,7 +18,7 @@ ratis org.apache.ratis - 3.1.1-SNAPSHOT + 3.1.1 ratis-metrics-default diff --git a/ratis-metrics-dropwizard3/pom.xml b/ratis-metrics-dropwizard3/pom.xml index c948c75797..0fc07eddee 100644 --- a/ratis-metrics-dropwizard3/pom.xml +++ b/ratis-metrics-dropwizard3/pom.xml @@ -18,7 +18,7 @@ ratis org.apache.ratis - 3.1.1-SNAPSHOT + 3.1.1 ratis-metrics-dropwizard3 diff --git a/ratis-netty/pom.xml b/ratis-netty/pom.xml index ead5cb43d5..113c84371a 100644 --- a/ratis-netty/pom.xml +++ b/ratis-netty/pom.xml @@ -17,7 +17,7 @@ ratis org.apache.ratis - 3.1.1-SNAPSHOT + 3.1.1 ratis-netty diff --git a/ratis-proto/pom.xml b/ratis-proto/pom.xml index 33e8e6abed..010bd96fc1 100644 --- a/ratis-proto/pom.xml +++ b/ratis-proto/pom.xml @@ -17,7 +17,7 @@ ratis org.apache.ratis - 3.1.1-SNAPSHOT + 3.1.1 ratis-proto diff --git a/ratis-replicated-map/pom.xml b/ratis-replicated-map/pom.xml index 57ddfe3ec8..77276f7f88 100644 --- a/ratis-replicated-map/pom.xml +++ b/ratis-replicated-map/pom.xml @@ -17,7 +17,7 @@ ratis org.apache.ratis - 3.1.1-SNAPSHOT + 3.1.1 ratis-replicated-map diff --git a/ratis-resource-bundle/pom.xml b/ratis-resource-bundle/pom.xml index 64c6a284b2..f22640a6e6 100644 --- a/ratis-resource-bundle/pom.xml +++ b/ratis-resource-bundle/pom.xml @@ -23,7 +23,7 @@ ratis org.apache.ratis - 3.1.1-SNAPSHOT + 3.1.1 .. diff --git a/ratis-server-api/pom.xml b/ratis-server-api/pom.xml index 672175ec1b..345d66a1b4 100644 --- a/ratis-server-api/pom.xml +++ b/ratis-server-api/pom.xml @@ -17,7 +17,7 @@ ratis org.apache.ratis - 3.1.1-SNAPSHOT + 3.1.1 ratis-server-api diff --git a/ratis-server/pom.xml b/ratis-server/pom.xml index b3abd5a9e9..cfc8a9e88e 100644 --- a/ratis-server/pom.xml +++ b/ratis-server/pom.xml @@ -17,7 +17,7 @@ ratis org.apache.ratis - 3.1.1-SNAPSHOT + 3.1.1 ratis-server diff --git a/ratis-shell/pom.xml b/ratis-shell/pom.xml index a9304ee715..4c3caef784 100644 --- a/ratis-shell/pom.xml +++ b/ratis-shell/pom.xml @@ -17,7 +17,7 @@ ratis org.apache.ratis - 3.1.1-SNAPSHOT + 3.1.1 ratis-shell diff --git a/ratis-test/pom.xml b/ratis-test/pom.xml index f00ba94393..03e7326e88 100644 --- a/ratis-test/pom.xml +++ b/ratis-test/pom.xml @@ -17,7 +17,7 @@ ratis org.apache.ratis - 3.1.1-SNAPSHOT + 3.1.1 ratis-test diff --git a/ratis-tools/pom.xml b/ratis-tools/pom.xml index fd13c14fc6..b7a0c15768 100644 --- a/ratis-tools/pom.xml +++ b/ratis-tools/pom.xml @@ -17,7 +17,7 @@ ratis org.apache.ratis - 3.1.1-SNAPSHOT + 3.1.1 ratis-tools From c8948acf1b03fa7cf07b2ce41540c4f597a7719f Mon Sep 17 00:00:00 2001 From: "Doroszlai, Attila" <6454655+adoroszlai@users.noreply.github.com> Date: Tue, 10 Sep 2024 08:13:07 +0200 Subject: [PATCH 090/397] RATIS-2153. ratis-version.properties missing from src bundle (#1147) (cherry picked from commit 19184326a00535d28f8c09dfcd37852c3a8b1266) --- ratis-assembly/src/main/assembly/src.xml | 1 + 1 file changed, 1 insertion(+) diff --git a/ratis-assembly/src/main/assembly/src.xml b/ratis-assembly/src/main/assembly/src.xml index 8c025c9c66..6aac4ec062 100644 --- a/ratis-assembly/src/main/assembly/src.xml +++ b/ratis-assembly/src/main/assembly/src.xml @@ -103,6 +103,7 @@ README.md mvnw.cmd pom.xml + src/** start-build-env.sh 0644 From d2be89a231e522828a7b45acad5883e317283594 Mon Sep 17 00:00:00 2001 From: 133tosakarin <97331129+133tosakarin@users.noreply.github.com> Date: Sat, 7 Sep 2024 10:56:42 +0800 Subject: [PATCH 091/397] RATIS-2149.Do not perform leader election if the current RaftServer has not started yet (#1143) --- .../java/org/apache/ratis/server/impl/LeaderElection.java | 4 ++++ .../java/org/apache/ratis/server/impl/RaftServerImpl.java | 4 ++++ 2 files changed, 8 insertions(+) diff --git a/ratis-server/src/main/java/org/apache/ratis/server/impl/LeaderElection.java b/ratis-server/src/main/java/org/apache/ratis/server/impl/LeaderElection.java index da9c513486..af25ae9126 100644 --- a/ratis-server/src/main/java/org/apache/ratis/server/impl/LeaderElection.java +++ b/ratis-server/src/main/java/org/apache/ratis/server/impl/LeaderElection.java @@ -241,6 +241,10 @@ public void run() { } try (AutoCloseable ignored = Timekeeper.start(server.getLeaderElectionMetrics().getLeaderElectionTimer())) { + if (!server.isRunning()) { + LOG.info("{}: skip since the server is not running", this); + return; + } for (int round = 0; shouldRun(); round++) { if (skipPreVote || askForVotes(Phase.PRE_VOTE, round)) { if (askForVotes(Phase.ELECTION, round)) { diff --git a/ratis-server/src/main/java/org/apache/ratis/server/impl/RaftServerImpl.java b/ratis-server/src/main/java/org/apache/ratis/server/impl/RaftServerImpl.java index c682de8bfb..a66c8ec5f7 100644 --- a/ratis-server/src/main/java/org/apache/ratis/server/impl/RaftServerImpl.java +++ b/ratis-server/src/main/java/org/apache/ratis/server/impl/RaftServerImpl.java @@ -1859,4 +1859,8 @@ public RaftServerMetricsImpl getRaftServerMetrics() { void onGroupLeaderElected() { transferLeadership.complete(TransferLeadership.Result.SUCCESS); } + + boolean isRunning() { + return startComplete.get() && lifeCycle.getCurrentState() == State.RUNNING; + } } From 852d0b758ce541dda5d5f45e74a2d1cb2d0e1e61 Mon Sep 17 00:00:00 2001 From: 133tosakarin <97331129+133tosakarin@users.noreply.github.com> Date: Sat, 7 Sep 2024 10:29:47 +0800 Subject: [PATCH 092/397] RATIS-2148. Snapshot transfer may cause followers to trigger reloadStateMachine incorrectly (#1145) --- .../ratis/server/impl/SnapshotInstallationHandler.java | 10 ++++++++++ 1 file changed, 10 insertions(+) diff --git a/ratis-server/src/main/java/org/apache/ratis/server/impl/SnapshotInstallationHandler.java b/ratis-server/src/main/java/org/apache/ratis/server/impl/SnapshotInstallationHandler.java index 4a63e64ee0..9b84e8f1d1 100644 --- a/ratis-server/src/main/java/org/apache/ratis/server/impl/SnapshotInstallationHandler.java +++ b/ratis-server/src/main/java/org/apache/ratis/server/impl/SnapshotInstallationHandler.java @@ -47,6 +47,7 @@ import java.io.IOException; import java.util.Optional; import java.util.concurrent.atomic.AtomicBoolean; +import java.util.concurrent.atomic.AtomicInteger; import java.util.concurrent.atomic.AtomicLong; import java.util.concurrent.atomic.AtomicReference; @@ -68,6 +69,7 @@ class SnapshotInstallationHandler { new AtomicReference<>(INVALID_TERM_INDEX); private final AtomicBoolean isSnapshotNull = new AtomicBoolean(); private final AtomicLong installedIndex = new AtomicLong(INVALID_LOG_INDEX); + private final AtomicInteger nextChunkIndex = new AtomicInteger(-1); SnapshotInstallationHandler(RaftServerImpl server, RaftProperties properties) { this.server = server; @@ -172,6 +174,12 @@ private InstallSnapshotReplyProto checkAndInstallSnapshot(InstallSnapshotRequest state.setLeader(leaderId, "installSnapshot"); server.updateLastRpcTime(FollowerState.UpdateType.INSTALL_SNAPSHOT_START); + if (snapshotChunkRequest.getRequestIndex() == 0) { + nextChunkIndex.set(0); + } else if (nextChunkIndex.get() != snapshotChunkRequest.getRequestIndex()) { + throw new IOException("Snapshot request already failed at chunk index " + nextChunkIndex.get() + + "; ignoring request with chunk index " + snapshotChunkRequest.getRequestIndex()); + } try { // Check and append the snapshot chunk. We simply put this in lock // considering a follower peer requiring a snapshot installation does not @@ -184,6 +192,8 @@ private InstallSnapshotReplyProto checkAndInstallSnapshot(InstallSnapshotRequest //TODO: We should only update State with installed snapshot once the request is done. state.installSnapshot(request); + int idx = nextChunkIndex.getAndIncrement(); + Preconditions.assertEquals(snapshotChunkRequest.getRequestIndex(), idx, "nextChunkIndex"); // update the committed index // re-load the state machine if this is the last chunk if (snapshotChunkRequest.getDone()) { From 94e430c29d40eaf92e1cad3e02c74cff768f2aaa Mon Sep 17 00:00:00 2001 From: 133tosakarin <97331129+133tosakarin@users.noreply.github.com> Date: Thu, 12 Sep 2024 23:42:06 +0800 Subject: [PATCH 093/397] RATIS-2154. The old leader may send appendEntries after term changed. (#1148) --- .../java/org/apache/ratis/server/impl/RaftServerImpl.java | 6 ++++-- 1 file changed, 4 insertions(+), 2 deletions(-) diff --git a/ratis-server/src/main/java/org/apache/ratis/server/impl/RaftServerImpl.java b/ratis-server/src/main/java/org/apache/ratis/server/impl/RaftServerImpl.java index a66c8ec5f7..0473564b39 100644 --- a/ratis-server/src/main/java/org/apache/ratis/server/impl/RaftServerImpl.java +++ b/ratis-server/src/main/java/org/apache/ratis/server/impl/RaftServerImpl.java @@ -571,11 +571,10 @@ private synchronized boolean changeToFollower( boolean allowListener, Object reason) { final RaftPeerRole old = role.getCurrentRole(); - final boolean metadataUpdated = state.updateCurrentTerm(newTerm); if (old == RaftPeerRole.LISTENER && !allowListener) { throw new IllegalStateException("Unexpected role " + old); } - + boolean metadataUpdated; if ((old != RaftPeerRole.FOLLOWER || force) && old != RaftPeerRole.LISTENER) { setRole(RaftPeerRole.FOLLOWER, reason); if (old == RaftPeerRole.LEADER) { @@ -596,8 +595,11 @@ private synchronized boolean changeToFollower( } else if (old == RaftPeerRole.FOLLOWER) { role.shutdownFollowerState(); } + metadataUpdated = state.updateCurrentTerm(newTerm); role.startFollowerState(this, reason); setFirstElection(reason); + } else { + metadataUpdated = state.updateCurrentTerm(newTerm); } return metadataUpdated; } From a29e8507038881f13ae1ece516f5a0be1cd38686 Mon Sep 17 00:00:00 2001 From: Chung En Lee Date: Fri, 13 Sep 2024 00:26:57 +0800 Subject: [PATCH 094/397] RATIS-2152. GrpcLogAppender stucks while sending an installSnapshot notification request (#1146) --- .../ratis/grpc/server/GrpcLogAppender.java | 34 +++++++------------ 1 file changed, 13 insertions(+), 21 deletions(-) diff --git a/ratis-grpc/src/main/java/org/apache/ratis/grpc/server/GrpcLogAppender.java b/ratis-grpc/src/main/java/org/apache/ratis/grpc/server/GrpcLogAppender.java index 0229f8e848..192bc75641 100644 --- a/ratis-grpc/src/main/java/org/apache/ratis/grpc/server/GrpcLogAppender.java +++ b/ratis-grpc/src/main/java/org/apache/ratis/grpc/server/GrpcLogAppender.java @@ -54,8 +54,8 @@ import java.util.*; import java.util.concurrent.CompletableFuture; import java.util.concurrent.ConcurrentHashMap; +import java.util.concurrent.ExecutionException; import java.util.concurrent.TimeUnit; -import java.util.concurrent.atomic.AtomicBoolean; import java.util.concurrent.atomic.AtomicLong; /** @@ -567,7 +567,7 @@ private void updateNextIndex(long replyNextIndex) { private class InstallSnapshotResponseHandler implements StreamObserver { private final String name = getFollower().getName() + "-" + JavaUtils.getClassSimpleName(getClass()); private final Queue pending; - private final AtomicBoolean done = new AtomicBoolean(false); + private final CompletableFuture done = new CompletableFuture<>(); private final boolean isNotificationOnly; InstallSnapshotResponseHandler() { @@ -628,12 +628,18 @@ void notifyInstallSnapshotFinished(InstallSnapshotResult result, long snapshotIn getServer().getStateMachine().event().notifySnapshotInstalled(result, snapshotIndex, getFollower().getPeer()); } - boolean isDone() { - return done.get(); + void waitForResponse() { + try { + done.get(); + } catch (InterruptedException e) { + Thread.currentThread().interrupt(); + } catch (ExecutionException e) { + throw new IllegalStateException("Failed to complete " + name, e); + } } void close() { - done.set(true); + done.complete(null); notifyLogAppender(); } @@ -767,14 +773,7 @@ private void installSnapshot(SnapshotInfo snapshot) { } return; } - - while (isRunning() && !responseHandler.isDone()) { - try { - getEventAwaitForSignal().await(getWaitTimeMs(), TimeUnit.MILLISECONDS); - } catch (InterruptedException ignored) { - Thread.currentThread().interrupt(); - } - } + responseHandler.waitForResponse(); if (responseHandler.hasAllResponse()) { getFollower().setSnapshotIndex(snapshot.getTermIndex().getIndex()); @@ -812,14 +811,7 @@ private void notifyInstallSnapshot(TermIndex firstAvailable) { } return; } - - while (isRunning() && !responseHandler.isDone()) { - try { - getEventAwaitForSignal().await(getWaitTimeMs(), TimeUnit.MILLISECONDS); - } catch (InterruptedException ignored) { - Thread.currentThread().interrupt(); - } - } + responseHandler.waitForResponse(); } /** From e50987b9ba018a68bc795877cd129fd13a0c7064 Mon Sep 17 00:00:00 2001 From: Potato Date: Fri, 13 Sep 2024 10:16:56 +0800 Subject: [PATCH 095/397] RATIS-2150. No need for manual assembly:single execution when mvn deploy #1144 Signed-off-by: OneSizeFitQuorum --- dev-support/make_rc.sh | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/dev-support/make_rc.sh b/dev-support/make_rc.sh index 64a6d3d413..fa6a2662a4 100755 --- a/dev-support/make_rc.sh +++ b/dev-support/make_rc.sh @@ -147,7 +147,7 @@ publish-svn() { publish-mvn(){ cd "$projectdir" - mvnFun -X clean deploy assembly:single -DskipTests=true -Prelease -Papache-release -Dgpg.keyname="${CODESIGNINGKEY}" + mvnFun -X clean deploy -DskipTests=true -Prelease -Papache-release -Dgpg.keyname="${CODESIGNINGKEY}" } if [ "$#" -ne 1 ]; then From c93e7b8a0b5263f67ba5113d2ae783fb33cb7f5d Mon Sep 17 00:00:00 2001 From: 133tosakarin <97331129+133tosakarin@users.noreply.github.com> Date: Sat, 14 Sep 2024 17:38:04 +0800 Subject: [PATCH 096/397] RATIS-2147. Md5 mismatch when snapshot install (#1142) --- .../src/main/java/org/apache/ratis/io/MD5Hash.java | 8 +++++--- .../apache/ratis/server/storage/SnapshotManager.java | 10 ++++------ 2 files changed, 9 insertions(+), 9 deletions(-) diff --git a/ratis-common/src/main/java/org/apache/ratis/io/MD5Hash.java b/ratis-common/src/main/java/org/apache/ratis/io/MD5Hash.java index e60bef9652..0d19feb938 100644 --- a/ratis-common/src/main/java/org/apache/ratis/io/MD5Hash.java +++ b/ratis-common/src/main/java/org/apache/ratis/io/MD5Hash.java @@ -30,13 +30,15 @@ public class MD5Hash { public static final int MD5_LEN = 16; private static final ThreadLocal DIGESTER_FACTORY = - ThreadLocal.withInitial(() -> { + ThreadLocal.withInitial(MD5Hash::newDigester); + + public static MessageDigest newDigester() { try { return MessageDigest.getInstance("MD5"); } catch (NoSuchAlgorithmException e) { - throw new RuntimeException(e); + throw new IllegalStateException("Failed to create MessageDigest for MD5", e); } - }); + } private byte[] digest; diff --git a/ratis-server/src/main/java/org/apache/ratis/server/storage/SnapshotManager.java b/ratis-server/src/main/java/org/apache/ratis/server/storage/SnapshotManager.java index 794604d66b..91c1ba5b95 100644 --- a/ratis-server/src/main/java/org/apache/ratis/server/storage/SnapshotManager.java +++ b/ratis-server/src/main/java/org/apache/ratis/server/storage/SnapshotManager.java @@ -27,7 +27,6 @@ import org.apache.ratis.statemachine.StateMachine; import org.apache.ratis.statemachine.StateMachineStorage; import org.apache.ratis.util.FileUtils; -import org.apache.ratis.util.JavaUtils; import org.apache.ratis.util.MD5FileUtil; import org.apache.ratis.util.MemoizedSupplier; import org.apache.ratis.util.Preconditions; @@ -63,7 +62,7 @@ public class SnapshotManager { private final Supplier snapshotDir; private final Supplier snapshotTmpDir; private final Function getRelativePath; - private final Supplier digester = JavaUtils.memoize(MD5Hash::getDigester); + private MessageDigest digester; SnapshotManager(RaftPeerId selfId, Supplier dir, StateMachineStorage smStorage) { this.selfId = selfId; @@ -88,7 +87,7 @@ private FileChannel open(FileChunkProto chunk, File tmpSnapshotFile) throws IOEx } // create the temp snapshot file and put padding inside out = FileUtils.newFileChannel(tmpSnapshotFile, StandardOpenOption.WRITE, StandardOpenOption.CREATE); - digester.get().reset(); + digester = MD5Hash.newDigester(); } else { if (!exists) { throw new FileNotFoundException("Chunk offset is non-zero but file is not found: " + tmpSnapshotFile @@ -114,7 +113,6 @@ public void installSnapshot(InstallSnapshotRequestProto request, StateMachine st // TODO: Make sure that subsequent requests for the same installSnapshot are coming in order, // and are not lost when whole request cycle is done. Check requestId and requestIndex here - for (FileChunkProto chunk : snapshotChunkRequest.getFileChunksList()) { SnapshotInfo pi = stateMachine.getLatestSnapshot(); if (pi != null && pi.getTermIndex().getIndex() >= lastIncludedIndex) { @@ -128,7 +126,7 @@ public void installSnapshot(InstallSnapshotRequestProto request, StateMachine st try (FileChannel out = open(chunk, tmpSnapshotFile)) { final ByteBuffer data = chunk.getData().asReadOnlyByteBuffer(); - digester.get().update(data.duplicate()); + digester.update(data.duplicate()); int written = 0; for(; data.remaining() > 0; ) { @@ -144,7 +142,7 @@ public void installSnapshot(InstallSnapshotRequestProto request, StateMachine st new MD5Hash(chunk.getFileDigest().toByteArray()); // calculate the checksum of the snapshot file and compare it with the // file digest in the request - final MD5Hash digest = new MD5Hash(digester.get().digest()); + final MD5Hash digest = new MD5Hash(digester.digest()); if (!digest.equals(expectedDigest)) { LOG.warn("The snapshot md5 digest {} does not match expected {}", digest, expectedDigest); From deddb24a3c0ce72d97124ed77d1f1f7e2b2dbfff Mon Sep 17 00:00:00 2001 From: Potato Date: Sun, 15 Sep 2024 10:31:19 +0800 Subject: [PATCH 097/397] RATIS-2157 Enhance make_rc.sh for non-first rc at release time #1149 Signed-off-by: OneSizeFitQuorum --- dev-support/make_rc.sh | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/dev-support/make_rc.sh b/dev-support/make_rc.sh index fa6a2662a4..dc56af595d 100755 --- a/dev-support/make_rc.sh +++ b/dev-support/make_rc.sh @@ -96,7 +96,7 @@ prepare-src() { git reset --hard git clean -fdx mvnFun versions:set -DnewVersion="$RATISVERSION" - git commit -a -m "Change version for the version $RATISVERSION $RC" + git commit --allow-empty -a -m "Change version for the version $RATISVERSION $RC" git config user.signingkey "${CODESIGNINGKEY}" git tag -s -m "Release $RATISVERSION $RC" ratis-"${RATISVERSION}${RC}" @@ -147,7 +147,7 @@ publish-svn() { publish-mvn(){ cd "$projectdir" - mvnFun -X clean deploy -DskipTests=true -Prelease -Papache-release -Dgpg.keyname="${CODESIGNINGKEY}" + mvnFun clean deploy -DskipTests=true -Prelease -Papache-release -Dgpg.keyname="${CODESIGNINGKEY}" } if [ "$#" -ne 1 ]; then From 1dea40686ed5ffebab1538f16889b1bb807f7cb9 Mon Sep 17 00:00:00 2001 From: Tsz-Wo Nicholas Sze Date: Tue, 17 Sep 2024 22:13:38 -0700 Subject: [PATCH 098/397] RATIS-2155. Add a builder for RatisShell. (#1150) --- .../grpc/client/GrpcClientProtocolClient.java | 1 + .../apache/ratis/grpc/server/GrpcService.java | 1 + .../ratis/grpc/MiniRaftClusterWithGrpc.java | 6 +- .../org/apache/ratis/shell/cli/CliUtils.java | 33 +---- .../apache/ratis/shell/cli/sh/RatisShell.java | 44 ++++++- .../shell/cli/sh/command/AbstractCommand.java | 14 +- .../cli/sh/command/AbstractRatisCommand.java | 6 +- .../ratis/shell/cli/sh/command/Context.java | 69 ++++++++++ .../shell/cli/sh/election/PauseCommand.java | 3 +- .../shell/cli/sh/election/ResumeCommand.java | 3 +- .../cli/sh/election/StepDownCommand.java | 3 +- .../cli/sh/election/TransferCommand.java | 3 +- .../shell/cli/sh/group/GroupListCommand.java | 2 +- .../ratis/shell/cli/sh/peer/AddCommand.java | 2 +- .../shell/cli/sh/peer/RemoveCommand.java | 3 +- .../shell/cli/sh/peer/SetPriorityCommand.java | 3 +- .../cli/sh/snapshot/TakeSnapshotCommand.java | 3 +- .../ratis/security/SecurityTestUtils.java | 17 +++ .../shell/cli/sh/TestSecureRatisShell.java | 123 ++++++++++++++++++ 19 files changed, 283 insertions(+), 56 deletions(-) create mode 100644 ratis-test/src/test/java/org/apache/ratis/shell/cli/sh/TestSecureRatisShell.java diff --git a/ratis-grpc/src/main/java/org/apache/ratis/grpc/client/GrpcClientProtocolClient.java b/ratis-grpc/src/main/java/org/apache/ratis/grpc/client/GrpcClientProtocolClient.java index a1f01a5129..3b9d512683 100644 --- a/ratis-grpc/src/main/java/org/apache/ratis/grpc/client/GrpcClientProtocolClient.java +++ b/ratis-grpc/src/main/java/org/apache/ratis/grpc/client/GrpcClientProtocolClient.java @@ -132,6 +132,7 @@ private ManagedChannel buildChannel(String address, GrpcTlsConfig tlsConf, channelBuilder.proxyDetector(uri -> null); if (tlsConf != null) { + LOG.debug("Setting TLS for {}", address); SslContextBuilder sslContextBuilder = GrpcSslContexts.forClient(); GrpcUtil.setTrustManager(sslContextBuilder, tlsConf.getTrustManager()); if (tlsConf.getMtlsEnabled()) { diff --git a/ratis-grpc/src/main/java/org/apache/ratis/grpc/server/GrpcService.java b/ratis-grpc/src/main/java/org/apache/ratis/grpc/server/GrpcService.java index 33f23315d2..864c0166a8 100644 --- a/ratis-grpc/src/main/java/org/apache/ratis/grpc/server/GrpcService.java +++ b/ratis-grpc/src/main/java/org/apache/ratis/grpc/server/GrpcService.java @@ -193,6 +193,7 @@ private NettyServerBuilder newNettyServerBuilder(String hostname, int port, Grpc .flowControlWindow(flowControlWindow.getSizeInt()); if (tlsConfig != null) { + LOG.info("Setting TLS for {}", address); SslContextBuilder sslContextBuilder = GrpcUtil.initSslContextBuilderForServer(tlsConfig.getKeyManager()); if (tlsConfig.getMtlsEnabled()) { sslContextBuilder.clientAuth(ClientAuth.REQUIRE); diff --git a/ratis-grpc/src/test/java/org/apache/ratis/grpc/MiniRaftClusterWithGrpc.java b/ratis-grpc/src/test/java/org/apache/ratis/grpc/MiniRaftClusterWithGrpc.java index 387574f858..84cb72bbe9 100644 --- a/ratis-grpc/src/test/java/org/apache/ratis/grpc/MiniRaftClusterWithGrpc.java +++ b/ratis-grpc/src/test/java/org/apache/ratis/grpc/MiniRaftClusterWithGrpc.java @@ -54,8 +54,12 @@ default Factory getFactory() { public static final DelayLocalExecutionInjection SEND_SERVER_REQUEST_INJECTION = new DelayLocalExecutionInjection(GrpcService.GRPC_SEND_SERVER_REQUEST); + + public MiniRaftClusterWithGrpc(String[] ids, RaftProperties properties, Parameters parameters) { + this(ids, new String[0], properties, parameters); + } - protected MiniRaftClusterWithGrpc(String[] ids, String[] listenerIds, RaftProperties properties, Parameters parameters) { + public MiniRaftClusterWithGrpc(String[] ids, String[] listenerIds, RaftProperties properties, Parameters parameters) { super(ids, listenerIds, properties, parameters); } diff --git a/ratis-shell/src/main/java/org/apache/ratis/shell/cli/CliUtils.java b/ratis-shell/src/main/java/org/apache/ratis/shell/cli/CliUtils.java index 98e21be2a8..1cecc665c6 100644 --- a/ratis-shell/src/main/java/org/apache/ratis/shell/cli/CliUtils.java +++ b/ratis-shell/src/main/java/org/apache/ratis/shell/cli/CliUtils.java @@ -18,17 +18,12 @@ package org.apache.ratis.shell.cli; import org.apache.ratis.client.RaftClient; -import org.apache.ratis.client.RaftClientConfigKeys; -import org.apache.ratis.conf.RaftProperties; import org.apache.ratis.protocol.GroupInfoReply; import org.apache.ratis.protocol.RaftClientReply; -import org.apache.ratis.protocol.RaftGroup; import org.apache.ratis.protocol.RaftGroupId; import org.apache.ratis.protocol.RaftPeer; import org.apache.ratis.protocol.RaftPeerId; import org.apache.ratis.protocol.exceptions.RaftException; -import org.apache.ratis.retry.ExponentialBackoffRetry; -import org.apache.ratis.util.TimeDuration; import org.apache.ratis.util.function.CheckedFunction; import java.io.IOException; @@ -36,24 +31,16 @@ import java.net.InetSocketAddress; import java.util.ArrayList; import java.util.Collection; -import java.util.concurrent.TimeUnit; import java.util.List; import java.util.Optional; -import java.util.Properties; +import java.util.UUID; import java.util.function.Supplier; import java.util.stream.Collectors; -import java.util.UUID; /** * Utilities for command line interface. */ public final class CliUtils { - private static final ExponentialBackoffRetry RETRY_POLICY = ExponentialBackoffRetry.newBuilder() - .setBaseSleepTime(TimeDuration.valueOf(1000, TimeUnit.MILLISECONDS)) - .setMaxAttempts(10) - .setMaxSleepTime(TimeDuration.valueOf(100_000, TimeUnit.MILLISECONDS)) - .build(); - private CliUtils() { // prevent instantiation } @@ -68,24 +55,6 @@ public static RaftPeerId getPeerId(String host, int port) { return RaftPeerId.getRaftPeerId(host + "_" + port); } - /** Create a new {@link RaftClient} from the given group. */ - public static RaftClient newRaftClient(RaftGroup group) { - RaftProperties properties = new RaftProperties(); - RaftClientConfigKeys.Rpc.setRequestTimeout(properties, - TimeDuration.valueOf(15, TimeUnit.SECONDS)); - - // Since ratis-shell support GENERIC_COMMAND_OPTIONS, here we should - // merge these options to raft properties to make it work. - final Properties sys = System.getProperties(); - sys.stringPropertyNames().forEach(key -> properties.set(key, sys.getProperty(key))); - - return RaftClient.newBuilder() - .setRaftGroup(group) - .setProperties(properties) - .setRetryPolicy(RETRY_POLICY) - .build(); - } - /** * Apply the given function to the given parameter a list. * diff --git a/ratis-shell/src/main/java/org/apache/ratis/shell/cli/sh/RatisShell.java b/ratis-shell/src/main/java/org/apache/ratis/shell/cli/sh/RatisShell.java index 2e53e31912..cdad683011 100644 --- a/ratis-shell/src/main/java/org/apache/ratis/shell/cli/sh/RatisShell.java +++ b/ratis-shell/src/main/java/org/apache/ratis/shell/cli/sh/RatisShell.java @@ -17,6 +17,9 @@ */ package org.apache.ratis.shell.cli.sh; +import org.apache.ratis.conf.Parameters; +import org.apache.ratis.conf.RaftProperties; +import org.apache.ratis.retry.RetryPolicy; import org.apache.ratis.shell.cli.AbstractShell; import org.apache.ratis.shell.cli.Command; import org.apache.ratis.shell.cli.sh.command.AbstractParentCommand; @@ -60,7 +63,11 @@ public static void main(String[] args) { } public RatisShell(PrintStream out) { - super(new Context(out)); + this(new Context(out)); + } + + private RatisShell(Context context) { + super(context); } @Override @@ -73,4 +80,39 @@ protected Map loadCommands(Context context) { return allParentCommands(context).stream() .collect(Collectors.toMap(Command::getCommandName, Function.identity())); } + + public static Builder newBuilder() { + return new Builder(); + } + + public static class Builder { + private PrintStream printStream = System.out; + private RetryPolicy retryPolicy; + private RaftProperties properties; + private Parameters parameters; + + public Builder setPrintStream(PrintStream printStream) { + this.printStream = printStream; + return this; + } + + public Builder setRetryPolicy(RetryPolicy retryPolicy) { + this.retryPolicy = retryPolicy; + return this; + } + + public Builder setProperties(RaftProperties properties) { + this.properties = properties; + return this; + } + + public Builder setParameters(Parameters parameters) { + this.parameters = parameters; + return this; + } + + public RatisShell build() { + return new RatisShell(new Context(printStream, false, retryPolicy, properties, parameters)); + } + } } diff --git a/ratis-shell/src/main/java/org/apache/ratis/shell/cli/sh/command/AbstractCommand.java b/ratis-shell/src/main/java/org/apache/ratis/shell/cli/sh/command/AbstractCommand.java index f02761de4d..e1d7f8e0b6 100644 --- a/ratis-shell/src/main/java/org/apache/ratis/shell/cli/sh/command/AbstractCommand.java +++ b/ratis-shell/src/main/java/org/apache/ratis/shell/cli/sh/command/AbstractCommand.java @@ -26,21 +26,25 @@ */ public abstract class AbstractCommand implements Command { - private final PrintStream printStream; + private final Context context; protected AbstractCommand(Context context) { - printStream = context.getPrintStream(); + this.context = context; + } + + protected Context getContext() { + return context; } protected PrintStream getPrintStream() { - return printStream; + return getContext().getPrintStream(); } protected void printf(String format, Object... args) { - printStream.printf(format, args); + getPrintStream().printf(format, args); } protected void println(Object message) { - printStream.println(message); + getPrintStream().println(message); } } diff --git a/ratis-shell/src/main/java/org/apache/ratis/shell/cli/sh/command/AbstractRatisCommand.java b/ratis-shell/src/main/java/org/apache/ratis/shell/cli/sh/command/AbstractRatisCommand.java index a9d391f866..aea1f7c4b3 100644 --- a/ratis-shell/src/main/java/org/apache/ratis/shell/cli/sh/command/AbstractRatisCommand.java +++ b/ratis-shell/src/main/java/org/apache/ratis/shell/cli/sh/command/AbstractRatisCommand.java @@ -66,7 +66,7 @@ public int run(CommandLine cl) throws IOException { final RaftGroupId groupIdSpecified = CliUtils.parseRaftGroupId(cl.getOptionValue(GROUPID_OPTION_NAME)); raftGroup = RaftGroup.valueOf(groupIdSpecified != null? groupIdSpecified: RaftGroupId.randomId(), peers); PrintStream printStream = getPrintStream(); - try (final RaftClient client = CliUtils.newRaftClient(raftGroup)) { + try (final RaftClient client = newRaftClient()) { final RaftGroupId remoteGroupId = CliUtils.getGroupId(client, peers, groupIdSpecified, printStream); groupInfoReply = CliUtils.getGroupInfo(client, peers, remoteGroupId, printStream); raftGroup = groupInfoReply.getGroup(); @@ -74,6 +74,10 @@ public int run(CommandLine cl) throws IOException { return 0; } + protected RaftClient newRaftClient() { + return getContext().newRaftClient(getRaftGroup()); + } + @Override public Options getOptions() { return new Options() diff --git a/ratis-shell/src/main/java/org/apache/ratis/shell/cli/sh/command/Context.java b/ratis-shell/src/main/java/org/apache/ratis/shell/cli/sh/command/Context.java index bae98dc0b5..6f80256d44 100644 --- a/ratis-shell/src/main/java/org/apache/ratis/shell/cli/sh/command/Context.java +++ b/ratis-shell/src/main/java/org/apache/ratis/shell/cli/sh/command/Context.java @@ -17,27 +17,59 @@ */ package org.apache.ratis.shell.cli.sh.command; +import org.apache.ratis.client.RaftClient; +import org.apache.ratis.client.RaftClientConfigKeys; +import org.apache.ratis.conf.Parameters; +import org.apache.ratis.conf.RaftProperties; +import org.apache.ratis.protocol.RaftGroup; +import org.apache.ratis.retry.ExponentialBackoffRetry; +import org.apache.ratis.retry.RetryPolicy; import org.apache.ratis.thirdparty.com.google.common.io.Closer; +import org.apache.ratis.util.TimeDuration; import java.io.Closeable; import java.io.IOException; import java.io.PrintStream; import java.util.Objects; +import java.util.Properties; +import java.util.concurrent.TimeUnit; /** * A context for ratis-shell. */ public final class Context implements Closeable { + private static final TimeDuration DEFAULT_REQUEST_TIMEOUT = TimeDuration.valueOf(15, TimeUnit.SECONDS); + private static final RetryPolicy DEFAULT_RETRY_POLICY = ExponentialBackoffRetry.newBuilder() + .setBaseSleepTime(TimeDuration.valueOf(1000, TimeUnit.MILLISECONDS)) + .setMaxAttempts(10) + .setMaxSleepTime(TimeDuration.valueOf(100_000, TimeUnit.MILLISECONDS)) + .build(); + private final PrintStream mPrintStream; private final Closer mCloser; + private final boolean cli; + private final RetryPolicy retryPolicy; + private final RaftProperties properties; + private final Parameters parameters; + /** * Build a context. * @param printStream the print stream */ public Context(PrintStream printStream) { + this(printStream, true, DEFAULT_RETRY_POLICY, new RaftProperties(), null); + } + + public Context(PrintStream printStream, boolean cli, RetryPolicy retryPolicy, + RaftProperties properties, Parameters parameters) { mCloser = Closer.create(); mPrintStream = mCloser.register(Objects.requireNonNull(printStream, "printStream == null")); + + this.cli = cli; + this.retryPolicy = retryPolicy != null? retryPolicy : DEFAULT_RETRY_POLICY; + this.properties = properties != null? properties : new RaftProperties(); + this.parameters = parameters; } /** @@ -47,6 +79,43 @@ public PrintStream getPrintStream() { return mPrintStream; } + /** Is this from CLI? */ + public boolean isCli() { + return cli; + } + + public RetryPolicy getRetryPolicy() { + return retryPolicy; + } + + public RaftProperties getProperties() { + return properties; + } + + public Parameters getParameters() { + return parameters; + } + + /** Create a new {@link RaftClient} from the given group. */ + public RaftClient newRaftClient(RaftGroup group) { + final RaftProperties p = getProperties(); + if (isCli()) { + RaftClientConfigKeys.Rpc.setRequestTimeout(p, DEFAULT_REQUEST_TIMEOUT); + + // Since ratis-shell support GENERIC_COMMAND_OPTIONS, here we should + // merge these options to raft p to make it work. + final Properties sys = System.getProperties(); + sys.stringPropertyNames().forEach(key -> p.set(key, sys.getProperty(key))); + } + + return RaftClient.newBuilder() + .setRaftGroup(group) + .setProperties(p) + .setParameters(getParameters()) + .setRetryPolicy(getRetryPolicy()) + .build(); + } + @Override public void close() throws IOException { mCloser.close(); diff --git a/ratis-shell/src/main/java/org/apache/ratis/shell/cli/sh/election/PauseCommand.java b/ratis-shell/src/main/java/org/apache/ratis/shell/cli/sh/election/PauseCommand.java index 242e1886a2..f8a627a8fa 100644 --- a/ratis-shell/src/main/java/org/apache/ratis/shell/cli/sh/election/PauseCommand.java +++ b/ratis-shell/src/main/java/org/apache/ratis/shell/cli/sh/election/PauseCommand.java @@ -24,7 +24,6 @@ import org.apache.ratis.protocol.RaftClientReply; import org.apache.ratis.protocol.RaftPeer; import org.apache.ratis.protocol.RaftPeerId; -import org.apache.ratis.shell.cli.CliUtils; import org.apache.ratis.shell.cli.sh.command.AbstractRatisCommand; import org.apache.ratis.shell.cli.sh.command.Context; @@ -61,7 +60,7 @@ public int run(CommandLine cl) throws IOException { printf("Peer not found: %s", strAddr); return -1; } - try(final RaftClient raftClient = CliUtils.newRaftClient(getRaftGroup())) { + try(final RaftClient raftClient = newRaftClient()) { RaftClientReply reply = raftClient.getLeaderElectionManagementApi(peerId).pause(); processReply(reply, () -> String.format("Failed to pause leader election on peer %s", strAddr)); printf(String.format("Successful pause leader election on peer %s", strAddr)); diff --git a/ratis-shell/src/main/java/org/apache/ratis/shell/cli/sh/election/ResumeCommand.java b/ratis-shell/src/main/java/org/apache/ratis/shell/cli/sh/election/ResumeCommand.java index dbcee7bd3f..1b5c80fac1 100644 --- a/ratis-shell/src/main/java/org/apache/ratis/shell/cli/sh/election/ResumeCommand.java +++ b/ratis-shell/src/main/java/org/apache/ratis/shell/cli/sh/election/ResumeCommand.java @@ -24,7 +24,6 @@ import org.apache.ratis.protocol.RaftClientReply; import org.apache.ratis.protocol.RaftPeer; import org.apache.ratis.protocol.RaftPeerId; -import org.apache.ratis.shell.cli.CliUtils; import org.apache.ratis.shell.cli.sh.command.AbstractRatisCommand; import org.apache.ratis.shell.cli.sh.command.Context; @@ -61,7 +60,7 @@ public int run(CommandLine cl) throws IOException { printf("Can't find a sever with the address:%s", strAddr); return -1; } - try(final RaftClient raftClient = CliUtils.newRaftClient(getRaftGroup())) { + try(final RaftClient raftClient = newRaftClient()) { RaftClientReply reply = raftClient.getLeaderElectionManagementApi(peerId).resume(); processReply(reply, () -> String.format("Failed to resume leader election on peer %s", strAddr)); printf(String.format("Successful pause leader election on peer %s", strAddr)); diff --git a/ratis-shell/src/main/java/org/apache/ratis/shell/cli/sh/election/StepDownCommand.java b/ratis-shell/src/main/java/org/apache/ratis/shell/cli/sh/election/StepDownCommand.java index da641a07ba..f18921e5ec 100644 --- a/ratis-shell/src/main/java/org/apache/ratis/shell/cli/sh/election/StepDownCommand.java +++ b/ratis-shell/src/main/java/org/apache/ratis/shell/cli/sh/election/StepDownCommand.java @@ -21,7 +21,6 @@ import org.apache.ratis.client.RaftClient; import org.apache.ratis.protocol.RaftClientReply; import org.apache.ratis.protocol.RaftPeerId; -import org.apache.ratis.shell.cli.CliUtils; import org.apache.ratis.shell.cli.sh.command.AbstractRatisCommand; import org.apache.ratis.shell.cli.sh.command.Context; @@ -48,7 +47,7 @@ public String getCommandName() { public int run(CommandLine cl) throws IOException { super.run(cl); - try (RaftClient client = CliUtils.newRaftClient(getRaftGroup())) { + try (RaftClient client = newRaftClient()) { RaftPeerId leaderId = RaftPeerId.valueOf(getLeader(getGroupInfoReply().getRoleInfoProto()).getId()); final RaftClientReply transferLeadershipReply = client.admin().transferLeadership(null, leaderId, 60_000); processReply(transferLeadershipReply, () -> "Failed to step down leader"); diff --git a/ratis-shell/src/main/java/org/apache/ratis/shell/cli/sh/election/TransferCommand.java b/ratis-shell/src/main/java/org/apache/ratis/shell/cli/sh/election/TransferCommand.java index 24eae7353a..88cfec9148 100644 --- a/ratis-shell/src/main/java/org/apache/ratis/shell/cli/sh/election/TransferCommand.java +++ b/ratis-shell/src/main/java/org/apache/ratis/shell/cli/sh/election/TransferCommand.java @@ -25,7 +25,6 @@ import org.apache.ratis.protocol.RaftClientReply; import org.apache.ratis.protocol.RaftPeer; import org.apache.ratis.protocol.exceptions.TransferLeadershipException; -import org.apache.ratis.shell.cli.CliUtils; import org.apache.ratis.shell.cli.sh.command.AbstractRatisCommand; import org.apache.ratis.shell.cli.sh.command.Context; import org.apache.ratis.util.TimeDuration; @@ -74,7 +73,7 @@ public int run(CommandLine cl) throws IOException { printf("Peer with address %s not found.", strAddr); return -2; } - try (RaftClient client = CliUtils.newRaftClient(getRaftGroup())) { + try (RaftClient client = newRaftClient()) { // transfer leadership if (!tryTransfer(client, newLeader, highestPriority, timeout.orElse(timeoutDefault))) { // legacy mode, transfer leadership by setting priority. diff --git a/ratis-shell/src/main/java/org/apache/ratis/shell/cli/sh/group/GroupListCommand.java b/ratis-shell/src/main/java/org/apache/ratis/shell/cli/sh/group/GroupListCommand.java index 5ee89c53cb..214ed15078 100644 --- a/ratis-shell/src/main/java/org/apache/ratis/shell/cli/sh/group/GroupListCommand.java +++ b/ratis-shell/src/main/java/org/apache/ratis/shell/cli/sh/group/GroupListCommand.java @@ -69,7 +69,7 @@ public int run(CommandLine cl) throws IOException { + " options are missing."); } - try(final RaftClient raftClient = CliUtils.newRaftClient(getRaftGroup())) { + try(final RaftClient raftClient = newRaftClient()) { GroupListReply reply = raftClient.getGroupManagementApi(peerId).list(); processReply(reply, () -> String.format("Failed to get group information of peerId %s (server %s)", peerId, address)); diff --git a/ratis-shell/src/main/java/org/apache/ratis/shell/cli/sh/peer/AddCommand.java b/ratis-shell/src/main/java/org/apache/ratis/shell/cli/sh/peer/AddCommand.java index 62c6c57934..be8f789954 100644 --- a/ratis-shell/src/main/java/org/apache/ratis/shell/cli/sh/peer/AddCommand.java +++ b/ratis-shell/src/main/java/org/apache/ratis/shell/cli/sh/peer/AddCommand.java @@ -81,7 +81,7 @@ public int run(CommandLine cl) throws IOException { "Both " + PEER_ID_OPTION_NAME + " and " + ADDRESS_OPTION_NAME + " options are missing."); } - try (RaftClient client = CliUtils.newRaftClient(getRaftGroup())) { + try (RaftClient client = newRaftClient()) { final Stream remaining = getPeerStream(RaftPeerRole.FOLLOWER); final Stream adding = ids.stream().map(raftPeerId -> RaftPeer.newBuilder() .setId(raftPeerId) diff --git a/ratis-shell/src/main/java/org/apache/ratis/shell/cli/sh/peer/RemoveCommand.java b/ratis-shell/src/main/java/org/apache/ratis/shell/cli/sh/peer/RemoveCommand.java index e2aa786b39..904a897880 100644 --- a/ratis-shell/src/main/java/org/apache/ratis/shell/cli/sh/peer/RemoveCommand.java +++ b/ratis-shell/src/main/java/org/apache/ratis/shell/cli/sh/peer/RemoveCommand.java @@ -25,7 +25,6 @@ import org.apache.ratis.protocol.RaftClientReply; import org.apache.ratis.protocol.RaftPeer; import org.apache.ratis.protocol.RaftPeerId; -import org.apache.ratis.shell.cli.CliUtils; import org.apache.ratis.shell.cli.sh.command.AbstractRatisCommand; import org.apache.ratis.shell.cli.sh.command.Context; @@ -66,7 +65,7 @@ public int run(CommandLine cl) throws IOException { throw new IllegalArgumentException( "Both " + PEER_ID_OPTION_NAME + " and " + ADDRESS_OPTION_NAME + " options are missing."); } - try (RaftClient client = CliUtils.newRaftClient(getRaftGroup())) { + try (RaftClient client = newRaftClient()) { final List peers = getPeerStream(RaftPeerRole.FOLLOWER) .filter(raftPeer -> !ids.contains(raftPeer.getId())).collect(Collectors.toList()); final List listeners = getPeerStream(RaftPeerRole.LISTENER) diff --git a/ratis-shell/src/main/java/org/apache/ratis/shell/cli/sh/peer/SetPriorityCommand.java b/ratis-shell/src/main/java/org/apache/ratis/shell/cli/sh/peer/SetPriorityCommand.java index e2d4d1a539..2834ef5cc8 100644 --- a/ratis-shell/src/main/java/org/apache/ratis/shell/cli/sh/peer/SetPriorityCommand.java +++ b/ratis-shell/src/main/java/org/apache/ratis/shell/cli/sh/peer/SetPriorityCommand.java @@ -24,7 +24,6 @@ import org.apache.ratis.proto.RaftProtos.RaftPeerRole; import org.apache.ratis.protocol.RaftClientReply; import org.apache.ratis.protocol.RaftPeer; -import org.apache.ratis.shell.cli.CliUtils; import org.apache.ratis.shell.cli.sh.command.AbstractRatisCommand; import org.apache.ratis.shell.cli.sh.command.Context; @@ -63,7 +62,7 @@ public int run(CommandLine cl) throws IOException { addressPriorityMap.put(str[0], Integer.parseInt(str[1])); } - try (RaftClient client = CliUtils.newRaftClient(getRaftGroup())) { + try (RaftClient client = newRaftClient()) { final List peers = getPeerStream(RaftPeerRole.FOLLOWER).map(peer -> { final Integer newPriority = addressPriorityMap.get(peer.getAddress()); final int priority = newPriority != null ? newPriority : peer.getPriority(); diff --git a/ratis-shell/src/main/java/org/apache/ratis/shell/cli/sh/snapshot/TakeSnapshotCommand.java b/ratis-shell/src/main/java/org/apache/ratis/shell/cli/sh/snapshot/TakeSnapshotCommand.java index e76f215f45..521b22e4e3 100644 --- a/ratis-shell/src/main/java/org/apache/ratis/shell/cli/sh/snapshot/TakeSnapshotCommand.java +++ b/ratis-shell/src/main/java/org/apache/ratis/shell/cli/sh/snapshot/TakeSnapshotCommand.java @@ -23,7 +23,6 @@ import org.apache.ratis.client.RaftClient; import org.apache.ratis.protocol.RaftClientReply; import org.apache.ratis.protocol.RaftPeerId; -import org.apache.ratis.shell.cli.CliUtils; import org.apache.ratis.shell.cli.sh.command.AbstractRatisCommand; import org.apache.ratis.shell.cli.sh.command.Context; @@ -58,7 +57,7 @@ public int run(CommandLine cl) throws IOException { } else { timeout = 3000; } - try(final RaftClient raftClient = CliUtils.newRaftClient(getRaftGroup())) { + try(final RaftClient raftClient = newRaftClient()) { if (cl.hasOption(PEER_ID_OPTION_NAME)) { peerId = RaftPeerId.getRaftPeerId(cl.getOptionValue(PEER_ID_OPTION_NAME)); } else { diff --git a/ratis-test/src/test/java/org/apache/ratis/security/SecurityTestUtils.java b/ratis-test/src/test/java/org/apache/ratis/security/SecurityTestUtils.java index c390f10f4a..cbb8e1b799 100644 --- a/ratis-test/src/test/java/org/apache/ratis/security/SecurityTestUtils.java +++ b/ratis-test/src/test/java/org/apache/ratis/security/SecurityTestUtils.java @@ -54,6 +54,23 @@ public interface SecurityTestUtils { ClassLoader CLASS_LOADER = SecurityTestUtils.class.getClassLoader(); + TrustManager EMPTY_TRUST_MANAGER = new X509TrustManager() { + @Override + public X509Certificate[] getAcceptedIssuers() { + return null; + } + + @Override + public void checkClientTrusted(X509Certificate[] certs, String authType) { } + + @Override + public void checkServerTrusted(X509Certificate[] certs, String authType) { } + }; + + static TrustManager emptyTrustManager() { + return EMPTY_TRUST_MANAGER; + } + static File getResource(String name) { final File file = Optional.ofNullable(CLASS_LOADER.getResource(name)) .map(URL::getFile) diff --git a/ratis-test/src/test/java/org/apache/ratis/shell/cli/sh/TestSecureRatisShell.java b/ratis-test/src/test/java/org/apache/ratis/shell/cli/sh/TestSecureRatisShell.java new file mode 100644 index 0000000000..785cd2e829 --- /dev/null +++ b/ratis-test/src/test/java/org/apache/ratis/shell/cli/sh/TestSecureRatisShell.java @@ -0,0 +1,123 @@ +/* + * 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.ratis.shell.cli.sh; + +import org.apache.ratis.BaseTest; +import org.apache.ratis.client.RaftClientConfigKeys; +import org.apache.ratis.conf.Parameters; +import org.apache.ratis.conf.RaftProperties; +import org.apache.ratis.grpc.GrpcConfigKeys; +import org.apache.ratis.grpc.GrpcTlsConfig; +import org.apache.ratis.grpc.MiniRaftClusterWithGrpc; +import org.apache.ratis.netty.NettyUtils; +import org.apache.ratis.protocol.RaftPeer; +import org.apache.ratis.security.SecurityTestUtils; +import org.apache.ratis.util.Slf4jUtils; +import org.apache.ratis.util.TimeDuration; +import org.junit.jupiter.api.Assertions; +import org.junit.jupiter.api.Test; +import org.slf4j.event.Level; + +import javax.net.ssl.KeyManager; +import javax.net.ssl.TrustManager; +import java.io.ByteArrayOutputStream; +import java.io.OutputStream; +import java.io.PrintStream; +import java.util.List; + +public class TestSecureRatisShell extends BaseTest { + { + Slf4jUtils.setLogLevel(NettyUtils.LOG, Level.DEBUG); + } + + private static final Parameters SERVER_PARAMETERS = new Parameters(); + private static final Parameters CLIENT_PARAMETERS = new Parameters(); + + static { + final TrustManager emptyTrustManager = SecurityTestUtils.emptyTrustManager(); + try { + final KeyManager serverKeyManager = SecurityTestUtils.getKeyManager(SecurityTestUtils::getServerKeyStore); + final GrpcTlsConfig serverConfig = new GrpcTlsConfig(serverKeyManager, emptyTrustManager, true); + GrpcConfigKeys.Server.setTlsConf(SERVER_PARAMETERS, serverConfig); + GrpcConfigKeys.Admin.setTlsConf(SERVER_PARAMETERS, serverConfig); + GrpcConfigKeys.Client.setTlsConf(SERVER_PARAMETERS, serverConfig); + } catch (Exception e) { + throw new IllegalStateException("Failed to init SERVER_PARAMETERS", e); + } + + try { + final KeyManager clientKeyManager = SecurityTestUtils.getKeyManager(SecurityTestUtils::getClientKeyStore); + final GrpcTlsConfig clientConfig = new GrpcTlsConfig(clientKeyManager, emptyTrustManager, true); + GrpcConfigKeys.Admin.setTlsConf(CLIENT_PARAMETERS, clientConfig); + GrpcConfigKeys.Client.setTlsConf(CLIENT_PARAMETERS, clientConfig); + } catch (Exception e) { + throw new IllegalStateException("Failed to init CLIENT_PARAMETERS", e); + } + } + + @Test + public void testRatisShell() throws Exception { + final String[] ids = {"s0"}; + final RaftProperties properties = new RaftProperties(); + RaftClientConfigKeys.Rpc.setRequestTimeout(properties, TimeDuration.ONE_MINUTE); + GrpcConfigKeys.TLS.setEnabled(properties, true); + GrpcConfigKeys.TLS.setMutualAuthnEnabled(properties, true); + + try(MiniRaftClusterWithGrpc cluster = new MiniRaftClusterWithGrpc(ids, properties, SERVER_PARAMETERS)) { + cluster.start(); + + runTestRatisShell(cluster, true); + runTestRatisShell(cluster, false); + } + } + + void runTestRatisShell(MiniRaftClusterWithGrpc cluster, boolean secure) throws Exception { + try(ByteArrayOutputStream out = new ByteArrayOutputStream(1 << 16); + RatisShell shell = newRatisShell(out, cluster.getProperties(), secure)) { + shell.run("group", "info", "-peers", toCliArg(cluster.getPeers())); + final String output = out.toString(); + LOG.info("output (secure? {}):\n{}", secure, output); + final String gid = cluster.getGroup().getGroupId().getUuid().toString(); + if (secure) { + Assertions.assertTrue(output.contains(gid), () -> gid + " not found for secure shell"); + } else { + Assertions.assertTrue(output.contains("Failed to get group ID"), "Unexpected output for unsecure shell"); + } + } + } + + static RatisShell newRatisShell(OutputStream out, RaftProperties properties, boolean secure) { + final PrintStream printStream = new PrintStream(out, true); + if (!secure) { + return new RatisShell(printStream); + } + return RatisShell.newBuilder() + .setPrintStream(printStream) + .setProperties(properties) + .setParameters(CLIENT_PARAMETERS) + .build(); + } + + static String toCliArg(List peers) { + final StringBuilder b = new StringBuilder(); + for(RaftPeer peer : peers) { + b.append(peer.getAdminAddress()).append(","); + } + return b.substring(0, b.length() - 1); + } +} From 5dea5f1ab32c1293e716012017dec6733a66ca49 Mon Sep 17 00:00:00 2001 From: 133tosakarin <97331129+133tosakarin@users.noreply.github.com> Date: Thu, 19 Sep 2024 15:02:32 +0800 Subject: [PATCH 099/397] RATIS-2158. Let the snapshot sender and receiver use a new digester each time (#1151) --- .../java/org/apache/ratis/server/storage/FileChunkReader.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/ratis-server/src/main/java/org/apache/ratis/server/storage/FileChunkReader.java b/ratis-server/src/main/java/org/apache/ratis/server/storage/FileChunkReader.java index 65bfc8b809..a12818443e 100644 --- a/ratis-server/src/main/java/org/apache/ratis/server/storage/FileChunkReader.java +++ b/ratis-server/src/main/java/org/apache/ratis/server/storage/FileChunkReader.java @@ -56,7 +56,7 @@ public FileChunkReader(FileInfo info, Path relativePath) throws IOException { this.relativePath = relativePath; final File f = info.getPath().toFile(); if (info.getFileDigest() == null) { - digester = MD5Hash.getDigester(); + digester = MD5Hash.newDigester(); this.in = new DigestInputStream(FileUtils.newInputStream(f), digester); } else { digester = null; From 4fc07e1f00a2657a82d18684b58f74377ade1b64 Mon Sep 17 00:00:00 2001 From: OneSizeFitQuorum Date: Thu, 19 Sep 2024 15:52:24 +0800 Subject: [PATCH 100/397] Change version for the version 3.1.1 -rc3 From 84d103bac113fda4d65405ef48531a8186c93451 Mon Sep 17 00:00:00 2001 From: Tsz-Wo Nicholas Sze Date: Sun, 29 Sep 2024 15:47:51 -0700 Subject: [PATCH 101/397] RATIS-2160. MetricRegistriesLoader should not print INFO message in CLI. (#1152) --- .../ratis/metrics/MetricRegistriesLoader.java | 37 ++++++++----------- .../metrics/TestMetricRegistriesLoader.java | 6 +++ 2 files changed, 21 insertions(+), 22 deletions(-) diff --git a/ratis-metrics-api/src/main/java/org/apache/ratis/metrics/MetricRegistriesLoader.java b/ratis-metrics-api/src/main/java/org/apache/ratis/metrics/MetricRegistriesLoader.java index 0837b0bdd1..8baac7a465 100644 --- a/ratis-metrics-api/src/main/java/org/apache/ratis/metrics/MetricRegistriesLoader.java +++ b/ratis-metrics-api/src/main/java/org/apache/ratis/metrics/MetricRegistriesLoader.java @@ -22,6 +22,7 @@ import java.util.ArrayList; import java.util.List; import java.util.ServiceLoader; +import java.util.stream.Collectors; import org.apache.ratis.thirdparty.com.google.common.annotations.VisibleForTesting; import org.apache.ratis.util.ReflectionUtils; @@ -30,10 +31,9 @@ public final class MetricRegistriesLoader { - private static final Logger LOG = LoggerFactory.getLogger(MetricRegistries.class); + private static final Logger LOG = LoggerFactory.getLogger(MetricRegistriesLoader.class); - private static final String DEFAULT_CLASS - = "org.apache.ratis.metrics.impl.MetricRegistriesImpl"; + static final String DEFAULT_CLASS = "org.apache.ratis.metrics.impl.MetricRegistriesImpl"; private MetricRegistriesLoader() { } @@ -56,32 +56,25 @@ public static MetricRegistries load() { * @return A {@link MetricRegistries} implementation. */ @VisibleForTesting - static MetricRegistries load(List availableImplementations) { - - if (availableImplementations.size() == 1) { - // One and only one instance -- what we want/expect - MetricRegistries impl = availableImplementations.get(0); - LOG.info("Loaded MetricRegistries " + impl.getClass()); - return impl; - } else if (availableImplementations.isEmpty()) { + static MetricRegistries load(List registries) { + if (registries.isEmpty()) { try { return ReflectionUtils.newInstance(Class.forName(DEFAULT_CLASS).asSubclass(MetricRegistries.class)); } catch (ClassNotFoundException e) { - throw new RuntimeException(e); + throw new IllegalStateException("Failed to load default MetricRegistries " + DEFAULT_CLASS, e); } + } + + final MetricRegistries first = registries.get(0); + if (registries.size() == 1) { + // One and only one instance -- what we want/expect + LOG.debug("Loaded {}", first.getClass()); } else { // Tell the user they're doing something wrong, and choose the first impl. - StringBuilder sb = new StringBuilder(); - for (MetricRegistries factory : availableImplementations) { - if (sb.length() > 0) { - sb.append(", "); - } - sb.append(factory.getClass()); - } - LOG.warn("Found multiple MetricRegistries implementations: " + sb - + ". Using first found implementation: " + availableImplementations.get(0)); - return availableImplementations.get(0); + final List> classes = registries.stream().map(Object::getClass).collect(Collectors.toList()); + LOG.warn("Found multiple MetricRegistries: {}. Using the first: {}", classes, first.getClass()); } + return first; } private static List getDefinedImplementations() { diff --git a/ratis-metrics-default/src/test/java/org/apache/ratis/metrics/TestMetricRegistriesLoader.java b/ratis-metrics-default/src/test/java/org/apache/ratis/metrics/TestMetricRegistriesLoader.java index cefc6b7947..9816cc99c5 100644 --- a/ratis-metrics-default/src/test/java/org/apache/ratis/metrics/TestMetricRegistriesLoader.java +++ b/ratis-metrics-default/src/test/java/org/apache/ratis/metrics/TestMetricRegistriesLoader.java @@ -34,6 +34,12 @@ * Test class for {@link MetricRegistriesLoader}. */ public class TestMetricRegistriesLoader { + @Test + public void testLoadEmptyInstance() { + MetricRegistries instance = MetricRegistriesLoader.load(Collections.emptyList()); + assertEquals(MetricRegistriesLoader.DEFAULT_CLASS, instance.getClass().getName()); + } + @Test public void testLoadSingleInstance() { MetricRegistries loader = mock(MetricRegistries.class); From 8dc31f439af15774bd29583c4ae32435b4fb3fea Mon Sep 17 00:00:00 2001 From: 133tosakarin <97331129+133tosakarin@users.noreply.github.com> Date: Mon, 30 Sep 2024 14:43:38 +0800 Subject: [PATCH 102/397] RATIS-2162. When closing leaderState, if the logAppender thread sends a snapshot, a deadlock may occur. (#1154) --- .../server/impl/ConfigurationManager.java | 10 +++--- .../ratis/server/impl/FollowerState.java | 14 +++++++- .../ratis/server/impl/LeaderElection.java | 14 +++++++- .../ratis/server/impl/LeaderStateImpl.java | 7 ++-- .../ratis/server/impl/RaftServerImpl.java | 32 +++++++++++-------- .../apache/ratis/server/impl/RoleInfo.java | 26 +++++++-------- .../ratis/server/leader/LogAppenderBase.java | 2 +- 7 files changed, 64 insertions(+), 41 deletions(-) diff --git a/ratis-server/src/main/java/org/apache/ratis/server/impl/ConfigurationManager.java b/ratis-server/src/main/java/org/apache/ratis/server/impl/ConfigurationManager.java index 10c59c8b19..6d3f68d5ce 100644 --- a/ratis-server/src/main/java/org/apache/ratis/server/impl/ConfigurationManager.java +++ b/ratis-server/src/main/java/org/apache/ratis/server/impl/ConfigurationManager.java @@ -41,11 +41,9 @@ public class ConfigurationManager { * The current raft configuration. If configurations is not empty, should be * the last entry of the map. Otherwise is initialConf. */ - @SuppressWarnings({"squid:S3077"}) // Suppress volatile for generic type - private volatile RaftConfigurationImpl currentConf; + private RaftConfigurationImpl currentConf; /** Cache the peer corresponding to {@link #id}. */ - @SuppressWarnings({"squid:S3077"}) // Suppress volatile for generic type - private volatile RaftPeer currentPeer; + private RaftPeer currentPeer; ConfigurationManager(RaftPeerId id, RaftConfigurationImpl initialConf) { this.id = id; @@ -78,11 +76,11 @@ private void addRaftConfigurationImpl(long logIndex, RaftConfigurationImpl conf) } } - RaftConfigurationImpl getCurrent() { + synchronized RaftConfigurationImpl getCurrent() { return currentConf; } - RaftPeer getCurrentPeer() { + synchronized RaftPeer getCurrentPeer() { return currentPeer; } diff --git a/ratis-server/src/main/java/org/apache/ratis/server/impl/FollowerState.java b/ratis-server/src/main/java/org/apache/ratis/server/impl/FollowerState.java index e980daede5..fa61e90883 100644 --- a/ratis-server/src/main/java/org/apache/ratis/server/impl/FollowerState.java +++ b/ratis-server/src/main/java/org/apache/ratis/server/impl/FollowerState.java @@ -26,6 +26,7 @@ import org.slf4j.Logger; import org.slf4j.LoggerFactory; +import java.util.concurrent.CompletableFuture; import java.util.concurrent.TimeUnit; import java.util.concurrent.atomic.AtomicInteger; import java.util.function.ToIntFunction; @@ -62,6 +63,7 @@ int update(AtomicInteger outstanding) { @SuppressWarnings({"squid:S3077"}) // Suppress volatile for generic type private volatile Timestamp lastRpcTime = creationTime; private volatile boolean isRunning = true; + private final CompletableFuture stopped = new CompletableFuture<>(); private final AtomicInteger outstandingOp = new AtomicInteger(); FollowerState(RaftServerImpl server, Object reason) { @@ -93,8 +95,10 @@ boolean isCurrentLeaderValid() { return lastRpcTime.elapsedTime().compareTo(server.properties().minRpcTimeout()) < 0; } - void stopRunning() { + CompletableFuture stopRunning() { this.isRunning = false; + interrupt(); + return stopped; } boolean lostMajorityHeartbeatsRecently() { @@ -122,6 +126,14 @@ private boolean shouldRun() { @Override public void run() { + try { + runImpl(); + } finally { + stopped.complete(null); + } + } + + private void runImpl() { final TimeDuration sleepDeviationThreshold = server.getSleepDeviationThreshold(); while (shouldRun()) { final TimeDuration electionTimeout = server.getRandomElectionTimeout(); diff --git a/ratis-server/src/main/java/org/apache/ratis/server/impl/LeaderElection.java b/ratis-server/src/main/java/org/apache/ratis/server/impl/LeaderElection.java index af25ae9126..a5bfba7bec 100644 --- a/ratis-server/src/main/java/org/apache/ratis/server/impl/LeaderElection.java +++ b/ratis-server/src/main/java/org/apache/ratis/server/impl/LeaderElection.java @@ -46,6 +46,7 @@ import java.util.Map; import java.util.Optional; import java.util.concurrent.Callable; +import java.util.concurrent.CompletableFuture; import java.util.concurrent.ExecutionException; import java.util.concurrent.ExecutorCompletionService; import java.util.concurrent.ExecutorService; @@ -183,6 +184,7 @@ public String toString() { private final String name; private final LifeCycle lifeCycle; private final Daemon daemon; + private final CompletableFuture stopped = new CompletableFuture<>(); private final RaftServerImpl server; private final boolean skipPreVote; @@ -223,8 +225,10 @@ private void startIfNew(Runnable starter) { } } - void shutdown() { + CompletableFuture shutdown() { lifeCycle.checkStateAndClose(); + stopped.complete(null); + return stopped; } @VisibleForTesting @@ -234,6 +238,14 @@ LifeCycle.State getCurrentState() { @Override public void run() { + try { + runImpl(); + } finally { + stopped.complete(null); + } + } + + private void runImpl() { if (!lifeCycle.compareAndTransition(STARTING, RUNNING)) { final LifeCycle.State state = lifeCycle.getCurrentState(); LOG.info("{}: skip running since this is already {}", this, state); diff --git a/ratis-server/src/main/java/org/apache/ratis/server/impl/LeaderStateImpl.java b/ratis-server/src/main/java/org/apache/ratis/server/impl/LeaderStateImpl.java index 68121cef6a..3d8bc22195 100644 --- a/ratis-server/src/main/java/org/apache/ratis/server/impl/LeaderStateImpl.java +++ b/ratis-server/src/main/java/org/apache/ratis/server/impl/LeaderStateImpl.java @@ -223,11 +223,8 @@ boolean removeAll(Collection c) { } CompletableFuture stopAll() { - final CompletableFuture[] futures = new CompletableFuture[senders.size()]; - for(int i = 0; i < futures.length; i++) { - futures[i] = senders.get(i).stopAsync(); - } - return CompletableFuture.allOf(futures); + return CompletableFuture.allOf(senders.stream(). + map(LogAppender::stopAsync).toArray(CompletableFuture[]::new)); } } diff --git a/ratis-server/src/main/java/org/apache/ratis/server/impl/RaftServerImpl.java b/ratis-server/src/main/java/org/apache/ratis/server/impl/RaftServerImpl.java index 0473564b39..ae158ad75e 100644 --- a/ratis-server/src/main/java/org/apache/ratis/server/impl/RaftServerImpl.java +++ b/ratis-server/src/main/java/org/apache/ratis/server/impl/RaftServerImpl.java @@ -132,6 +132,7 @@ import java.util.concurrent.ThreadLocalRandom; import java.util.concurrent.TimeUnit; import java.util.concurrent.atomic.AtomicBoolean; +import java.util.concurrent.atomic.AtomicReference; import java.util.function.Function; import java.util.function.Supplier; import java.util.stream.Collectors; @@ -565,20 +566,23 @@ void setFirstElection(Object reason) { * @param force Force to start a new {@link FollowerState} even if this server is already a follower. * @return if the term/votedFor should be updated to the new term */ - private synchronized boolean changeToFollower( - long newTerm, - boolean force, - boolean allowListener, - Object reason) { + private boolean changeToFollower(long newTerm, boolean force, boolean allowListener, Object reason) { + final AtomicReference metadataUpdated = new AtomicReference<>(); + changeToFollowerAsync(newTerm, force, allowListener, reason, metadataUpdated).join(); + return metadataUpdated.get(); + } + + private synchronized CompletableFuture changeToFollowerAsync( + long newTerm, boolean force, boolean allowListener, Object reason, AtomicReference metadataUpdated) { final RaftPeerRole old = role.getCurrentRole(); if (old == RaftPeerRole.LISTENER && !allowListener) { throw new IllegalStateException("Unexpected role " + old); } - boolean metadataUpdated; + CompletableFuture future = CompletableFuture.completedFuture(null); if ((old != RaftPeerRole.FOLLOWER || force) && old != RaftPeerRole.LISTENER) { setRole(RaftPeerRole.FOLLOWER, reason); if (old == RaftPeerRole.LEADER) { - role.shutdownLeaderState(false) + future = role.shutdownLeaderState(false) .exceptionally(e -> { if (e != null) { if (!getInfo().isAlive()) { @@ -587,21 +591,21 @@ private synchronized boolean changeToFollower( } } throw new CompletionException("Failed to shutdownLeaderState: " + this, e); - }) - .join(); + }); state.setLeader(null, reason); } else if (old == RaftPeerRole.CANDIDATE) { - role.shutdownLeaderElection(); + future = role.shutdownLeaderElection(); } else if (old == RaftPeerRole.FOLLOWER) { - role.shutdownFollowerState(); + future = role.shutdownFollowerState(); } - metadataUpdated = state.updateCurrentTerm(newTerm); + + metadataUpdated.set(state.updateCurrentTerm(newTerm)); role.startFollowerState(this, reason); setFirstElection(reason); } else { - metadataUpdated = state.updateCurrentTerm(newTerm); + metadataUpdated.set(state.updateCurrentTerm(newTerm)); } - return metadataUpdated; + return future; } synchronized void changeToFollowerAndPersistMetadata( diff --git a/ratis-server/src/main/java/org/apache/ratis/server/impl/RoleInfo.java b/ratis-server/src/main/java/org/apache/ratis/server/impl/RoleInfo.java index 5eb01a9d6b..a5cd7da665 100644 --- a/ratis-server/src/main/java/org/apache/ratis/server/impl/RoleInfo.java +++ b/ratis-server/src/main/java/org/apache/ratis/server/impl/RoleInfo.java @@ -47,7 +47,7 @@ class RoleInfo { public static final Logger LOG = LoggerFactory.getLogger(RoleInfo.class); private final RaftPeerId id; - private volatile RaftPeerRole role; + private final AtomicReference role = new AtomicReference<>(); /** Used when the peer is leader */ private final AtomicReference leaderState = new AtomicReference<>(); /** Used when the peer is follower, to monitor election timeout */ @@ -64,7 +64,7 @@ class RoleInfo { } void transitionRole(RaftPeerRole newRole) { - this.role = newRole; + this.role.set(newRole); this.transitionTime.set(Timestamp.currentTime()); } @@ -73,7 +73,7 @@ long getRoleElapsedTimeMs() { } RaftPeerRole getCurrentRole() { - return role; + return role.get(); } boolean isLeaderReady() { @@ -113,13 +113,13 @@ void startFollowerState(RaftServerImpl server, Object reason) { updateAndGet(followerState, new FollowerState(server, reason)).start(); } - void shutdownFollowerState() { + CompletableFuture shutdownFollowerState() { final FollowerState follower = followerState.getAndSet(null); - if (follower != null) { - LOG.info("{}: shutdown {}", id, follower); - follower.stopRunning(); - follower.interrupt(); + if (follower == null) { + return CompletableFuture.completedFuture(null); } + LOG.info("{}: shutdown {}", id, follower); + return follower.stopRunning(); } void startLeaderElection(RaftServerImpl server, boolean force) { @@ -133,13 +133,13 @@ void setLeaderElectionPause(boolean pause) { pauseLeaderElection.set(pause); } - void shutdownLeaderElection() { + CompletableFuture shutdownLeaderElection() { final LeaderElection election = leaderElection.getAndSet(null); - if (election != null) { - LOG.info("{}: shutdown {}", id, election); - election.shutdown(); - // no need to interrupt the election thread + if (election == null) { + return CompletableFuture.completedFuture(null); } + LOG.info("{}: shutdown {}", id, election); + return election.shutdown(); } private T updateAndGet(AtomicReference ref, T current) { diff --git a/ratis-server/src/main/java/org/apache/ratis/server/leader/LogAppenderBase.java b/ratis-server/src/main/java/org/apache/ratis/server/leader/LogAppenderBase.java index 958cc6fa81..5a27cda510 100644 --- a/ratis-server/src/main/java/org/apache/ratis/server/leader/LogAppenderBase.java +++ b/ratis-server/src/main/java/org/apache/ratis/server/leader/LogAppenderBase.java @@ -124,7 +124,7 @@ public void start() { @Override public boolean isRunning() { - return daemon.isWorking(); + return daemon.isWorking() && server.getInfo().isLeader(); } @Override From 8679f138fb4f96a61dde32226e02891b67bc8f31 Mon Sep 17 00:00:00 2001 From: 133tosakarin <97331129+133tosakarin@users.noreply.github.com> Date: Wed, 9 Oct 2024 08:06:22 +0800 Subject: [PATCH 103/397] RATIS-2165. Do not include RaftClientRequest Message in ReadIndexRequest (#1160) --- .../java/org/apache/ratis/client/impl/ClientProtoUtils.java | 6 +++++- .../java/org/apache/ratis/server/impl/ServerProtoUtils.java | 2 +- 2 files changed, 6 insertions(+), 2 deletions(-) diff --git a/ratis-client/src/main/java/org/apache/ratis/client/impl/ClientProtoUtils.java b/ratis-client/src/main/java/org/apache/ratis/client/impl/ClientProtoUtils.java index 5e217e7da2..44e3a5cd63 100644 --- a/ratis-client/src/main/java/org/apache/ratis/client/impl/ClientProtoUtils.java +++ b/ratis-client/src/main/java/org/apache/ratis/client/impl/ClientProtoUtils.java @@ -204,9 +204,13 @@ static ByteBuffer toRaftClientRequestProtoByteBuffer(RaftClientRequest request) } static RaftClientRequestProto toRaftClientRequestProto(RaftClientRequest request) { + return toRaftClientRequestProto(request, true); + } + + static RaftClientRequestProto toRaftClientRequestProto(RaftClientRequest request, boolean withMsg) { final RaftClientRequestProto.Builder b = RaftClientRequestProto.newBuilder() .setRpcRequest(toRaftRpcRequestProtoBuilder(request)); - if (request.getMessage() != null) { + if (withMsg && request.getMessage() != null) { b.setMessage(toClientMessageEntryProtoBuilder(request.getMessage())); } diff --git a/ratis-server/src/main/java/org/apache/ratis/server/impl/ServerProtoUtils.java b/ratis-server/src/main/java/org/apache/ratis/server/impl/ServerProtoUtils.java index e35cb23867..29a42f65a0 100644 --- a/ratis-server/src/main/java/org/apache/ratis/server/impl/ServerProtoUtils.java +++ b/ratis-server/src/main/java/org/apache/ratis/server/impl/ServerProtoUtils.java @@ -114,7 +114,7 @@ static ReadIndexRequestProto toReadIndexRequestProto( RaftClientRequest clientRequest, RaftGroupMemberId requestorId, RaftPeerId replyId) { return ReadIndexRequestProto.newBuilder() .setServerRequest(ClientProtoUtils.toRaftRpcRequestProtoBuilder(requestorId, replyId)) - .setClientRequest(ClientProtoUtils.toRaftClientRequestProto(clientRequest)) + .setClientRequest(ClientProtoUtils.toRaftClientRequestProto(clientRequest, false)) .build(); } From 69c2c0d7cff469ec5b4085801778be212511afc9 Mon Sep 17 00:00:00 2001 From: 133tosakarin <97331129+133tosakarin@users.noreply.github.com> Date: Wed, 9 Oct 2024 12:22:15 +0800 Subject: [PATCH 104/397] RATIS-2166. Update nextChunkIndex when snapshot is ALREADY_INSTALLED (#1159) --- .../ratis/server/impl/SnapshotInstallationHandler.java | 8 ++++++-- 1 file changed, 6 insertions(+), 2 deletions(-) diff --git a/ratis-server/src/main/java/org/apache/ratis/server/impl/SnapshotInstallationHandler.java b/ratis-server/src/main/java/org/apache/ratis/server/impl/SnapshotInstallationHandler.java index 9b84e8f1d1..537b384c64 100644 --- a/ratis-server/src/main/java/org/apache/ratis/server/impl/SnapshotInstallationHandler.java +++ b/ratis-server/src/main/java/org/apache/ratis/server/impl/SnapshotInstallationHandler.java @@ -185,6 +185,7 @@ private InstallSnapshotReplyProto checkAndInstallSnapshot(InstallSnapshotRequest // considering a follower peer requiring a snapshot installation does not // have a lot of requests if (state.getLog().getLastCommittedIndex() >= lastIncludedIndex) { + nextChunkIndex.set(snapshotChunkRequest.getRequestIndex() + 1); return toInstallSnapshotReplyProto(leaderId, getMemberId(), currentTerm, snapshotChunkRequest.getRequestIndex(), InstallSnapshotResult.ALREADY_INSTALLED); } @@ -192,8 +193,11 @@ private InstallSnapshotReplyProto checkAndInstallSnapshot(InstallSnapshotRequest //TODO: We should only update State with installed snapshot once the request is done. state.installSnapshot(request); - int idx = nextChunkIndex.getAndIncrement(); - Preconditions.assertEquals(snapshotChunkRequest.getRequestIndex(), idx, "nextChunkIndex"); + final int expectedChunkIndex = nextChunkIndex.getAndIncrement(); + if (expectedChunkIndex != snapshotChunkRequest.getRequestIndex()) { + throw new IOException("Unexpected request chunk index: " + snapshotChunkRequest.getRequestIndex() + + " (the expected index is " + expectedChunkIndex + ")"); + } // update the committed index // re-load the state machine if this is the last chunk if (snapshotChunkRequest.getDone()) { From 4ff9b8bc40d516914db6aa3962bf9bd9ea9866cd Mon Sep 17 00:00:00 2001 From: 133tosakarin <97331129+133tosakarin@users.noreply.github.com> Date: Sat, 12 Oct 2024 05:55:35 +0800 Subject: [PATCH 105/397] RATIS-2172. RaftServer may lose FollowerState (#1166) --- .../ratis/util/CodeInjectionForTesting.java | 5 ++ .../ratis/server/impl/FollowerState.java | 13 +++- .../ratis/server/impl/LeaderElection.java | 4 - .../ratis/server/impl/RaftServerImpl.java | 6 +- .../server/impl/LeaderElectionTests.java | 77 +++++++++++++++++++ 5 files changed, 96 insertions(+), 9 deletions(-) diff --git a/ratis-common/src/main/java/org/apache/ratis/util/CodeInjectionForTesting.java b/ratis-common/src/main/java/org/apache/ratis/util/CodeInjectionForTesting.java index a7d36ac0eb..112f6bd250 100644 --- a/ratis-common/src/main/java/org/apache/ratis/util/CodeInjectionForTesting.java +++ b/ratis-common/src/main/java/org/apache/ratis/util/CodeInjectionForTesting.java @@ -68,4 +68,9 @@ public static boolean execute(String injectionPoint, Object localId, } return code.execute(localId, remoteId, args); } + + /** Remove an injection point. */ + public static void remove(String injectionPoint) { + INJECTION_POINTS.remove(injectionPoint); + } } diff --git a/ratis-server/src/main/java/org/apache/ratis/server/impl/FollowerState.java b/ratis-server/src/main/java/org/apache/ratis/server/impl/FollowerState.java index fa61e90883..1be160f182 100644 --- a/ratis-server/src/main/java/org/apache/ratis/server/impl/FollowerState.java +++ b/ratis-server/src/main/java/org/apache/ratis/server/impl/FollowerState.java @@ -133,6 +133,14 @@ public void run() { } } + private boolean roleChangeChecking(TimeDuration electionTimeout) { + return outstandingOp.get() == 0 + && isRunning && server.getInfo().isFollower() + && lastRpcTime.elapsedTime().compareTo(electionTimeout) >= 0 + && !lostMajorityHeartbeatsRecently() + && server.isRunning(); + } + private void runImpl() { final TimeDuration sleepDeviationThreshold = server.getSleepDeviationThreshold(); while (shouldRun()) { @@ -149,10 +157,7 @@ private void runImpl() { break; } synchronized (server) { - if (outstandingOp.get() == 0 - && isRunning && server.getInfo().isFollower() - && lastRpcTime.elapsedTime().compareTo(electionTimeout) >= 0 - && !lostMajorityHeartbeatsRecently()) { + if (roleChangeChecking(electionTimeout)) { LOG.info("{}: change to CANDIDATE, lastRpcElapsedTime:{}, electionTimeout:{}", this, lastRpcTime.elapsedTime(), electionTimeout); server.getLeaderElectionMetrics().onLeaderElectionTimeout(); // Update timeout metric counters. diff --git a/ratis-server/src/main/java/org/apache/ratis/server/impl/LeaderElection.java b/ratis-server/src/main/java/org/apache/ratis/server/impl/LeaderElection.java index a5bfba7bec..4badd09cd1 100644 --- a/ratis-server/src/main/java/org/apache/ratis/server/impl/LeaderElection.java +++ b/ratis-server/src/main/java/org/apache/ratis/server/impl/LeaderElection.java @@ -253,10 +253,6 @@ private void runImpl() { } try (AutoCloseable ignored = Timekeeper.start(server.getLeaderElectionMetrics().getLeaderElectionTimer())) { - if (!server.isRunning()) { - LOG.info("{}: skip since the server is not running", this); - return; - } for (int round = 0; shouldRun(); round++) { if (skipPreVote || askForVotes(Phase.PRE_VOTE, round)) { if (askForVotes(Phase.ELECTION, round)) { diff --git a/ratis-server/src/main/java/org/apache/ratis/server/impl/RaftServerImpl.java b/ratis-server/src/main/java/org/apache/ratis/server/impl/RaftServerImpl.java index ae158ad75e..1ac62fd987 100644 --- a/ratis-server/src/main/java/org/apache/ratis/server/impl/RaftServerImpl.java +++ b/ratis-server/src/main/java/org/apache/ratis/server/impl/RaftServerImpl.java @@ -160,6 +160,7 @@ class RaftServerImpl implements RaftServer.Division, static final String APPEND_TRANSACTION = CLASS_NAME + ".appendTransaction"; static final String LOG_SYNC = APPEND_ENTRIES + ".logComplete"; static final String START_LEADER_ELECTION = CLASS_NAME + ".startLeaderElection"; + static final String START_COMPLETE = CLASS_NAME + ".startComplete"; class Info implements DivisionInfo { @Override @@ -400,7 +401,10 @@ boolean start() throws IOException { jmxAdapter.registerMBean(); state.start(); - startComplete.compareAndSet(false, true); + CodeInjectionForTesting.execute(START_COMPLETE, getId(), null, role); + if (startComplete.compareAndSet(false, true)) { + LOG.info("{}: Successfully started.", getMemberId()); + } return true; } diff --git a/ratis-server/src/test/java/org/apache/ratis/server/impl/LeaderElectionTests.java b/ratis-server/src/test/java/org/apache/ratis/server/impl/LeaderElectionTests.java index b29b537abe..b175ffe292 100644 --- a/ratis-server/src/test/java/org/apache/ratis/server/impl/LeaderElectionTests.java +++ b/ratis-server/src/test/java/org/apache/ratis/server/impl/LeaderElectionTests.java @@ -29,6 +29,7 @@ import org.apache.ratis.protocol.RaftGroupMemberId; import org.apache.ratis.protocol.RaftPeer; import org.apache.ratis.protocol.RaftPeerId; +import org.apache.ratis.protocol.SetConfigurationRequest; import org.apache.ratis.protocol.exceptions.LeaderSteppingDownException; import org.apache.ratis.protocol.exceptions.TransferLeadershipException; import org.apache.ratis.server.DivisionInfo; @@ -44,6 +45,8 @@ import org.apache.ratis.util.TimeDuration; import org.apache.ratis.util.Timestamp; import org.apache.ratis.util.function.CheckedBiConsumer; +import org.apache.ratis.util.CodeInjectionForTesting; +import org.junit.Assert; import org.junit.jupiter.api.Assertions; import org.junit.jupiter.api.Test; import org.slf4j.Logger; @@ -97,6 +100,80 @@ public void testBasicLeaderElection() throws Exception { cluster.shutdown(); } + static class SleepCode implements CodeInjectionForTesting.Code { + private final long sleepMs; + + SleepCode(long sleepMs) { + this.sleepMs = sleepMs; + } + + @Override + public boolean execute(Object localId, Object remoteId, Object... args) { + try { + LOG.info("{}: Simulate RaftServer startup blocking", localId); + Thread.sleep(sleepMs); + } catch (InterruptedException e) { + throw new RuntimeException(e); + } + return true; + } + } + + @Test + public void testWaitServerReady() throws Exception { + final int sleepMs = 1000 + ThreadLocalRandom.current().nextInt(1000); + LOG.info("Running testWaitServerReady, sleep = {}ms", sleepMs); + CodeInjectionForTesting.put(RaftServerImpl.START_COMPLETE, new SleepCode(sleepMs)); + final MiniRaftCluster cluster = newCluster(1); + final Timestamp startTime = Timestamp.currentTime(); + cluster.start(); + LOG.info("Cluster started at {}ms", startTime.elapsedTimeMs()); + final RaftGroupId groupId = cluster.getGroupId(); + final RaftServerImpl server = (RaftServerImpl) cluster.getServers().iterator().next().getDivision(groupId); + final boolean isRunning = server.isRunning(); + LOG.info("{} isRunning at {}ms? {}", server.getId(), startTime.elapsedTimeMs(), isRunning); + + // Leader will be elected if the server is ready + Assertions.assertNotNull(waitForLeader(cluster), "No leader is elected."); + final long elapsedMs = startTime.elapsedTimeMs(); + // allow a small difference to tolerate system timer inaccuracy + Assertions.assertTrue(elapsedMs > sleepMs - 10, () -> "elapseMs = " + elapsedMs + " but sleepMs = " + sleepMs); + cluster.shutdown(); + CodeInjectionForTesting.remove(RaftServerImpl.START_COMPLETE); + } + + @Test + public void testAddServerForWaitReady() throws IOException, InterruptedException { + LOG.info("Running testAddServerForWaitReady"); + // normal startup cluster with 3 server + final MiniRaftCluster cluster = newCluster(3); + cluster.start(); + RaftTestUtil.waitForLeader(cluster); + try (RaftClient client = cluster.createClient()) { + for (int i = 0; i < 10; ++i) { + RaftClientReply reply = client.io().send(new RaftTestUtil.SimpleMessage("message_" + i)); + Assertions.assertTrue(reply.isSuccess()); + } + // add 3 new servers and wait longer time + CodeInjectionForTesting.put(RaftServerImpl.START_COMPLETE, new SleepCode(2000)); + MiniRaftCluster.PeerChanges peerChanges = cluster.addNewPeers(2, true, false); + LOG.info("add new 3 servers"); + LOG.info(cluster.printServers()); + RaftClientReply reply = client.admin().setConfiguration(SetConfigurationRequest.Arguments.newBuilder() + .setServersInNewConf(peerChanges.newPeers) + .setMode(SetConfigurationRequest.Mode.ADD).build()); + Assert.assertTrue(reply.isSuccess()); + for (RaftServer server : cluster.getServers()) { + RaftServerProxy proxy = (RaftServerProxy) server; + proxy.getImpls().forEach(s -> { + Assertions.assertTrue(s.isRunning()); + }); + } + } + cluster.shutdown();; + CodeInjectionForTesting.remove(RaftServerImpl.START_COMPLETE); + } + @Test public void testChangeLeader() throws Exception { SegmentedRaftLogTestUtils.setRaftLogWorkerLogLevel(Level.TRACE); From 0c10d4f056496ad15795264c9262e782a92cca58 Mon Sep 17 00:00:00 2001 From: 133tosakarin <97331129+133tosakarin@users.noreply.github.com> Date: Tue, 22 Oct 2024 02:26:53 +0800 Subject: [PATCH 106/397] RATIS-2174. Move future.join outside the lock (#1168) --- .../ratis/server/impl/LeaderElection.java | 1 - .../ratis/server/impl/LeaderStateImpl.java | 2 +- .../ratis/server/impl/RaftServerImpl.java | 34 +++++++-------- .../impl/SnapshotInstallationHandler.java | 41 +++++++++++-------- 4 files changed, 44 insertions(+), 34 deletions(-) diff --git a/ratis-server/src/main/java/org/apache/ratis/server/impl/LeaderElection.java b/ratis-server/src/main/java/org/apache/ratis/server/impl/LeaderElection.java index 4badd09cd1..39b401dda1 100644 --- a/ratis-server/src/main/java/org/apache/ratis/server/impl/LeaderElection.java +++ b/ratis-server/src/main/java/org/apache/ratis/server/impl/LeaderElection.java @@ -227,7 +227,6 @@ private void startIfNew(Runnable starter) { CompletableFuture shutdown() { lifeCycle.checkStateAndClose(); - stopped.complete(null); return stopped; } diff --git a/ratis-server/src/main/java/org/apache/ratis/server/impl/LeaderStateImpl.java b/ratis-server/src/main/java/org/apache/ratis/server/impl/LeaderStateImpl.java index 3d8bc22195..0dfbf263df 100644 --- a/ratis-server/src/main/java/org/apache/ratis/server/impl/LeaderStateImpl.java +++ b/ratis-server/src/main/java/org/apache/ratis/server/impl/LeaderStateImpl.java @@ -705,7 +705,7 @@ void submitStepDownEvent(long term, StepDownReason reason) { private void stepDown(long term, StepDownReason reason) { try { lease.getAndSetEnabled(false); - server.changeToFollowerAndPersistMetadata(term, false, reason); + server.changeToFollowerAndPersistMetadata(term, false, reason).join(); pendingStepDown.complete(server::newSuccessReply); } catch(IOException e) { final String s = this + ": Failed to persist metadata for term " + term; diff --git a/ratis-server/src/main/java/org/apache/ratis/server/impl/RaftServerImpl.java b/ratis-server/src/main/java/org/apache/ratis/server/impl/RaftServerImpl.java index 1ac62fd987..ba7d8668c9 100644 --- a/ratis-server/src/main/java/org/apache/ratis/server/impl/RaftServerImpl.java +++ b/ratis-server/src/main/java/org/apache/ratis/server/impl/RaftServerImpl.java @@ -132,7 +132,6 @@ import java.util.concurrent.ThreadLocalRandom; import java.util.concurrent.TimeUnit; import java.util.concurrent.atomic.AtomicBoolean; -import java.util.concurrent.atomic.AtomicReference; import java.util.function.Function; import java.util.function.Supplier; import java.util.stream.Collectors; @@ -570,14 +569,8 @@ void setFirstElection(Object reason) { * @param force Force to start a new {@link FollowerState} even if this server is already a follower. * @return if the term/votedFor should be updated to the new term */ - private boolean changeToFollower(long newTerm, boolean force, boolean allowListener, Object reason) { - final AtomicReference metadataUpdated = new AtomicReference<>(); - changeToFollowerAsync(newTerm, force, allowListener, reason, metadataUpdated).join(); - return metadataUpdated.get(); - } - - private synchronized CompletableFuture changeToFollowerAsync( - long newTerm, boolean force, boolean allowListener, Object reason, AtomicReference metadataUpdated) { + private synchronized CompletableFuture changeToFollower( + long newTerm, boolean force, boolean allowListener, Object reason, AtomicBoolean metadataUpdated) { final RaftPeerRole old = role.getCurrentRole(); if (old == RaftPeerRole.LISTENER && !allowListener) { throw new IllegalStateException("Unexpected role " + old); @@ -612,13 +605,16 @@ private synchronized CompletableFuture changeToFollowerAsync( return future; } - synchronized void changeToFollowerAndPersistMetadata( + synchronized CompletableFuture changeToFollowerAndPersistMetadata( long newTerm, boolean allowListener, Object reason) throws IOException { - if (changeToFollower(newTerm, false, allowListener, reason)) { + final AtomicBoolean metadataUpdated = new AtomicBoolean(); + final CompletableFuture future = changeToFollower(newTerm, false, allowListener, reason, metadataUpdated); + if (metadataUpdated.get()) { state.persistMetadata(); } + return future; } synchronized void changeToLeader() { @@ -1406,6 +1402,7 @@ private RequestVoteReplyProto requestVote(Phase phase, boolean shouldShutdown = false; final RequestVoteReplyProto reply; + CompletableFuture future = null; synchronized (this) { // Check life cycle state again to avoid the PAUSING/PAUSED state. assertLifeCycleState(LifeCycle.States.RUNNING); @@ -1415,12 +1412,12 @@ private RequestVoteReplyProto requestVote(Phase phase, final boolean voteGranted = context.decideVote(candidate, candidateLastEntry); if (candidate != null && phase == Phase.ELECTION) { // change server state in the ELECTION phase - final boolean termUpdated = - changeToFollower(candidateTerm, true, false, "candidate:" + candidateId); + final AtomicBoolean termUpdated = new AtomicBoolean(); + future = changeToFollower(candidateTerm, true, false, "candidate:" + candidateId, termUpdated); if (voteGranted) { state.grantVote(candidate.getId()); } - if (termUpdated || voteGranted) { + if (termUpdated.get() || voteGranted) { state.persistMetadata(); // sync metafile } } @@ -1436,6 +1433,9 @@ private RequestVoteReplyProto requestVote(Phase phase, getMemberId(), phase, toRequestVoteReplyString(reply), state); } } + if (future != null) { + future.join(); + } return reply; } @@ -1533,6 +1533,7 @@ private CompletableFuture appendEntriesAsync(RaftPeerId final long followerCommit = state.getLog().getLastCommittedIndex(); final Optional followerState; final Timekeeper.Context timer = raftServerMetrics.getFollowerAppendEntryTimer(isHeartbeat).time(); + final CompletableFuture future; synchronized (this) { // Check life cycle state again to avoid the PAUSING/PAUSED state. assertLifeCycleState(LifeCycle.States.STARTING_OR_RUNNING); @@ -1544,7 +1545,7 @@ leaderId, getMemberId(), currentTerm, followerCommit, state.getNextIndex(), AppendResult.NOT_LEADER, callId, RaftLog.INVALID_LOG_INDEX, isHeartbeat)); } try { - changeToFollowerAndPersistMetadata(leaderTerm, true, "appendEntries"); + future = changeToFollowerAndPersistMetadata(leaderTerm, true, "appendEntries"); } catch (IOException e) { return JavaUtils.completeExceptionally(e); } @@ -1569,11 +1570,12 @@ leaderId, getMemberId(), currentTerm, followerCommit, inconsistencyReplyNextInde AppendResult.INCONSISTENCY, callId, RaftLog.INVALID_LOG_INDEX, isHeartbeat); LOG.info("{}: appendEntries* reply {}", getMemberId(), toAppendEntriesReplyString(reply)); followerState.ifPresent(fs -> fs.updateLastRpcTime(FollowerState.UpdateType.APPEND_COMPLETE)); - return CompletableFuture.completedFuture(reply); + return future.thenApply(dummy -> reply); } state.updateConfiguration(entries); } + future.join(); final List> futures = entries.isEmpty() ? Collections.emptyList() : state.getLog().append(entries); diff --git a/ratis-server/src/main/java/org/apache/ratis/server/impl/SnapshotInstallationHandler.java b/ratis-server/src/main/java/org/apache/ratis/server/impl/SnapshotInstallationHandler.java index 537b384c64..8de9a37569 100644 --- a/ratis-server/src/main/java/org/apache/ratis/server/impl/SnapshotInstallationHandler.java +++ b/ratis-server/src/main/java/org/apache/ratis/server/impl/SnapshotInstallationHandler.java @@ -46,6 +46,7 @@ import java.io.IOException; import java.util.Optional; +import java.util.concurrent.CompletableFuture; import java.util.concurrent.atomic.AtomicBoolean; import java.util.concurrent.atomic.AtomicInteger; import java.util.concurrent.atomic.AtomicLong; @@ -122,12 +123,12 @@ private InstallSnapshotReplyProto installSnapshotImpl(InstallSnapshotRequestProt if (installSnapshotEnabled) { // Leader has sent InstallSnapshot request with SnapshotInfo. Install the snapshot. if (request.hasSnapshotChunk()) { - reply = checkAndInstallSnapshot(request, leaderId); + reply = checkAndInstallSnapshot(request, leaderId).join(); } } else { // Leader has only sent a notification to install snapshot. Inform State Machine to install snapshot. if (request.hasNotification()) { - reply = notifyStateMachineToInstallSnapshot(request, leaderId); + reply = notifyStateMachineToInstallSnapshot(request, leaderId).join(); } } @@ -156,21 +157,22 @@ private InstallSnapshotReplyProto installSnapshotImpl(InstallSnapshotRequestProt return failedReply; } - private InstallSnapshotReplyProto checkAndInstallSnapshot(InstallSnapshotRequestProto request, + private CompletableFuture checkAndInstallSnapshot(InstallSnapshotRequestProto request, RaftPeerId leaderId) throws IOException { final long currentTerm; final long leaderTerm = request.getLeaderTerm(); final InstallSnapshotRequestProto.SnapshotChunkProto snapshotChunkRequest = request.getSnapshotChunk(); final TermIndex lastIncluded = TermIndex.valueOf(snapshotChunkRequest.getTermIndex()); final long lastIncludedIndex = lastIncluded.getIndex(); + final CompletableFuture future; synchronized (server) { final boolean recognized = state.recognizeLeader(RaftServerProtocol.Op.INSTALL_SNAPSHOT, leaderId, leaderTerm); currentTerm = state.getCurrentTerm(); if (!recognized) { - return toInstallSnapshotReplyProto(leaderId, getMemberId(), - currentTerm, snapshotChunkRequest.getRequestIndex(), InstallSnapshotResult.NOT_LEADER); + return CompletableFuture.completedFuture(toInstallSnapshotReplyProto(leaderId, getMemberId(), + currentTerm, snapshotChunkRequest.getRequestIndex(), InstallSnapshotResult.NOT_LEADER)); } - server.changeToFollowerAndPersistMetadata(leaderTerm, true, "installSnapshot"); + future = server.changeToFollowerAndPersistMetadata(leaderTerm, true, "installSnapshot"); state.setLeader(leaderId, "installSnapshot"); server.updateLastRpcTime(FollowerState.UpdateType.INSTALL_SNAPSHOT_START); @@ -186,8 +188,9 @@ private InstallSnapshotReplyProto checkAndInstallSnapshot(InstallSnapshotRequest // have a lot of requests if (state.getLog().getLastCommittedIndex() >= lastIncludedIndex) { nextChunkIndex.set(snapshotChunkRequest.getRequestIndex() + 1); - return toInstallSnapshotReplyProto(leaderId, getMemberId(), + final InstallSnapshotReplyProto reply = toInstallSnapshotReplyProto(leaderId, getMemberId(), currentTerm, snapshotChunkRequest.getRequestIndex(), InstallSnapshotResult.ALREADY_INSTALLED); + return future.thenApply(dummy -> reply); } //TODO: We should only update State with installed snapshot once the request is done. @@ -210,25 +213,27 @@ private InstallSnapshotReplyProto checkAndInstallSnapshot(InstallSnapshotRequest if (snapshotChunkRequest.getDone()) { LOG.info("{}: successfully install the entire snapshot-{}", getMemberId(), lastIncludedIndex); } - return toInstallSnapshotReplyProto(leaderId, getMemberId(), + final InstallSnapshotReplyProto reply = toInstallSnapshotReplyProto(leaderId, getMemberId(), currentTerm, snapshotChunkRequest.getRequestIndex(), InstallSnapshotResult.SUCCESS); + return future.thenApply(dummy -> reply); } - private InstallSnapshotReplyProto notifyStateMachineToInstallSnapshot( + private CompletableFuture notifyStateMachineToInstallSnapshot( InstallSnapshotRequestProto request, RaftPeerId leaderId) throws IOException { final long currentTerm; final long leaderTerm = request.getLeaderTerm(); final TermIndex firstAvailableLogTermIndex = TermIndex.valueOf( request.getNotification().getFirstAvailableTermIndex()); final long firstAvailableLogIndex = firstAvailableLogTermIndex.getIndex(); + final CompletableFuture future; synchronized (server) { final boolean recognized = state.recognizeLeader("notifyInstallSnapshot", leaderId, leaderTerm); currentTerm = state.getCurrentTerm(); if (!recognized) { - return toInstallSnapshotReplyProto(leaderId, getMemberId(), - currentTerm, InstallSnapshotResult.NOT_LEADER); + return CompletableFuture.completedFuture(toInstallSnapshotReplyProto(leaderId, getMemberId(), + currentTerm, InstallSnapshotResult.NOT_LEADER)); } - server.changeToFollowerAndPersistMetadata(leaderTerm, true, "installSnapshot"); + future = server.changeToFollowerAndPersistMetadata(leaderTerm, true, "installSnapshot"); state.setLeader(leaderId, "installSnapshot"); server.updateLastRpcTime(FollowerState.UpdateType.INSTALL_SNAPSHOT_NOTIFICATION); @@ -245,8 +250,9 @@ private InstallSnapshotReplyProto notifyStateMachineToInstallSnapshot( inProgressInstallSnapshotIndex.compareAndSet(firstAvailableLogIndex, INVALID_LOG_INDEX); LOG.info("{}: InstallSnapshot notification result: {}, current snapshot index: {}", getMemberId(), InstallSnapshotResult.ALREADY_INSTALLED, snapshotIndex); - return toInstallSnapshotReplyProto(leaderId, getMemberId(), currentTerm, + final InstallSnapshotReplyProto reply = toInstallSnapshotReplyProto(leaderId, getMemberId(), currentTerm, InstallSnapshotResult.ALREADY_INSTALLED, snapshotIndex); + return future.thenApply(dummy -> reply); } final RaftPeerProto leaderProto; @@ -323,8 +329,9 @@ private InstallSnapshotReplyProto notifyStateMachineToInstallSnapshot( inProgressInstallSnapshotIndex.set(INVALID_LOG_INDEX); server.getStateMachine().event().notifySnapshotInstalled( InstallSnapshotResult.SNAPSHOT_UNAVAILABLE, INVALID_LOG_INDEX, server.getPeer()); - return toInstallSnapshotReplyProto(leaderId, getMemberId(), + final InstallSnapshotReplyProto reply = toInstallSnapshotReplyProto(leaderId, getMemberId(), currentTerm, InstallSnapshotResult.SNAPSHOT_UNAVAILABLE); + return future.thenApply(dummy -> reply); } // If a snapshot has been installed, return SNAPSHOT_INSTALLED with the installed snapshot index and reset @@ -341,8 +348,9 @@ private InstallSnapshotReplyProto notifyStateMachineToInstallSnapshot( server.getStateMachine().event().notifySnapshotInstalled( InstallSnapshotResult.SNAPSHOT_INSTALLED, latestInstalledIndex, server.getPeer()); installedIndex.set(latestInstalledIndex); - return toInstallSnapshotReplyProto(leaderId, getMemberId(), + final InstallSnapshotReplyProto reply = toInstallSnapshotReplyProto(leaderId, getMemberId(), currentTerm, InstallSnapshotResult.SNAPSHOT_INSTALLED, latestInstalledSnapshotTermIndex.getIndex()); + return future.thenApply(dummy -> reply); } // Otherwise, Snapshot installation is in progress. @@ -350,8 +358,9 @@ private InstallSnapshotReplyProto notifyStateMachineToInstallSnapshot( LOG.debug("{}: InstallSnapshot notification result: {}", getMemberId(), InstallSnapshotResult.IN_PROGRESS); } - return toInstallSnapshotReplyProto(leaderId, getMemberId(), + final InstallSnapshotReplyProto reply = toInstallSnapshotReplyProto(leaderId, getMemberId(), currentTerm, InstallSnapshotResult.IN_PROGRESS); + return future.thenApply(dummy -> reply); } } From f0143b5307120d3496a871f01d8471456dd27153 Mon Sep 17 00:00:00 2001 From: Nicholas Jiang Date: Fri, 25 Oct 2024 15:31:27 +0800 Subject: [PATCH 107/397] RATIS-2179. Fix NPE in RaftServerImpl.getLogInfo. (#1171) --- .../ratis/server/impl/RaftServerImpl.java | 17 +++++++++++++---- 1 file changed, 13 insertions(+), 4 deletions(-) diff --git a/ratis-server/src/main/java/org/apache/ratis/server/impl/RaftServerImpl.java b/ratis-server/src/main/java/org/apache/ratis/server/impl/RaftServerImpl.java index ba7d8668c9..3ffcee0797 100644 --- a/ratis-server/src/main/java/org/apache/ratis/server/impl/RaftServerImpl.java +++ b/ratis-server/src/main/java/org/apache/ratis/server/impl/RaftServerImpl.java @@ -664,10 +664,19 @@ GroupInfoReply getGroupInfo(GroupInfoRequest request) { LogInfoProto getLogInfo(){ final RaftLog log = getRaftLog(); - LogInfoProto.Builder logInfoBuilder = LogInfoProto.newBuilder() - .setApplied(getStateMachine().getLastAppliedTermIndex().toProto()) - .setCommitted(log.getTermIndex(log.getLastCommittedIndex()).toProto()) - .setLastEntry(log.getLastEntryTermIndex().toProto()); + LogInfoProto.Builder logInfoBuilder = LogInfoProto.newBuilder(); + final TermIndex applied = getStateMachine().getLastAppliedTermIndex(); + if (applied != null) { + logInfoBuilder.setApplied(applied.toProto()); + } + final TermIndex committed = log.getTermIndex(log.getLastCommittedIndex()); + if (committed != null) { + logInfoBuilder.setCommitted(committed.toProto()); + } + final TermIndex entry = log.getLastEntryTermIndex(); + if (entry != null) { + logInfoBuilder.setLastEntry(entry.toProto()); + } final SnapshotInfo snapshot = getStateMachine().getLatestSnapshot(); if (snapshot != null) { logInfoBuilder.setLastSnapshot(snapshot.getTermIndex().toProto()); From 824a76540ccaa11f63d11ce0cdd7c90035cb4c33 Mon Sep 17 00:00:00 2001 From: Tsz-Wo Nicholas Sze Date: Sun, 27 Oct 2024 20:56:18 -0700 Subject: [PATCH 108/397] RATIS-2178. Add a log message whenever deleting a RaftLog segment. (#1170) --- .../java/org/apache/ratis/util/FileUtils.java | 6 +- .../server/raftlog/segmented/LogSegment.java | 7 ++- .../raftlog/segmented/LogSegmentStartEnd.java | 39 ++++++++----- .../raftlog/segmented/SegmentedRaftLog.java | 3 +- .../segmented/SegmentedRaftLogCache.java | 25 +++++--- .../SegmentedRaftLogInputStream.java | 57 +++++-------------- .../segmented/SegmentedRaftLogWorker.java | 23 ++++---- .../segmented/SegmentedRaftLogTestUtils.java | 14 ++++- .../ratis/server/ServerRestartTests.java | 14 +---- .../raftlog/segmented/TestLogSegment.java | 12 ++-- .../segmented/TestRaftLogReadWrite.java | 12 ++-- 11 files changed, 108 insertions(+), 104 deletions(-) diff --git a/ratis-common/src/main/java/org/apache/ratis/util/FileUtils.java b/ratis-common/src/main/java/org/apache/ratis/util/FileUtils.java index d5141e9171..63bf3a4e5a 100644 --- a/ratis-common/src/main/java/org/apache/ratis/util/FileUtils.java +++ b/ratis-common/src/main/java/org/apache/ratis/util/FileUtils.java @@ -201,8 +201,10 @@ static File move(File src, String suffix) throws IOException { } /** The same as passing f.toPath() to {@link #delete(Path)}. */ - static void deleteFile(File f) throws IOException { - delete(f.toPath()); + static Path deleteFile(File f) throws IOException { + final Path path = f.toPath(); + delete(path); + return path; } /** diff --git a/ratis-server/src/main/java/org/apache/ratis/server/raftlog/segmented/LogSegment.java b/ratis-server/src/main/java/org/apache/ratis/server/raftlog/segmented/LogSegment.java index 89a6e20501..1b55b97275 100644 --- a/ratis-server/src/main/java/org/apache/ratis/server/raftlog/segmented/LogSegment.java +++ b/ratis-server/src/main/java/org/apache/ratis/server/raftlog/segmented/LogSegment.java @@ -37,6 +37,7 @@ import java.io.File; import java.io.IOException; import java.util.ArrayList; +import java.nio.file.Path; import java.util.Comparator; import java.util.List; import java.util.Map; @@ -127,8 +128,7 @@ public static int readSegmentFile(File file, LogSegmentStartEnd startEnd, SizeIn CorruptionPolicy corruptionPolicy, SegmentedRaftLogMetrics raftLogMetrics, Consumer entryConsumer) throws IOException { int count = 0; - try (SegmentedRaftLogInputStream in = new SegmentedRaftLogInputStream( - file, startEnd.getStartIndex(), startEnd.getEndIndex(), startEnd.isOpen(), maxOpSize, raftLogMetrics)) { + try(SegmentedRaftLogInputStream in = new SegmentedRaftLogInputStream(file, startEnd, maxOpSize, raftLogMetrics)) { for(LogEntryProto prev = null, next; (next = in.nextEntry()) != null; prev = next) { if (prev != null) { Preconditions.assertTrue(next.getIndex() == prev.getIndex() + 1, @@ -181,7 +181,8 @@ static LogSegment loadSegment(RaftStorage storage, File file, LogSegmentStartEnd if (entryCount == 0) { // The segment does not have any entries, delete the file. - FileUtils.deleteFile(file); + final Path deleted = FileUtils.deleteFile(file); + LOG.info("Deleted RaftLog segment since entry count is zero: startEnd={}, path={}", startEnd, deleted); return null; } else if (file.length() > segment.getTotalFileSize()) { // The segment has extra padding, truncate it. diff --git a/ratis-server/src/main/java/org/apache/ratis/server/raftlog/segmented/LogSegmentStartEnd.java b/ratis-server/src/main/java/org/apache/ratis/server/raftlog/segmented/LogSegmentStartEnd.java index 6fbaeab62a..4f0734ca9e 100644 --- a/ratis-server/src/main/java/org/apache/ratis/server/raftlog/segmented/LogSegmentStartEnd.java +++ b/ratis-server/src/main/java/org/apache/ratis/server/raftlog/segmented/LogSegmentStartEnd.java @@ -22,14 +22,12 @@ import org.apache.ratis.util.Preconditions; import java.io.File; -import java.util.Comparator; import java.util.Objects; -import java.util.Optional; import java.util.regex.Pattern; /** * The start index and an end index of a log segment. - * + *

* This is a value-based class. */ public final class LogSegmentStartEnd implements Comparable { @@ -76,21 +74,24 @@ static LogSegmentStartEnd valueOf(long startIndex, long endIndex, boolean isOpen private final Long endIndex; private LogSegmentStartEnd(long startIndex, Long endIndex) { - Preconditions.assertTrue(startIndex >= RaftLog.LEAST_VALID_LOG_INDEX); - Preconditions.assertTrue(endIndex == null || endIndex >= startIndex); this.startIndex = startIndex; this.endIndex = endIndex; + + Preconditions.assertTrue(startIndex >= RaftLog.LEAST_VALID_LOG_INDEX, this); + if (endIndex != null) { + Preconditions.assertTrue(endIndex >= startIndex, this); + } } - public long getStartIndex() { + long getStartIndex() { return startIndex; } - public long getEndIndex() { - return Optional.ofNullable(endIndex).orElse(RaftLog.INVALID_LOG_INDEX); + long getEndIndex() { + return Objects.requireNonNull(endIndex, "endIndex"); } - public boolean isOpen() { + boolean isOpen() { return endIndex == null; } @@ -108,9 +109,21 @@ File getFile(RaftStorage storage) { @Override public int compareTo(LogSegmentStartEnd that) { - return Comparator.comparingLong(LogSegmentStartEnd::getStartIndex) - .thenComparingLong(LogSegmentStartEnd::getEndIndex) - .compare(this, that); + if (this == that) { + return 0; + } + // startIndex always non-null + final int diff = Long.compare(this.getStartIndex(), that.getStartIndex()); + if (diff != 0) { + return diff; + } + + // same startIndex, compare endIndex + if (this.isOpen()) { + return that.isOpen()? 0 : -1; //open first + } else { + return that.isOpen() ? 1 : Long.compare(this.endIndex, that.endIndex); + } } @Override @@ -131,6 +144,6 @@ public int hashCode() { @Override public String toString() { - return startIndex + "-" + Optional.ofNullable(endIndex).map(Object::toString).orElse(""); + return startIndex + "-" + (endIndex != null? endIndex : ""); } } \ No newline at end of file diff --git a/ratis-server/src/main/java/org/apache/ratis/server/raftlog/segmented/SegmentedRaftLog.java b/ratis-server/src/main/java/org/apache/ratis/server/raftlog/segmented/SegmentedRaftLog.java index f49900f16a..6dc3d7961c 100644 --- a/ratis-server/src/main/java/org/apache/ratis/server/raftlog/segmented/SegmentedRaftLog.java +++ b/ratis-server/src/main/java/org/apache/ratis/server/raftlog/segmented/SegmentedRaftLog.java @@ -378,12 +378,13 @@ protected CompletableFuture purgeImpl(long index) { try (AutoCloseableLock writeLock = writeLock()) { SegmentedRaftLogCache.TruncationSegments ts = cache.purge(index); updateSnapshotIndexFromStateMachine(); - LOG.debug("purging segments:{}", ts); if (ts != null) { + LOG.info("{}: {}", getName(), ts); Task task = fileLogWorker.purge(ts); return task.getFuture(); } } + LOG.debug("{}: purge({}) found nothing to purge.", getName(), index); return CompletableFuture.completedFuture(index); } diff --git a/ratis-server/src/main/java/org/apache/ratis/server/raftlog/segmented/SegmentedRaftLogCache.java b/ratis-server/src/main/java/org/apache/ratis/server/raftlog/segmented/SegmentedRaftLogCache.java index 58c70c4af9..20b7a5c372 100644 --- a/ratis-server/src/main/java/org/apache/ratis/server/raftlog/segmented/SegmentedRaftLogCache.java +++ b/ratis-server/src/main/java/org/apache/ratis/server/raftlog/segmented/SegmentedRaftLogCache.java @@ -122,9 +122,14 @@ public String toString() { } static class TruncationSegments { + private final String reason; private final SegmentFileInfo toTruncate; // name of the file to be truncated private final SegmentFileInfo[] toDelete; // names of the files to be deleted + String getReason() { + return reason; + } + public SegmentFileInfo getToTruncate() { return toTruncate; } @@ -133,10 +138,10 @@ public SegmentFileInfo[] getToDelete() { return toDelete; } - TruncationSegments(SegmentFileInfo toTruncate, - List toDelete) { - this.toDelete = SegmentFileInfo.toSortedArray(toDelete); + TruncationSegments(String reason, SegmentFileInfo toTruncate, List toDelete) { + this.reason = reason; this.toTruncate = toTruncate; + this.toDelete = SegmentFileInfo.toSortedArray(toDelete); } long maxEndIndex() { @@ -152,8 +157,9 @@ long maxEndIndex() { @Override public String toString() { - return "toTruncate: " + toTruncate - + "\n toDelete: " + Arrays.toString(toDelete); + return reason + + "\n toTruncate: " + toTruncate + + "\n toDelete: " + Arrays.toString(toDelete); } } @@ -297,6 +303,7 @@ void clear() { } TruncationSegments truncate(long index, LogSegment openSegment, Runnable clearOpenSegment) { + final String reason = "truncate(" + index + ")"; try(AutoCloseableLock writeLock = writeLock()) { final int segmentIndex = binarySearch(index); if (segmentIndex == -segments.size() - 1) { @@ -305,7 +312,7 @@ TruncationSegments truncate(long index, LogSegment openSegment, Runnable clearOp if (index == openSegment.getStartIndex()) { // the open segment should be deleted final SegmentFileInfo deleted = deleteOpenSegment(openSegment, clearOpenSegment); - return new TruncationSegments(null, Collections.singletonList(deleted)); + return new TruncationSegments(reason, null, Collections.singletonList(deleted)); } else { openSegment.truncate(index); Preconditions.assertTrue(!openSegment.isOpen(), @@ -315,7 +322,7 @@ TruncationSegments truncate(long index, LogSegment openSegment, Runnable clearOp segments.add(openSegment); sizeInBytes += openSegment.getTotalFileSize(); clearOpenSegment.run(); - return new TruncationSegments(info, Collections.emptyList()); + return new TruncationSegments(reason, info, Collections.emptyList()); } } } else if (segmentIndex >= 0) { @@ -340,7 +347,7 @@ TruncationSegments truncate(long index, LogSegment openSegment, Runnable clearOp } SegmentFileInfo t = ts.numOfEntries() == 0? null: new SegmentFileInfo(ts.getStartIndex(), oldEnd, false, ts.getTotalFileSize(), ts.getEndIndex()); - return new TruncationSegments(t, list); + return new TruncationSegments(reason, t, list); } return null; } @@ -373,7 +380,7 @@ TruncationSegments purge(long index) { throw new IllegalStateException("Unexpected gap in segments: binarySearch(" + index + ") returns " + segmentIndex + ", segments=" + segments); } - return list.isEmpty() ? null : new TruncationSegments(null, list); + return list.isEmpty() ? null : new TruncationSegments("purge(" + index + ")", null, list); } } diff --git a/ratis-server/src/main/java/org/apache/ratis/server/raftlog/segmented/SegmentedRaftLogInputStream.java b/ratis-server/src/main/java/org/apache/ratis/server/raftlog/segmented/SegmentedRaftLogInputStream.java index 481f837f5b..050c472dd2 100644 --- a/ratis-server/src/main/java/org/apache/ratis/server/raftlog/segmented/SegmentedRaftLogInputStream.java +++ b/ratis-server/src/main/java/org/apache/ratis/server/raftlog/segmented/SegmentedRaftLogInputStream.java @@ -33,6 +33,7 @@ import org.slf4j.LoggerFactory; import static org.apache.ratis.server.raftlog.RaftLog.INVALID_LOG_INDEX; +import static org.apache.ratis.server.raftlog.RaftLog.LEAST_VALID_LOG_INDEX; public class SegmentedRaftLogInputStream implements Closeable { static final Logger LOG = LoggerFactory.getLogger(SegmentedRaftLogInputStream.class); @@ -62,27 +63,17 @@ boolean hasCorruptHeader() { } private final File logFile; - private final long startIndex; - private final long endIndex; - private final boolean isOpen; + private final LogSegmentStartEnd startEnd; private final OpenCloseState state; private SegmentedRaftLogReader reader; private final SizeInBytes maxOpSize; private final SegmentedRaftLogMetrics raftLogMetrics; - SegmentedRaftLogInputStream(File log, long startIndex, long endIndex, boolean isOpen, + SegmentedRaftLogInputStream(File log, LogSegmentStartEnd startEnd, SizeInBytes maxOpSize, SegmentedRaftLogMetrics raftLogMetrics) { this.maxOpSize = maxOpSize; - if (isOpen) { - Preconditions.assertTrue(endIndex == INVALID_LOG_INDEX); - } else { - Preconditions.assertTrue(endIndex >= startIndex); - } - this.logFile = log; - this.startIndex = startIndex; - this.endIndex = endIndex; - this.isOpen = isOpen; + this.startEnd = startEnd; this.state = new OpenCloseState(getName()); this.raftLogMetrics = raftLogMetrics; } @@ -104,14 +95,6 @@ private void init() throws IOException { } } - long getStartIndex() { - return startIndex; - } - - long getEndIndex() { - return endIndex; - } - String getName() { return logFile.getName(); } @@ -131,7 +114,7 @@ public LogEntryProto nextEntry() throws IOException { final LogEntryProto entry = reader.readEntry(); if (entry != null) { long index = entry.getIndex(); - if (!isOpen() && index >= endIndex) { + if (!startEnd.isOpen() && index >= startEnd.getEndIndex()) { /* * The end index may be derived from the segment recovery * process. It is possible that we still have some uncleaned garbage @@ -139,8 +122,8 @@ public LogEntryProto nextEntry() throws IOException { */ long skipAmt = logFile.length() - reader.getPos(); if (skipAmt > 0) { - LOG.debug("skipping {} bytes at the end of log '{}': reached" + - " entry {} out of {}", skipAmt, getName(), index, endIndex); + LOG.info("Skipping {} bytes at the end of log '{}': reached entry {} out of [{}]", + skipAmt, getName(), index, startEnd); reader.skipFully(skipAmt); } } @@ -172,10 +155,6 @@ public void close() throws IOException { } } - boolean isOpen() { - return isOpen; - } - @Override public String toString() { return getName(); @@ -188,24 +167,18 @@ public String toString() { * ID. The file portion beyond this ID is * potentially being updated. * @return Result of the validation - * @throws IOException */ static LogValidation scanEditLog(File file, long maxTxIdToScan, SizeInBytes maxOpSize) throws IOException { - SegmentedRaftLogInputStream in; - try { - in = new SegmentedRaftLogInputStream(file, INVALID_LOG_INDEX, INVALID_LOG_INDEX, false, maxOpSize, null); - // read the header, initialize the inputstream - in.init(); - } catch (EOFException e) { - LOG.warn("Log file " + file + " has no valid header", e); - return new LogValidation(0, INVALID_LOG_INDEX, true); - } - - try { + final LogSegmentStartEnd startEnd = LogSegmentStartEnd.valueOf(LEAST_VALID_LOG_INDEX); + try(SegmentedRaftLogInputStream in = new SegmentedRaftLogInputStream(file, startEnd, maxOpSize, null)) { + try { + in.init(); + } catch (EOFException e) { + LOG.warn("Invalid header for RaftLog segment {}", file, e); + return new LogValidation(0, INVALID_LOG_INDEX, true); + } return scanEditLog(in, maxTxIdToScan); - } finally { - IOUtils.cleanup(LOG, in); } } diff --git a/ratis-server/src/main/java/org/apache/ratis/server/raftlog/segmented/SegmentedRaftLogWorker.java b/ratis-server/src/main/java/org/apache/ratis/server/raftlog/segmented/SegmentedRaftLogWorker.java index 68266b4175..a8482e71bd 100644 --- a/ratis-server/src/main/java/org/apache/ratis/server/raftlog/segmented/SegmentedRaftLogWorker.java +++ b/ratis-server/src/main/java/org/apache/ratis/server/raftlog/segmented/SegmentedRaftLogWorker.java @@ -46,6 +46,7 @@ import java.io.File; import java.io.IOException; import java.nio.ByteBuffer; +import java.nio.file.Path; import java.util.LinkedList; import java.util.Objects; import java.util.Optional; @@ -476,7 +477,8 @@ void execute() throws IOException { if (segments.getToDelete() != null) { try(UncheckedAutoCloseable ignored = raftLogMetrics.startPurgeTimer()) { for (SegmentFileInfo fileInfo : segments.getToDelete()) { - FileUtils.deleteFile(fileInfo.getFile(storage)); + final Path deleted = FileUtils.deleteFile(fileInfo.getFile(storage)); + LOG.info("{}: Purged RaftLog segment: info={}, path={}", name, fileInfo, deleted); } } } @@ -568,8 +570,8 @@ public String toString() { } } - File getFile(long startIndex, Long endIndex) { - return LogSegmentStartEnd.valueOf(startIndex, endIndex).getFile(storage); + private File getFile(LogSegmentStartEnd startEnd) { + return startEnd.getFile(storage); } private class FinalizeLogSegment extends Task { @@ -586,19 +588,20 @@ private class FinalizeLogSegment extends Task { public void execute() throws IOException { freeSegmentedRaftLogOutputStream(); - final File openFile = getFile(startIndex, null); + final LogSegmentStartEnd openStartEnd = LogSegmentStartEnd.valueOf(startIndex); + final File openFile = getFile(openStartEnd); Preconditions.assertTrue(openFile.exists(), () -> name + ": File " + openFile + " to be rolled does not exist"); if (endIndex - startIndex + 1 > 0) { // finalize the current open segment - final File dstFile = getFile(startIndex, endIndex); + final File dstFile = getFile(LogSegmentStartEnd.valueOf(startIndex, endIndex)); Preconditions.assertTrue(!dstFile.exists()); FileUtils.move(openFile, dstFile); LOG.info("{}: Rolled log segment from {} to {}", name, openFile, dstFile); } else { // delete the file of the empty segment - FileUtils.deleteFile(openFile); - LOG.info("{}: Deleted empty log segment {}", name, openFile); + final Path deleted = FileUtils.deleteFile(openFile); + LOG.info("{}: Deleted empty RaftLog segment: startEnd={}, path={}", name, openStartEnd, deleted); } updateFlushedIndexIncreasingly(); safeCacheEvictIndex.updateToMax(endIndex, traceIndexChange); @@ -631,7 +634,7 @@ private class StartLogSegment extends Task { @Override void execute() throws IOException { - final File openFile = getFile(newStartIndex, null); + final File openFile = getFile(LogSegmentStartEnd.valueOf(newStartIndex)); Preconditions.assertTrue(!openFile.exists(), "open file %s exists for %s", openFile, name); Preconditions.assertTrue(pendingFlushNum == 0); @@ -673,8 +676,8 @@ void execute() throws IOException { final File delFile = del.getFile(storage); Preconditions.assertTrue(delFile.exists(), "File %s to be deleted does not exist", delFile); - FileUtils.deleteFile(delFile); - LOG.info("{}: Deleted log file {}", name, delFile); + final Path deleted = FileUtils.deleteFile(delFile); + LOG.info("{}: Deleted RaftLog segment for {}: path={}", name, segments.getReason(), deleted); minStart = Math.min(minStart, del.getStartIndex()); } if (segments.getToTruncate() == null) { diff --git a/ratis-server/src/test/java/org/apache/ratis/server/raftlog/segmented/SegmentedRaftLogTestUtils.java b/ratis-server/src/test/java/org/apache/ratis/server/raftlog/segmented/SegmentedRaftLogTestUtils.java index e242eddf59..473aa0a46c 100644 --- a/ratis-server/src/test/java/org/apache/ratis/server/raftlog/segmented/SegmentedRaftLogTestUtils.java +++ b/ratis-server/src/test/java/org/apache/ratis/server/raftlog/segmented/SegmentedRaftLogTestUtils.java @@ -17,25 +17,33 @@ */ package org.apache.ratis.server.raftlog.segmented; +import org.apache.ratis.server.RaftServer; import org.apache.ratis.util.SizeInBytes; import org.apache.ratis.util.Slf4jUtils; import org.slf4j.event.Level; import java.io.File; +import java.nio.file.Path; +import java.util.List; +import java.util.stream.Collectors; public interface SegmentedRaftLogTestUtils { SizeInBytes MAX_OP_SIZE = SizeInBytes.valueOf("32MB"); static SegmentedRaftLogInputStream newSegmentedRaftLogInputStream(File log, long startIndex, long endIndex, boolean isOpen) { - return new SegmentedRaftLogInputStream(log, startIndex, endIndex, isOpen, MAX_OP_SIZE, null); + final LogSegmentStartEnd startEnd = LogSegmentStartEnd.valueOf(startIndex, endIndex, isOpen); + return new SegmentedRaftLogInputStream(log, startEnd, MAX_OP_SIZE, null); } static void setRaftLogWorkerLogLevel(Level level) { Slf4jUtils.setLogLevel(SegmentedRaftLogWorker.LOG, level); } - static String getLogFlushTimeMetric(String memberId) { - return SegmentedRaftLogWorker.class.getName() + "." + memberId + ".flush-time"; + static List getOpenLogFiles(RaftServer.Division server) throws Exception { + return LogSegmentPath.getLogSegmentPaths(server.getRaftStorage()).stream() + .filter(p -> p.getStartEnd().isOpen()) + .map(LogSegmentPath::getPath) + .collect(Collectors.toList()); } } diff --git a/ratis-test/src/test/java/org/apache/ratis/server/ServerRestartTests.java b/ratis-test/src/test/java/org/apache/ratis/server/ServerRestartTests.java index ee93b68855..c9495fdf79 100644 --- a/ratis-test/src/test/java/org/apache/ratis/server/ServerRestartTests.java +++ b/ratis-test/src/test/java/org/apache/ratis/server/ServerRestartTests.java @@ -35,8 +35,8 @@ import org.apache.ratis.server.raftlog.RaftLogIOException; import org.apache.ratis.server.raftlog.segmented.SegmentedRaftLogFormat; import org.apache.ratis.server.RaftServerConfigKeys.Log; +import org.apache.ratis.server.raftlog.segmented.SegmentedRaftLogTestUtils; import org.apache.ratis.server.raftlog.segmented.TestSegmentedRaftLog; -import org.apache.ratis.server.raftlog.segmented.LogSegmentPath; import org.apache.ratis.statemachine.impl.SimpleStateMachine4Testing; import org.apache.ratis.statemachine.StateMachine; import org.apache.ratis.util.FileUtils; @@ -63,7 +63,6 @@ import java.util.concurrent.TimeUnit; import java.util.concurrent.atomic.AtomicInteger; import java.util.function.Supplier; -import java.util.stream.Collectors; /** * Test restarting raft peers. @@ -176,15 +175,8 @@ static void assertTruncatedLog(RaftPeerId id, File openLogFile, long lastIndex, server.getRaftServer().close(); } - static List getOpenLogFiles(RaftServer.Division server) throws Exception { - return LogSegmentPath.getLogSegmentPaths(server.getRaftStorage()).stream() - .filter(p -> p.getStartEnd().isOpen()) - .map(LogSegmentPath::getPath) - .collect(Collectors.toList()); - } - static File getOpenLogFile(RaftServer.Division server) throws Exception { - final List openLogs = getOpenLogFiles(server); + final List openLogs = SegmentedRaftLogTestUtils.getOpenLogFiles(server); Assertions.assertEquals(1, openLogs.size()); return openLogs.get(0).toFile(); } @@ -211,7 +203,7 @@ void runTestRestartWithCorruptedLogHeader(MiniRaftCluster cluster) throws Except 10, HUNDRED_MILLIS, impl.getId() + "-getOpenLogFile", LOG); for(int i = 0; i < SegmentedRaftLogFormat.getHeaderLength(); i++) { assertCorruptedLogHeader(impl.getId(), openLogFile, i, cluster, LOG); - Assertions.assertTrue(getOpenLogFiles(impl).isEmpty()); + Assertions.assertTrue(SegmentedRaftLogTestUtils.getOpenLogFiles(impl).isEmpty()); } } } diff --git a/ratis-test/src/test/java/org/apache/ratis/server/raftlog/segmented/TestLogSegment.java b/ratis-test/src/test/java/org/apache/ratis/server/raftlog/segmented/TestLogSegment.java index 50f9d2382b..a95c683c88 100644 --- a/ratis-test/src/test/java/org/apache/ratis/server/raftlog/segmented/TestLogSegment.java +++ b/ratis-test/src/test/java/org/apache/ratis/server/raftlog/segmented/TestLogSegment.java @@ -58,6 +58,8 @@ * Test basic functionality of {@link LogSegment} */ public class TestLogSegment extends BaseTest { + public static final LogSegmentStartEnd ZERO_START_NULL_END = LogSegmentStartEnd.valueOf(0); + private File storageDir; private long segmentMaxSize; private long preallocatedSize; @@ -173,7 +175,7 @@ private void testLoadSegment(boolean loadInitial, boolean isLastEntryPartiallyWr final File openSegmentFile = prepareLog(true, 0, 100, 0, isLastEntryPartiallyWritten); RaftStorage storage = RaftStorageTestUtils.newRaftStorage(storageDir); final LogSegment openSegment = LogSegment.loadSegment(storage, openSegmentFile, - LogSegmentStartEnd.valueOf(0), MAX_OP_SIZE, loadInitial, null, null); + ZERO_START_NULL_END, MAX_OP_SIZE, loadInitial, null, null); final int delta = isLastEntryPartiallyWritten? 1: 0; checkLogSegment(openSegment, 0, 99 - delta, true, openSegmentFile.length(), 0); storage.close(); @@ -218,7 +220,7 @@ public void testAppendEntryMetric() throws Exception { final File openSegmentFile = prepareLog(true, 0, 100, 0, true); RaftStorage storage = RaftStorageTestUtils.newRaftStorage(storageDir); final LogSegment openSegment = LogSegment.loadSegment(storage, openSegmentFile, - LogSegmentStartEnd.valueOf(0), MAX_OP_SIZE, true, null, raftLogMetrics); + ZERO_START_NULL_END, MAX_OP_SIZE, true, null, raftLogMetrics); checkLogSegment(openSegment, 0, 98, true, openSegmentFile.length(), 0); storage.close(); @@ -287,7 +289,7 @@ public void testTruncate() throws Exception { @Test public void testPreallocateSegment() throws Exception { RaftStorage storage = RaftStorageTestUtils.newRaftStorage(storageDir); - final File file = LogSegmentStartEnd.valueOf(0).getFile(storage); + final File file = ZERO_START_NULL_END.getFile(storage); final int[] maxSizes = new int[]{1024, 1025, 1024 * 1024 - 1, 1024 * 1024, 1024 * 1024 + 1, 2 * 1024 * 1024 - 1, 2 * 1024 * 1024, 2 * 1024 * 1024 + 1, 8 * 1024 * 1024}; @@ -338,7 +340,7 @@ public void testPreallocateSegment() throws Exception { public void testPreallocationAndAppend() throws Exception { final SizeInBytes max = SizeInBytes.valueOf(2, TraditionalBinaryPrefix.MEGA); RaftStorage storage = RaftStorageTestUtils.newRaftStorage(storageDir); - final File file = LogSegmentStartEnd.valueOf(0).getFile(storage); + final File file = ZERO_START_NULL_END.getFile(storage); final byte[] content = new byte[1024]; Arrays.fill(content, (byte) 1); @@ -368,7 +370,7 @@ public void testPreallocationAndAppend() throws Exception { @Test public void testZeroSizeInProgressFile() throws Exception { final RaftStorage storage = RaftStorageTestUtils.newRaftStorage(storageDir); - final File file = LogSegmentStartEnd.valueOf(0).getFile(storage); + final File file = ZERO_START_NULL_END.getFile(storage); storage.close(); // create zero size in-progress file diff --git a/ratis-test/src/test/java/org/apache/ratis/server/raftlog/segmented/TestRaftLogReadWrite.java b/ratis-test/src/test/java/org/apache/ratis/server/raftlog/segmented/TestRaftLogReadWrite.java index 4f151c733f..5600914070 100644 --- a/ratis-test/src/test/java/org/apache/ratis/server/raftlog/segmented/TestRaftLogReadWrite.java +++ b/ratis-test/src/test/java/org/apache/ratis/server/raftlog/segmented/TestRaftLogReadWrite.java @@ -44,6 +44,8 @@ import java.util.Collections; import java.util.List; +import static org.apache.ratis.server.raftlog.segmented.TestLogSegment.ZERO_START_NULL_END; + /** * Test basic functionality of LogReader, SegmentedRaftLogInputStream, and SegmentedRaftLogOutputStream. */ @@ -105,7 +107,7 @@ private long writeMessages(LogEntryProto[] entries, SegmentedRaftLogOutputStream @Test public void testReadWriteLog() throws IOException { final RaftStorage storage = RaftStorageTestUtils.newRaftStorage(storageDir); - final File openSegment = LogSegmentStartEnd.valueOf(0).getFile(storage); + final File openSegment = ZERO_START_NULL_END.getFile(storage); long size = SegmentedRaftLogFormat.getHeaderLength(); final LogEntryProto[] entries = new LogEntryProto[100]; @@ -125,7 +127,7 @@ public void testReadWriteLog() throws IOException { @Test public void testAppendLog() throws IOException { final RaftStorage storage = RaftStorageTestUtils.newRaftStorage(storageDir); - final File openSegment = LogSegmentStartEnd.valueOf(0).getFile(storage); + final File openSegment = ZERO_START_NULL_END.getFile(storage); LogEntryProto[] entries = new LogEntryProto[200]; try (SegmentedRaftLogOutputStream out = new SegmentedRaftLogOutputStream(openSegment, false, segmentMaxSize, preallocatedSize, ByteBuffer.allocateDirect(bufferSize))) { @@ -158,7 +160,7 @@ public void testAppendLog() throws IOException { @Test public void testReadWithPadding() throws IOException { final RaftStorage storage = RaftStorageTestUtils.newRaftStorage(storageDir); - final File openSegment = LogSegmentStartEnd.valueOf(0).getFile(storage); + final File openSegment = ZERO_START_NULL_END.getFile(storage); long size = SegmentedRaftLogFormat.getHeaderLength(); LogEntryProto[] entries = new LogEntryProto[100]; @@ -187,7 +189,7 @@ public void testReadWithPadding() throws IOException { @Test public void testReadWithCorruptPadding() throws IOException { final RaftStorage storage = RaftStorageTestUtils.newRaftStorage(storageDir); - final File openSegment = LogSegmentStartEnd.valueOf(0).getFile(storage); + final File openSegment = ZERO_START_NULL_END.getFile(storage); LogEntryProto[] entries = new LogEntryProto[10]; final SegmentedRaftLogOutputStream out = new SegmentedRaftLogOutputStream(openSegment, false, @@ -236,7 +238,7 @@ public void testReadWithCorruptPadding() throws IOException { @Test public void testReadWithEntryCorruption() throws IOException { RaftStorage storage = RaftStorageTestUtils.newRaftStorage(storageDir); - final File openSegment = LogSegmentStartEnd.valueOf(0).getFile(storage); + final File openSegment = ZERO_START_NULL_END.getFile(storage); try (SegmentedRaftLogOutputStream out = new SegmentedRaftLogOutputStream(openSegment, false, segmentMaxSize, preallocatedSize, ByteBuffer.allocateDirect(bufferSize))) { for (int i = 0; i < 100; i++) { From 6bbba497a5a83424999e58152ba0e88f86035256 Mon Sep 17 00:00:00 2001 From: 133tosakarin <97331129+133tosakarin@users.noreply.github.com> Date: Fri, 1 Nov 2024 23:29:18 +0800 Subject: [PATCH 109/397] RATIS-2183. Detect staled snapshot request. (#1173) --- .../ratis/grpc/server/GrpcLogAppender.java | 2 ++ ratis-proto/src/main/proto/Raft.proto | 1 + .../impl/SnapshotInstallationHandler.java | 17 +++++++++++++++++ .../ratis/server/leader/LogAppenderDefault.java | 1 + 4 files changed, 21 insertions(+) diff --git a/ratis-grpc/src/main/java/org/apache/ratis/grpc/server/GrpcLogAppender.java b/ratis-grpc/src/main/java/org/apache/ratis/grpc/server/GrpcLogAppender.java index 192bc75641..ce7bd315c5 100644 --- a/ratis-grpc/src/main/java/org/apache/ratis/grpc/server/GrpcLogAppender.java +++ b/ratis-grpc/src/main/java/org/apache/ratis/grpc/server/GrpcLogAppender.java @@ -708,6 +708,8 @@ public void onNext(InstallSnapshotReplyProto reply) { LOG.error("Unrecognized the reply result {}: Leader is {}, follower is {}", reply.getResult(), getServer().getId(), getFollowerId()); break; + case SNAPSHOT_EXPIRED: + LOG.warn("{}: Follower could not install snapshot as it is expired.", this); default: break; } diff --git a/ratis-proto/src/main/proto/Raft.proto b/ratis-proto/src/main/proto/Raft.proto index b2e96e283e..7cf2fd87c1 100644 --- a/ratis-proto/src/main/proto/Raft.proto +++ b/ratis-proto/src/main/proto/Raft.proto @@ -155,6 +155,7 @@ enum InstallSnapshotResult { CONF_MISMATCH = 4; SNAPSHOT_INSTALLED = 5; SNAPSHOT_UNAVAILABLE = 6; + SNAPSHOT_EXPIRED = 7; } message RequestVoteRequestProto { diff --git a/ratis-server/src/main/java/org/apache/ratis/server/impl/SnapshotInstallationHandler.java b/ratis-server/src/main/java/org/apache/ratis/server/impl/SnapshotInstallationHandler.java index 8de9a37569..70027e6dda 100644 --- a/ratis-server/src/main/java/org/apache/ratis/server/impl/SnapshotInstallationHandler.java +++ b/ratis-server/src/main/java/org/apache/ratis/server/impl/SnapshotInstallationHandler.java @@ -71,6 +71,8 @@ class SnapshotInstallationHandler { private final AtomicBoolean isSnapshotNull = new AtomicBoolean(); private final AtomicLong installedIndex = new AtomicLong(INVALID_LOG_INDEX); private final AtomicInteger nextChunkIndex = new AtomicInteger(-1); + /** The callId of the chunk with index 0. */ + private final AtomicLong chunk0CallId = new AtomicLong(-1); SnapshotInstallationHandler(RaftServerImpl server, RaftProperties properties) { this.server = server; @@ -176,8 +178,22 @@ private CompletableFuture checkAndInstallSnapshot(Ins state.setLeader(leaderId, "installSnapshot"); server.updateLastRpcTime(FollowerState.UpdateType.INSTALL_SNAPSHOT_START); + long callId = chunk0CallId.get(); + // 1. leaderTerm < currentTerm will never come here + // 2. leaderTerm == currentTerm && callId == request.getCallId() + // means the snapshotRequest is staled with the same leader + // 3. leaderTerm > currentTerm means this is a new snapshot request from a new leader, + // chunk0CallId will be reset when a snapshot request with requestIndex == 0 is received . + if (callId > request.getServerRequest().getCallId() && currentTerm == leaderTerm) { + LOG.warn("{}: Snapshot Request Staled: chunk 0 callId is {} but {}", getMemberId(), callId, + ServerStringUtils.toInstallSnapshotRequestString(request)); + InstallSnapshotReplyProto reply = toInstallSnapshotReplyProto(leaderId, getMemberId(), + currentTerm, snapshotChunkRequest.getRequestIndex(), InstallSnapshotResult.SNAPSHOT_EXPIRED); + return future.thenApply(dummy -> reply); + } if (snapshotChunkRequest.getRequestIndex() == 0) { nextChunkIndex.set(0); + chunk0CallId.set(request.getServerRequest().getCallId()); } else if (nextChunkIndex.get() != snapshotChunkRequest.getRequestIndex()) { throw new IOException("Snapshot request already failed at chunk index " + nextChunkIndex.get() + "; ignoring request with chunk index " + snapshotChunkRequest.getRequestIndex()); @@ -205,6 +221,7 @@ private CompletableFuture checkAndInstallSnapshot(Ins // re-load the state machine if this is the last chunk if (snapshotChunkRequest.getDone()) { state.reloadStateMachine(lastIncluded); + chunk0CallId.set(-1); } } finally { server.updateLastRpcTime(FollowerState.UpdateType.INSTALL_SNAPSHOT_COMPLETE); diff --git a/ratis-server/src/main/java/org/apache/ratis/server/leader/LogAppenderDefault.java b/ratis-server/src/main/java/org/apache/ratis/server/leader/LogAppenderDefault.java index 1b21bb7e48..9d1edd4695 100644 --- a/ratis-server/src/main/java/org/apache/ratis/server/leader/LogAppenderDefault.java +++ b/ratis-server/src/main/java/org/apache/ratis/server/leader/LogAppenderDefault.java @@ -150,6 +150,7 @@ public void run() throws InterruptedException, IOException { case SUCCESS: case SNAPSHOT_UNAVAILABLE: case ALREADY_INSTALLED: + case SNAPSHOT_EXPIRED: getFollower().setAttemptedToInstallSnapshot(); break; default: From 3db9c1a8180df30d43e63b6999376f4616fe2f13 Mon Sep 17 00:00:00 2001 From: OneSizeFitQuorum Date: Mon, 4 Nov 2024 15:06:57 +0800 Subject: [PATCH 110/397] Change version for the version 3.1.2 -rc1 --- pom.xml | 4 ++-- ratis-assembly/pom.xml | 2 +- ratis-client/pom.xml | 2 +- ratis-common/pom.xml | 2 +- ratis-docs/pom.xml | 2 +- ratis-examples/pom.xml | 2 +- ratis-experiments/pom.xml | 2 +- ratis-grpc/pom.xml | 2 +- ratis-metrics-api/pom.xml | 2 +- ratis-metrics-default/pom.xml | 2 +- ratis-metrics-dropwizard3/pom.xml | 2 +- ratis-netty/pom.xml | 2 +- ratis-proto/pom.xml | 2 +- ratis-replicated-map/pom.xml | 2 +- ratis-resource-bundle/pom.xml | 2 +- ratis-server-api/pom.xml | 2 +- ratis-server/pom.xml | 2 +- ratis-shell/pom.xml | 2 +- ratis-test/pom.xml | 2 +- ratis-tools/pom.xml | 2 +- 20 files changed, 21 insertions(+), 21 deletions(-) diff --git a/pom.xml b/pom.xml index 00aee4949f..0264dbf84f 100644 --- a/pom.xml +++ b/pom.xml @@ -24,7 +24,7 @@ ratis org.apache.ratis - 3.1.1 + 3.1.2 Apache Ratis pom @@ -162,7 +162,7 @@ - 2024-09-04T02:00:06Z + 2024-11-04T07:06:57Z UTF-8 UTF-8 diff --git a/ratis-assembly/pom.xml b/ratis-assembly/pom.xml index 60f04f7695..34e6e52c58 100644 --- a/ratis-assembly/pom.xml +++ b/ratis-assembly/pom.xml @@ -17,7 +17,7 @@ ratis org.apache.ratis - 3.1.1 + 3.1.2 ratis-assembly diff --git a/ratis-client/pom.xml b/ratis-client/pom.xml index 1c4cac0f8e..707db06fe2 100644 --- a/ratis-client/pom.xml +++ b/ratis-client/pom.xml @@ -17,7 +17,7 @@ ratis org.apache.ratis - 3.1.1 + 3.1.2 ratis-client diff --git a/ratis-common/pom.xml b/ratis-common/pom.xml index 1b25ae818a..66af275adf 100644 --- a/ratis-common/pom.xml +++ b/ratis-common/pom.xml @@ -17,7 +17,7 @@ ratis org.apache.ratis - 3.1.1 + 3.1.2 ratis-common diff --git a/ratis-docs/pom.xml b/ratis-docs/pom.xml index c794dd310e..578bef6c54 100644 --- a/ratis-docs/pom.xml +++ b/ratis-docs/pom.xml @@ -20,7 +20,7 @@ https://maven.apache.org/xsd/maven-4.0.0.xsd"> ratis org.apache.ratis - 3.1.1 + 3.1.2 ratis-docs diff --git a/ratis-examples/pom.xml b/ratis-examples/pom.xml index 58ce273c18..484ac096e3 100644 --- a/ratis-examples/pom.xml +++ b/ratis-examples/pom.xml @@ -17,7 +17,7 @@ ratis org.apache.ratis - 3.1.1 + 3.1.2 ratis-examples diff --git a/ratis-experiments/pom.xml b/ratis-experiments/pom.xml index 7addeb7bf5..5233bde418 100644 --- a/ratis-experiments/pom.xml +++ b/ratis-experiments/pom.xml @@ -17,7 +17,7 @@ ratis org.apache.ratis - 3.1.1 + 3.1.2 ratis-experiments diff --git a/ratis-grpc/pom.xml b/ratis-grpc/pom.xml index b1379bfc42..7a8f1b6c8f 100644 --- a/ratis-grpc/pom.xml +++ b/ratis-grpc/pom.xml @@ -17,7 +17,7 @@ ratis org.apache.ratis - 3.1.1 + 3.1.2 ratis-grpc diff --git a/ratis-metrics-api/pom.xml b/ratis-metrics-api/pom.xml index 660f8d1f77..2be02faa2e 100644 --- a/ratis-metrics-api/pom.xml +++ b/ratis-metrics-api/pom.xml @@ -18,7 +18,7 @@ ratis org.apache.ratis - 3.1.1 + 3.1.2 ratis-metrics-api diff --git a/ratis-metrics-default/pom.xml b/ratis-metrics-default/pom.xml index 91bd4e1e3d..9f8bc550be 100644 --- a/ratis-metrics-default/pom.xml +++ b/ratis-metrics-default/pom.xml @@ -18,7 +18,7 @@ ratis org.apache.ratis - 3.1.1 + 3.1.2 ratis-metrics-default diff --git a/ratis-metrics-dropwizard3/pom.xml b/ratis-metrics-dropwizard3/pom.xml index 0fc07eddee..bcb8a822f1 100644 --- a/ratis-metrics-dropwizard3/pom.xml +++ b/ratis-metrics-dropwizard3/pom.xml @@ -18,7 +18,7 @@ ratis org.apache.ratis - 3.1.1 + 3.1.2 ratis-metrics-dropwizard3 diff --git a/ratis-netty/pom.xml b/ratis-netty/pom.xml index 113c84371a..58605ca912 100644 --- a/ratis-netty/pom.xml +++ b/ratis-netty/pom.xml @@ -17,7 +17,7 @@ ratis org.apache.ratis - 3.1.1 + 3.1.2 ratis-netty diff --git a/ratis-proto/pom.xml b/ratis-proto/pom.xml index 010bd96fc1..e6c16473e6 100644 --- a/ratis-proto/pom.xml +++ b/ratis-proto/pom.xml @@ -17,7 +17,7 @@ ratis org.apache.ratis - 3.1.1 + 3.1.2 ratis-proto diff --git a/ratis-replicated-map/pom.xml b/ratis-replicated-map/pom.xml index 77276f7f88..b45b735745 100644 --- a/ratis-replicated-map/pom.xml +++ b/ratis-replicated-map/pom.xml @@ -17,7 +17,7 @@ ratis org.apache.ratis - 3.1.1 + 3.1.2 ratis-replicated-map diff --git a/ratis-resource-bundle/pom.xml b/ratis-resource-bundle/pom.xml index f22640a6e6..df59a55fd1 100644 --- a/ratis-resource-bundle/pom.xml +++ b/ratis-resource-bundle/pom.xml @@ -23,7 +23,7 @@ ratis org.apache.ratis - 3.1.1 + 3.1.2 .. diff --git a/ratis-server-api/pom.xml b/ratis-server-api/pom.xml index 345d66a1b4..ceeaeb1b24 100644 --- a/ratis-server-api/pom.xml +++ b/ratis-server-api/pom.xml @@ -17,7 +17,7 @@ ratis org.apache.ratis - 3.1.1 + 3.1.2 ratis-server-api diff --git a/ratis-server/pom.xml b/ratis-server/pom.xml index cfc8a9e88e..70098160a3 100644 --- a/ratis-server/pom.xml +++ b/ratis-server/pom.xml @@ -17,7 +17,7 @@ ratis org.apache.ratis - 3.1.1 + 3.1.2 ratis-server diff --git a/ratis-shell/pom.xml b/ratis-shell/pom.xml index 4c3caef784..608e1e8f19 100644 --- a/ratis-shell/pom.xml +++ b/ratis-shell/pom.xml @@ -17,7 +17,7 @@ ratis org.apache.ratis - 3.1.1 + 3.1.2 ratis-shell diff --git a/ratis-test/pom.xml b/ratis-test/pom.xml index 03e7326e88..5c0cefb259 100644 --- a/ratis-test/pom.xml +++ b/ratis-test/pom.xml @@ -17,7 +17,7 @@ ratis org.apache.ratis - 3.1.1 + 3.1.2 ratis-test diff --git a/ratis-tools/pom.xml b/ratis-tools/pom.xml index b7a0c15768..03ca14bf44 100644 --- a/ratis-tools/pom.xml +++ b/ratis-tools/pom.xml @@ -17,7 +17,7 @@ ratis org.apache.ratis - 3.1.1 + 3.1.2 ratis-tools From 2e4a748dda2d3379ded60ac1c9636854fc311e06 Mon Sep 17 00:00:00 2001 From: 133tosakarin <97331129+133tosakarin@users.noreply.github.com> Date: Sun, 10 Nov 2024 10:17:29 +0800 Subject: [PATCH 111/397] RATIS-2177. Purge should delete segmentLog from small to large according to logIndex (#1174) --- .../server/raftlog/segmented/SegmentedRaftLogWorker.java | 7 ++++--- 1 file changed, 4 insertions(+), 3 deletions(-) diff --git a/ratis-server/src/main/java/org/apache/ratis/server/raftlog/segmented/SegmentedRaftLogWorker.java b/ratis-server/src/main/java/org/apache/ratis/server/raftlog/segmented/SegmentedRaftLogWorker.java index a8482e71bd..c59c37be86 100644 --- a/ratis-server/src/main/java/org/apache/ratis/server/raftlog/segmented/SegmentedRaftLogWorker.java +++ b/ratis-server/src/main/java/org/apache/ratis/server/raftlog/segmented/SegmentedRaftLogWorker.java @@ -476,9 +476,10 @@ private PurgeLog(TruncationSegments segments) { void execute() throws IOException { if (segments.getToDelete() != null) { try(UncheckedAutoCloseable ignored = raftLogMetrics.startPurgeTimer()) { - for (SegmentFileInfo fileInfo : segments.getToDelete()) { - final Path deleted = FileUtils.deleteFile(fileInfo.getFile(storage)); - LOG.info("{}: Purged RaftLog segment: info={}, path={}", name, fileInfo, deleted); + SegmentFileInfo[] toDeletes = segments.getToDelete(); + for (int i = toDeletes.length - 1; i >= 0; i--) { + final Path deleted = FileUtils.deleteFile(toDeletes[i].getFile(storage)); + LOG.info("{}: Purged RaftLog segment: info={}, path={}", name, toDeletes[i], deleted); } } } From b0f6e2ef460d80290247c02b6734bacd7ea74b7f Mon Sep 17 00:00:00 2001 From: Ivan Andika Date: Wed, 13 Nov 2024 00:23:04 +0800 Subject: [PATCH 112/397] RATIS-2186. Raft log should not purge index lower than the log start index (#1175) --- .../ratis/server/raftlog/RaftLogBase.java | 20 +++++++++---- .../segmented/SegmentedRaftLogCache.java | 9 ++++-- .../segmented/TestSegmentedRaftLog.java | 30 ++++++++++++++++--- 3 files changed, 47 insertions(+), 12 deletions(-) diff --git a/ratis-server/src/main/java/org/apache/ratis/server/raftlog/RaftLogBase.java b/ratis-server/src/main/java/org/apache/ratis/server/raftlog/RaftLogBase.java index 1e22e07bd0..9b0367213a 100644 --- a/ratis-server/src/main/java/org/apache/ratis/server/raftlog/RaftLogBase.java +++ b/ratis-server/src/main/java/org/apache/ratis/server/raftlog/RaftLogBase.java @@ -314,20 +314,28 @@ public final CompletableFuture truncate(long index) { @Override public final CompletableFuture purge(long suggestedIndex) { + final long adjustedIndex; if (purgePreservation > 0) { final long currentIndex = getNextIndex() - 1; - suggestedIndex = Math.min(suggestedIndex, currentIndex - purgePreservation); + adjustedIndex = Math.min(suggestedIndex, currentIndex - purgePreservation); + } else { + adjustedIndex = suggestedIndex; } final long lastPurge = purgeIndex.get(); - if (suggestedIndex - lastPurge < purgeGap) { + if (adjustedIndex - lastPurge < purgeGap) { + return CompletableFuture.completedFuture(lastPurge); + } + final long startIndex = getStartIndex(); + if (adjustedIndex < startIndex) { + LOG.info("{}: purge({}) is skipped: adjustedIndex = {} < startIndex = {}, purgePreservation = {}", + getName(), suggestedIndex, adjustedIndex, startIndex, purgePreservation); return CompletableFuture.completedFuture(lastPurge); } - LOG.info("{}: purge {}", getName(), suggestedIndex); - final long finalSuggestedIndex = suggestedIndex; - return purgeImpl(suggestedIndex).whenComplete((purged, e) -> { + LOG.info("{}: purge {}", getName(), adjustedIndex ); + return purgeImpl(adjustedIndex).whenComplete((purged, e) -> { updatePurgeIndex(purged); if (e != null) { - LOG.warn(getName() + ": Failed to purge " + finalSuggestedIndex, e); + LOG.warn(getName() + ": Failed to purge " + adjustedIndex, e); } }); } diff --git a/ratis-server/src/main/java/org/apache/ratis/server/raftlog/segmented/SegmentedRaftLogCache.java b/ratis-server/src/main/java/org/apache/ratis/server/raftlog/segmented/SegmentedRaftLogCache.java index 20b7a5c372..e2ddb1508c 100644 --- a/ratis-server/src/main/java/org/apache/ratis/server/raftlog/segmented/SegmentedRaftLogCache.java +++ b/ratis-server/src/main/java/org/apache/ratis/server/raftlog/segmented/SegmentedRaftLogCache.java @@ -357,6 +357,10 @@ TruncationSegments purge(long index) { try (AutoCloseableLock writeLock = writeLock()) { int segmentIndex = binarySearch(index); List list = new ArrayList<>(); + if (segmentIndex == -1) { + // nothing to purge + return null; + } if (segmentIndex == -segments.size() - 1) { for (LogSegment ls : segments) { @@ -527,8 +531,9 @@ void rollOpenSegment(boolean createNewOpen) { } LogSegment getSegment(long index) { - if (openSegment != null && index >= openSegment.getStartIndex()) { - return openSegment; + final LogSegment open = this.openSegment; + if (open != null && index >= open.getStartIndex()) { + return open; } else { return closedSegments.search(index); } diff --git a/ratis-test/src/test/java/org/apache/ratis/server/raftlog/segmented/TestSegmentedRaftLog.java b/ratis-test/src/test/java/org/apache/ratis/server/raftlog/segmented/TestSegmentedRaftLog.java index 93eb7db0e6..a772b00029 100644 --- a/ratis-test/src/test/java/org/apache/ratis/server/raftlog/segmented/TestSegmentedRaftLog.java +++ b/ratis-test/src/test/java/org/apache/ratis/server/raftlog/segmented/TestSegmentedRaftLog.java @@ -563,6 +563,7 @@ public void testPurgeOnOpenSegment() throws Exception { int segmentSize = 200; long beginIndexOfOpenSegment = segmentSize * (endTerm - startTerm - 1); long expectedIndex = segmentSize * (endTerm - startTerm - 1); + long purgePreservation = 0L; purgeAndVerify(startTerm, endTerm, segmentSize, 1, beginIndexOfOpenSegment, expectedIndex); } @@ -599,15 +600,36 @@ public void testPurgeOnClosedSegmentsWithPurgeGap() throws Exception { purgeAndVerify(startTerm, endTerm, segmentSize, 1000, endIndexOfClosedSegment, expectedIndex); } + @Test + public void testPurgeWithLargePurgePreservationAndSmallPurgeGap() throws Exception { + int startTerm = 0; + int endTerm = 5; + int segmentSize = 200; + long endIndex = segmentSize * (endTerm - startTerm) - 1; + // start index is set so that the suggested index will not be negative, which will not trigger any purge + long startIndex = 200; + // purge preservation is larger than the total size of the log entries + // which causes suggested index to be lower than the start index + long purgePreservation = (segmentSize * (endTerm - startTerm )) + 100; + // if the suggested index is lower than the start index due to the purge preservation, we should not purge anything + purgeAndVerify(startTerm, endTerm, segmentSize, 1, endIndex, startIndex, startIndex, purgePreservation); + } + private void purgeAndVerify(int startTerm, int endTerm, int segmentSize, int purgeGap, long purgeIndex, - long expectedIndex) throws Exception { - List ranges = prepareRanges(startTerm, endTerm, segmentSize, 0); + long expectedIndex) throws Exception { + purgeAndVerify(startTerm, endTerm, segmentSize, purgeGap, purgeIndex, expectedIndex, 0, 0); + } + + private void purgeAndVerify(int startTerm, int endTerm, int segmentSize, int purgeGap, long purgeIndex, + long expectedIndex, long startIndex, long purgePreservation) throws Exception { + List ranges = prepareRanges(startTerm, endTerm, segmentSize, startIndex); List entries = prepareLogEntries(ranges, null); final RaftProperties p = new RaftProperties(); RaftServerConfigKeys.Log.setPurgeGap(p, purgeGap); - try (SegmentedRaftLog raftLog = newSegmentedRaftLog(storage, p)) { - raftLog.open(RaftLog.INVALID_LOG_INDEX, null); + RaftServerConfigKeys.Log.setPurgePreservationLogNum(p, purgePreservation); + try (SegmentedRaftLog raftLog = newSegmentedRaftLogWithSnapshotIndex(storage, p, () -> startIndex - 1)) { + raftLog.open(startIndex - 1, null); entries.stream().map(raftLog::appendEntry).forEach(CompletableFuture::join); final CompletableFuture f = raftLog.purge(purgeIndex); final Long purged = f.get(); From fa833bda49f54816590ef297660b9df05f1617f9 Mon Sep 17 00:00:00 2001 From: Ivan Andika Date: Sat, 16 Nov 2024 06:33:29 +0800 Subject: [PATCH 113/397] RATIS-2189. Use ByteBufAllocator#ioBuffer in NettyDataStreamUtils (#1178) --- .../java/org/apache/ratis/netty/NettyDataStreamUtils.java | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/ratis-netty/src/main/java/org/apache/ratis/netty/NettyDataStreamUtils.java b/ratis-netty/src/main/java/org/apache/ratis/netty/NettyDataStreamUtils.java index aa46cba53f..583d6e3e94 100644 --- a/ratis-netty/src/main/java/org/apache/ratis/netty/NettyDataStreamUtils.java +++ b/ratis-netty/src/main/java/org/apache/ratis/netty/NettyDataStreamUtils.java @@ -103,11 +103,11 @@ static void encodeDataStreamRequestHeader(DataStreamRequest request, Consumer out, ByteBufAllocator allocator) { ByteBuffer headerBuf = getDataStreamReplyHeaderProtoByteBuf(reply); - final ByteBuf headerLenBuf = allocator.directBuffer(DataStreamPacketHeader.getSizeOfHeaderLen()); + final ByteBuf headerLenBuf = allocator.ioBuffer(DataStreamPacketHeader.getSizeOfHeaderLen()); headerLenBuf.writeInt(headerBuf.remaining()); out.accept(headerLenBuf); out.accept(Unpooled.wrappedBuffer(headerBuf)); From ed434f592e5a15a3586f8cba69206965bcbf0a94 Mon Sep 17 00:00:00 2001 From: Ivan Andika Date: Sat, 23 Nov 2024 01:32:38 +0800 Subject: [PATCH 114/397] RATIS-2191. Add ResourceLeakDetector to Netty tests (#1180) --- .../filestore/FileStoreStreamingBaseTest.java | 1 + .../ratis/datastream/DataStreamClusterTests.java | 1 + .../apache/ratis/datastream/DataStreamTestUtils.java | 12 ++++++++++++ 3 files changed, 14 insertions(+) diff --git a/ratis-examples/src/test/java/org/apache/ratis/examples/filestore/FileStoreStreamingBaseTest.java b/ratis-examples/src/test/java/org/apache/ratis/examples/filestore/FileStoreStreamingBaseTest.java index 92147de4e9..490f12c8da 100644 --- a/ratis-examples/src/test/java/org/apache/ratis/examples/filestore/FileStoreStreamingBaseTest.java +++ b/ratis-examples/src/test/java/org/apache/ratis/examples/filestore/FileStoreStreamingBaseTest.java @@ -56,6 +56,7 @@ public abstract class FileStoreStreamingBaseTest ex implements MiniRaftCluster.Factory.Get { { setStateMachine(MultiDataStreamStateMachine.class); + DataStreamTestUtils.enableResourceLeakDetector(); } public static final int NUM_SERVERS = 3; diff --git a/ratis-test/src/test/java/org/apache/ratis/datastream/DataStreamTestUtils.java b/ratis-test/src/test/java/org/apache/ratis/datastream/DataStreamTestUtils.java index dfbd98addd..a2c3162f55 100644 --- a/ratis-test/src/test/java/org/apache/ratis/datastream/DataStreamTestUtils.java +++ b/ratis-test/src/test/java/org/apache/ratis/datastream/DataStreamTestUtils.java @@ -46,6 +46,9 @@ import org.apache.ratis.statemachine.TransactionContext; import org.apache.ratis.statemachine.impl.BaseStateMachine; import org.apache.ratis.thirdparty.com.google.protobuf.ByteString; +import org.apache.ratis.thirdparty.io.netty.buffer.ByteBufUtil; +import org.apache.ratis.thirdparty.io.netty.util.ResourceLeakDetector; +import org.apache.ratis.thirdparty.io.netty.util.ResourceLeakDetector.Level; import org.apache.ratis.util.CollectionUtils; import org.apache.ratis.util.FileUtils; import org.apache.ratis.util.JavaUtils; @@ -416,4 +419,13 @@ static void assertLogEntry(RaftServer.Division division, SingleDataStream stream final LogEntryProto entryFromLog = searchLogEntry(ClientInvocationId.valueOf(request), division.getRaftLog()); Assertions.assertEquals(entryFromStream, entryFromLog); } + + ResourceLeakDetector.LeakListener LEAK_LISTENER = (resourceType, records) -> { + throw new IllegalStateException("Leak detected for resource type: " + resourceType + records); + }; + + static void enableResourceLeakDetector() { + ResourceLeakDetector.setLevel(Level.PARANOID); + ByteBufUtil.setLeakListener(DataStreamTestUtils.LEAK_LISTENER); + } } From 9ed1792ae484d6f698a30567d1effa809c154073 Mon Sep 17 00:00:00 2001 From: Chung En Lee Date: Wed, 27 Nov 2024 07:45:18 +0800 Subject: [PATCH 115/397] RATIS-2194. FileLock didn't unlock properly (#1183) --- .../apache/ratis/server/impl/RaftServerProxy.java | 13 +++++++------ 1 file changed, 7 insertions(+), 6 deletions(-) diff --git a/ratis-server/src/main/java/org/apache/ratis/server/impl/RaftServerProxy.java b/ratis-server/src/main/java/org/apache/ratis/server/impl/RaftServerProxy.java index 84221cfcfe..242bb377b7 100644 --- a/ratis-server/src/main/java/org/apache/ratis/server/impl/RaftServerProxy.java +++ b/ratis-server/src/main/java/org/apache/ratis/server/impl/RaftServerProxy.java @@ -415,6 +415,13 @@ private void startImpl() throws IOException { public void close() { lifeCycle.checkStateAndClose(() -> { LOG.info("{}: close", getId()); + + try { + ConcurrentUtils.shutdownAndWait(implExecutor.get()); + } catch (Exception ignored) { + LOG.warn(getId() + ": Failed to shutdown implExecutor", ignored); + } + impls.close(); try { @@ -429,12 +436,6 @@ public void close() { LOG.warn(getId() + ": Failed to close " + SupportedDataStreamType.NETTY + " server", ignored); } - try { - ConcurrentUtils.shutdownAndWait(implExecutor.get()); - } catch (Exception ignored) { - LOG.warn(getId() + ": Failed to shutdown implExecutor", ignored); - } - try { ConcurrentUtils.shutdownAndWait(executor.get()); } catch (Exception ignored) { From e0aa88ce13806835d22887cb8e117221ad6b4102 Mon Sep 17 00:00:00 2001 From: Potato Date: Thu, 28 Nov 2024 17:28:51 +0800 Subject: [PATCH 116/397] RATIS-2201. Bump ratis-thirdparty from 1.0.6 to 1.0.7 (#1185) --- pom.xml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/pom.xml b/pom.xml index 0264dbf84f..0ccb292d45 100644 --- a/pom.xml +++ b/pom.xml @@ -210,7 +210,7 @@ 3.3.9 - 1.0.6 + 1.0.7 3.24.4 From 9b5b3ac0c5d15ed0dbcf1d02c88315f708141a84 Mon Sep 17 00:00:00 2001 From: Tsz-Wo Nicholas Sze Date: Sun, 1 Dec 2024 18:01:21 -0800 Subject: [PATCH 117/397] RATIS-2185. Improve gRPC log messages debugability. (#1186) --- .../java/org/apache/ratis/util/LogUtils.java | 8 +++++- .../java/org/apache/ratis/grpc/GrpcUtil.java | 7 ----- .../grpc/server/GrpcAdminProtocolService.java | 27 +++++++++++++------ 3 files changed, 26 insertions(+), 16 deletions(-) diff --git a/ratis-common/src/main/java/org/apache/ratis/util/LogUtils.java b/ratis-common/src/main/java/org/apache/ratis/util/LogUtils.java index d29f1e56ec..63bb08accf 100644 --- a/ratis-common/src/main/java/org/apache/ratis/util/LogUtils.java +++ b/ratis-common/src/main/java/org/apache/ratis/util/LogUtils.java @@ -118,7 +118,13 @@ static void warn(Logger log, Supplier message, Throwable t, Class... if (log.isWarnEnabled()) { if (ReflectionUtils.isInstance(t, exceptionClasses)) { // do not print stack trace for known exceptions. - log.warn(message.get() + ": " + t); + final StringBuilder b = new StringBuilder() + .append(message.get()) + .append(": ").append(t); + for(Throwable cause = t.getCause(); cause != null; cause = cause.getCause()) { + b.append("\n Caused by: ").append(cause); + } + log.warn(b.toString()); } else { log.warn(message.get(), t); } diff --git a/ratis-grpc/src/main/java/org/apache/ratis/grpc/GrpcUtil.java b/ratis-grpc/src/main/java/org/apache/ratis/grpc/GrpcUtil.java index 22653b6efb..3645980abf 100644 --- a/ratis-grpc/src/main/java/org/apache/ratis/grpc/GrpcUtil.java +++ b/ratis-grpc/src/main/java/org/apache/ratis/grpc/GrpcUtil.java @@ -163,13 +163,6 @@ static IOException unwrapIOException(Throwable t) { return e; } - static void asyncCall( - StreamObserver responseObserver, - CheckedSupplier, IOException> supplier, - Function toProto) { - asyncCall(responseObserver, supplier, toProto, throwable -> {}); - } - static void asyncCall( StreamObserver responseObserver, CheckedSupplier, IOException> supplier, diff --git a/ratis-grpc/src/main/java/org/apache/ratis/grpc/server/GrpcAdminProtocolService.java b/ratis-grpc/src/main/java/org/apache/ratis/grpc/server/GrpcAdminProtocolService.java index feb780302e..f6a6cfba6a 100644 --- a/ratis-grpc/src/main/java/org/apache/ratis/grpc/server/GrpcAdminProtocolService.java +++ b/ratis-grpc/src/main/java/org/apache/ratis/grpc/server/GrpcAdminProtocolService.java @@ -1,4 +1,4 @@ -/** +/* * 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 @@ -32,8 +32,12 @@ import org.apache.ratis.proto.RaftProtos.RaftClientReplyProto; import org.apache.ratis.proto.RaftProtos.GroupManagementRequestProto; import org.apache.ratis.proto.grpc.AdminProtocolServiceGrpc.AdminProtocolServiceImplBase; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; public class GrpcAdminProtocolService extends AdminProtocolServiceImplBase { + static final Logger LOG = LoggerFactory.getLogger(GrpcAdminProtocolService.class); + private final AdminAsynchronousProtocol protocol; public GrpcAdminProtocolService(AdminAsynchronousProtocol protocol) { @@ -45,7 +49,8 @@ public void groupManagement(GroupManagementRequestProto proto, StreamObserver responseObserver) { final GroupManagementRequest request = ClientProtoUtils.toGroupManagementRequest(proto); GrpcUtil.asyncCall(responseObserver, () -> protocol.groupManagementAsync(request), - ClientProtoUtils::toRaftClientReplyProto); + ClientProtoUtils::toRaftClientReplyProto, + t -> LOG.warn("Failed groupManagement: {}, {}", proto.getOpCase(), request, t)); } @Override @@ -53,14 +58,16 @@ public void groupList(GroupListRequestProto proto, StreamObserver responseObserver) { final GroupListRequest request = ClientProtoUtils.toGroupListRequest(proto); GrpcUtil.asyncCall(responseObserver, () -> protocol.getGroupListAsync(request), - ClientProtoUtils::toGroupListReplyProto); + ClientProtoUtils::toGroupListReplyProto, + t -> LOG.warn("Failed to groupList: {}", request, t)); } @Override public void groupInfo(GroupInfoRequestProto proto, StreamObserver responseObserver) { final GroupInfoRequest request = ClientProtoUtils.toGroupInfoRequest(proto); GrpcUtil.asyncCall(responseObserver, () -> protocol.getGroupInfoAsync(request), - ClientProtoUtils::toGroupInfoReplyProto); + ClientProtoUtils::toGroupInfoReplyProto, + t -> LOG.warn("Failed to groupInfo: {}", request, t)); } @Override @@ -68,7 +75,8 @@ public void setConfiguration(SetConfigurationRequestProto proto, StreamObserver responseObserver) { final SetConfigurationRequest request = ClientProtoUtils.toSetConfigurationRequest(proto); GrpcUtil.asyncCall(responseObserver, () -> protocol.setConfigurationAsync(request), - ClientProtoUtils::toRaftClientReplyProto); + ClientProtoUtils::toRaftClientReplyProto, + t -> LOG.warn("Failed to setConfiguration: {}", request, t)); } @Override @@ -76,7 +84,8 @@ public void transferLeadership(TransferLeadershipRequestProto proto, StreamObserver responseObserver) { final TransferLeadershipRequest request = ClientProtoUtils.toTransferLeadershipRequest(proto); GrpcUtil.asyncCall(responseObserver, () -> protocol.transferLeadershipAsync(request), - ClientProtoUtils::toRaftClientReplyProto); + ClientProtoUtils::toRaftClientReplyProto, + t -> LOG.warn("Failed to transferLeadership: {}", request, t)); } @Override @@ -84,7 +93,8 @@ public void snapshotManagement(SnapshotManagementRequestProto proto, StreamObserver responseObserver) { final SnapshotManagementRequest request = ClientProtoUtils.toSnapshotManagementRequest(proto); GrpcUtil.asyncCall(responseObserver, () -> protocol.snapshotManagementAsync(request), - ClientProtoUtils::toRaftClientReplyProto); + ClientProtoUtils::toRaftClientReplyProto, + t -> LOG.warn("Failed snapshotManagement: {}, {}", proto.getOpCase(), request, t)); } @Override @@ -92,6 +102,7 @@ public void leaderElectionManagement(LeaderElectionManagementRequestProto proto, StreamObserver responseObserver) { final LeaderElectionManagementRequest request = ClientProtoUtils.toLeaderElectionManagementRequest(proto); GrpcUtil.asyncCall(responseObserver, () -> protocol.leaderElectionManagementAsync(request), - ClientProtoUtils::toRaftClientReplyProto); + ClientProtoUtils::toRaftClientReplyProto, + t -> LOG.warn("Failed leaderElectionManagement: {}, {}", proto.getOpCase(), request, t)); } } From 6b5e978869d1651186adf16428931d67081825b0 Mon Sep 17 00:00:00 2001 From: "Doroszlai, Attila" <6454655+adoroszlai@users.noreply.github.com> Date: Wed, 11 Dec 2024 18:20:27 +0100 Subject: [PATCH 118/397] RATIS-2204. Avoid downloads from repository.apache.org (#1187) --- dev-support/checks/build.sh | 2 +- dev-support/checks/checkstyle.sh | 2 +- dev-support/checks/coverage.sh | 2 +- dev-support/checks/findbugs.sh | 2 +- dev-support/checks/rat.sh | 2 +- dev-support/checks/repro.sh | 2 +- dev-support/checks/sonar.sh | 2 +- dev-support/checks/unit.sh | 2 +- pom.xml | 12 ++++-------- 9 files changed, 12 insertions(+), 16 deletions(-) diff --git a/dev-support/checks/build.sh b/dev-support/checks/build.sh index 6add1ae605..ee8d3f3d7e 100755 --- a/dev-support/checks/build.sh +++ b/dev-support/checks/build.sh @@ -20,7 +20,7 @@ source "${DIR}/../find_maven.sh" : ${WITH_COVERAGE:="false"} -MAVEN_OPTIONS='-V -B -Dmaven.javadoc.skip=true -DskipTests --no-transfer-progress' +MAVEN_OPTIONS='-V -B -Dmaven.javadoc.skip=true -DskipTests' if [[ "${WITH_COVERAGE}" != "true" ]]; then MAVEN_OPTIONS="${MAVEN_OPTIONS} -Djacoco.skip" diff --git a/dev-support/checks/checkstyle.sh b/dev-support/checks/checkstyle.sh index a2ee427380..cb06fdaacd 100755 --- a/dev-support/checks/checkstyle.sh +++ b/dev-support/checks/checkstyle.sh @@ -23,7 +23,7 @@ REPORT_DIR=${OUTPUT_DIR:-"$DIR/../../target/checkstyle"} mkdir -p "$REPORT_DIR" REPORT_FILE="$REPORT_DIR/summary.txt" -MAVEN_OPTIONS='-B -fae --no-transfer-progress -Dcheckstyle.failOnViolation=false' +MAVEN_OPTIONS='-B -fae -Dcheckstyle.failOnViolation=false' declare -i rc ${MVN} ${MAVEN_OPTIONS} checkstyle:check | tee "${REPORT_DIR}/output.log" diff --git a/dev-support/checks/coverage.sh b/dev-support/checks/coverage.sh index a2fab9b32a..ff0aef1a48 100755 --- a/dev-support/checks/coverage.sh +++ b/dev-support/checks/coverage.sh @@ -29,7 +29,7 @@ mkdir -p "$REPORT_DIR" JACOCO_VERSION=$(${MVN} help:evaluate -Dexpression=jacoco.version -q -DforceStdout) #Install jacoco cli -${MVN} --non-recursive --no-transfer-progress \ +${MVN} --non-recursive \ org.apache.maven.plugins:maven-dependency-plugin:3.6.1:copy \ -Dartifact=org.jacoco:org.jacoco.cli:${JACOCO_VERSION}:jar:nodeps diff --git a/dev-support/checks/findbugs.sh b/dev-support/checks/findbugs.sh index 17c669b8d5..3a063b3fa1 100755 --- a/dev-support/checks/findbugs.sh +++ b/dev-support/checks/findbugs.sh @@ -20,7 +20,7 @@ source "${DIR}/../find_maven.sh" : ${WITH_COVERAGE:="false"} -MAVEN_OPTIONS='-B -fae --no-transfer-progress' +MAVEN_OPTIONS='-B -fae' if ! type unionBugs >/dev/null 2>&1 || ! type convertXmlToText >/dev/null 2>&1; then #shellcheck disable=SC2086 diff --git a/dev-support/checks/rat.sh b/dev-support/checks/rat.sh index 34d8a25854..9b55878eff 100755 --- a/dev-support/checks/rat.sh +++ b/dev-support/checks/rat.sh @@ -23,7 +23,7 @@ mkdir -p "$REPORT_DIR" REPORT_FILE="$REPORT_DIR/summary.txt" -${MVN} -B -fn --no-transfer-progress org.apache.rat:apache-rat-plugin:0.13:check +${MVN} -B -fn org.apache.rat:apache-rat-plugin:0.13:check cd "$DIR/../.." || exit 1 diff --git a/dev-support/checks/repro.sh b/dev-support/checks/repro.sh index c87443ef21..88941bc286 100755 --- a/dev-support/checks/repro.sh +++ b/dev-support/checks/repro.sh @@ -20,7 +20,7 @@ source "${DIR}/../find_maven.sh" : ${WITH_COVERAGE:="false"} -MAVEN_OPTIONS='-V -B -Dmaven.javadoc.skip=true -DskipTests --no-transfer-progress' +MAVEN_OPTIONS='-V -B -Dmaven.javadoc.skip=true -DskipTests' if [[ "${WITH_COVERAGE}" != "true" ]]; then MAVEN_OPTIONS="${MAVEN_OPTIONS} -Djacoco.skip" diff --git a/dev-support/checks/sonar.sh b/dev-support/checks/sonar.sh index 55a46cfec7..9646ab9d8f 100755 --- a/dev-support/checks/sonar.sh +++ b/dev-support/checks/sonar.sh @@ -23,7 +23,7 @@ if [ ! "$SONAR_TOKEN" ]; then exit 1 fi -${MVN} -B verify -DskipShade -DskipTests --no-transfer-progress \ +${MVN} -B verify -DskipShade -DskipTests \ org.sonarsource.scanner.maven:sonar-maven-plugin:3.6.0.1398:sonar \ -Dsonar.coverage.jacoco.xmlReportPaths="$(pwd)/target/coverage/all.xml" \ -Dsonar.host.url=https://sonarcloud.io -Dsonar.organization=apache -Dsonar.projectKey=apache-ratis diff --git a/dev-support/checks/unit.sh b/dev-support/checks/unit.sh index 389c5c5275..c652136da7 100755 --- a/dev-support/checks/unit.sh +++ b/dev-support/checks/unit.sh @@ -34,7 +34,7 @@ REPORT_DIR=${OUTPUT_DIR:-"$DIR/../../target/unit"} mkdir -p "$REPORT_DIR" export MAVEN_OPTS="-Xmx4096m" -MAVEN_OPTIONS='-V -B --no-transfer-progress' +MAVEN_OPTIONS='-V -B' if [[ "${FAIL_FAST}" == "true" ]]; then MAVEN_OPTIONS="${MAVEN_OPTIONS} --fail-fast -Dsurefire.skipAfterFailureCount=1" diff --git a/pom.xml b/pom.xml index 0ccb292d45..4f964224e1 100644 --- a/pom.xml +++ b/pom.xml @@ -51,6 +51,9 @@ ${distMgmtSnapshotsId} ${distMgmtSnapshotsName} ${distMgmtSnapshotsUrl} + + false + repository.jboss.org @@ -91,13 +94,6 @@ ratis-assembly - - - apache.snapshots - https://repository.apache.org/snapshots/ - - - scm:git:git://git.apache.org/ratis.git scm:git:https://git-wip-us.apache.org/repos/asf/ratis.git @@ -1102,7 +1098,7 @@ package - makeAggregateBom + makeBom From 4d004a168871f9fe6f01a27f77e2c667474a537d Mon Sep 17 00:00:00 2001 From: "Doroszlai, Attila" <6454655+adoroszlai@users.noreply.github.com> Date: Sat, 14 Dec 2024 13:16:24 +0100 Subject: [PATCH 119/397] RATIS-2206. Jars in Maven repo and binary tarball are not the same (#1188) --- dev-support/make_rc.sh | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/dev-support/make_rc.sh b/dev-support/make_rc.sh index dc56af595d..0be04945ef 100755 --- a/dev-support/make_rc.sh +++ b/dev-support/make_rc.sh @@ -118,7 +118,7 @@ prepare-bin() { mv "apache-ratis-${RATISVERSION}-src" "apache-ratis-${RATISVERSION}" cd "apache-ratis-${RATISVERSION}" - mvnFun clean install -DskipTests=true -Prelease -Papache-release -Dgpg.keyname="${CODESIGNINGKEY}" + mvnFun clean verify -DskipTests=true -Prelease -Papache-release -Dgpg.keyname="${CODESIGNINGKEY}" } assembly() { @@ -126,7 +126,7 @@ assembly() { RCDIR="$SVNDISTDIR/${RATISVERSION}/${RC#-}" mkdir -p "$RCDIR" cd "$RCDIR" - cp "$WORKINGDIR/apache-ratis-${RATISVERSION}/ratis-assembly/target/ratis-assembly-${RATISVERSION}-bin.tar.gz" "apache-ratis-${RATISVERSION}-bin.tar.gz" + cp "$projectdir/ratis-assembly/target/ratis-assembly-${RATISVERSION}-bin.tar.gz" "apache-ratis-${RATISVERSION}-bin.tar.gz" cp "$projectdir/ratis-assembly/target/ratis-assembly-${RATISVERSION}-src.tar.gz" "apache-ratis-${RATISVERSION}-src.tar.gz" for i in *.tar.gz; do gpg -u "${CODESIGNINGKEY}" --armor --output "${i}.asc" --detach-sig "${i}"; done for i in *.tar.gz; do gpg --print-md SHA512 "${i}" > "${i}.sha512"; done @@ -147,7 +147,7 @@ publish-svn() { publish-mvn(){ cd "$projectdir" - mvnFun clean deploy -DskipTests=true -Prelease -Papache-release -Dgpg.keyname="${CODESIGNINGKEY}" + mvnFun deploy:deploy } if [ "$#" -ne 1 ]; then From 4eac3412a48c58d9b3052f4d8dd03ac61ee57701 Mon Sep 17 00:00:00 2001 From: "Doroszlai, Attila" <6454655+adoroszlai@users.noreply.github.com> Date: Tue, 17 Dec 2024 08:09:55 +0100 Subject: [PATCH 120/397] RATIS-2211. publish-mvn fails with: ... did not assign a file to the build artifact (#1190) --- dev-support/make_rc.sh | 40 +++++++++++++++++++++------------------- 1 file changed, 21 insertions(+), 19 deletions(-) diff --git a/dev-support/make_rc.sh b/dev-support/make_rc.sh index 0be04945ef..65dbf79a10 100755 --- a/dev-support/make_rc.sh +++ b/dev-support/make_rc.sh @@ -91,7 +91,7 @@ mvnFun() { MAVEN_OPTS="${mvnopts}" ${MVN} -Dmaven.repo.local="${repodir}" "$@" } -prepare-src() { +1-prepare-src() { cd "$projectdir" git reset --hard git clean -fdx @@ -109,7 +109,7 @@ prepare-src() { mvnFun clean install -DskipTests=true -Prelease -Papache-release -Dgpg.keyname="${CODESIGNINGKEY}" } -prepare-bin() { +2-verify-bin() { echo "Cleaning up workingdir $WORKINGDIR" rm -rf "$WORKINGDIR" mkdir -p "$WORKINGDIR" @@ -121,7 +121,12 @@ prepare-bin() { mvnFun clean verify -DskipTests=true -Prelease -Papache-release -Dgpg.keyname="${CODESIGNINGKEY}" } -assembly() { +3-publish-mvn() { + cd "$projectdir" + mvnFun verify artifact:compare deploy:deploy -DdeployAtEnd=true -DskipTests=true -Prelease -Papache-release -Dgpg.keyname="${CODESIGNINGKEY}" "$@" +} + +4-assembly() { cd "$SVNDISTDIR" RCDIR="$SVNDISTDIR/${RATISVERSION}/${RC#-}" mkdir -p "$RCDIR" @@ -135,40 +140,35 @@ assembly() { svn add "${RATISVERSION}" || svn add "${RATISVERSION}/${RC#-}" } -publish-git(){ +5-publish-git(){ cd "$projectdir" git push apache "ratis-${RATISVERSION}${RC}" } -publish-svn() { +6-publish-svn() { cd "${SVNDISTDIR}" svn commit -m "Publish proposed version of the next Ratis release ${RATISVERSION}${RC}" } -publish-mvn(){ - cd "$projectdir" - mvnFun deploy:deploy -} - -if [ "$#" -ne 1 ]; then +if [ "$#" -lt 1 ]; then cat << EOF -Please choose from available phases (eg. make_rc.sh prepare-src): +Please choose from available phases (eg. make_rc.sh 1-prepare-src): - 1. prepare-src: This is the first step. It modifies the mvn version, creates the git tag and + 1-prepare-src: This is the first step. It modifies the mvn version, creates the git tag and builds the project to create the source artifacts. IT INCLUDES A GIT RESET + CLEAN. ALL THE LOCAL CHANGES WILL BE LOST! - 2. prepare-bin: The source artifact is copied to the $WORKINGDIR and the binary artifact is created from the source. + 2-verify-bin: The source artifact is copied to the $WORKINGDIR and the binary artifact is created from the source. This is an additional check as the the released source artifact should be enough to build the whole project. - 3. assembly : This step copies all the required artifacts to the svn directory and ($SVNDISTDIR) creates the signatures/checksum files. + 3-publish-mvn: Performs the final build, and uploads the artifacts to the maven staging repository - 4. publish-git: The first remote step, only do it if everything is fine. It pushes the rc tag to the repository. + 4-assembly: This step copies all the required artifacts to the svn directory and ($SVNDISTDIR) creates the signatures/checksum files. - 5. publish-svn: Uploads the artifacts to the apache dev staging area to start the vote. + 5-publish-git: Only do it if everything is fine. It pushes the rc tag to the repository. - 6. publish-mvn: Uploads the artifacts to the maven staging repository + 6-publish-svn: Uploads the artifacts to the apache dev staging area to start the vote. The next steps of the release process are not scripted: @@ -189,5 +189,7 @@ The next steps of the release process are not scripted: EOF else set -x - eval "$1" + func="$1" + shift + eval "$func" "$@" fi From 36ec0e12ff3c90ceb87736d50b1d42116a98aece Mon Sep 17 00:00:00 2001 From: "Doroszlai, Attila" <6454655+adoroszlai@users.noreply.github.com> Date: Sat, 21 Dec 2024 15:33:58 +0100 Subject: [PATCH 121/397] RATIS-2216. Bump maven-shade-plugin to 3.6.0 (#1191) --- pom.xml | 6 ++++++ ratis-examples/pom.xml | 2 +- ratis-experiments/pom.xml | 2 +- 3 files changed, 8 insertions(+), 2 deletions(-) diff --git a/pom.xml b/pom.xml index 4f964224e1..109c467285 100644 --- a/pom.xml +++ b/pom.xml @@ -172,6 +172,7 @@ 3.3.0 4.0.6 1.6.1 + 3.6.0 3.0.0-M4 3.5.3 3.4.0 @@ -494,6 +495,11 @@ + + org.apache.maven.plugins + maven-shade-plugin + ${maven-shade-plugin.version} + com.github.spotbugs spotbugs-maven-plugin diff --git a/ratis-examples/pom.xml b/ratis-examples/pom.xml index 484ac096e3..c3d0eee057 100644 --- a/ratis-examples/pom.xml +++ b/ratis-examples/pom.xml @@ -150,7 +150,7 @@ shade - ${project.build.directory}/dependency-reduced-pom.xml + false true diff --git a/ratis-experiments/pom.xml b/ratis-experiments/pom.xml index 5233bde418..4c3bc9ccd3 100644 --- a/ratis-experiments/pom.xml +++ b/ratis-experiments/pom.xml @@ -120,7 +120,7 @@ shade - ${project.build.directory}/dependency-reduced-pom.xml + false true From bf322bb80f7e775310c2b47131a2b3de261b678a Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?=E8=8D=BB=E5=8D=87?= <63136897+ongdisheng@users.noreply.github.com> Date: Sun, 22 Dec 2024 21:27:48 +0800 Subject: [PATCH 122/397] RATIS-212. Fix typo in RaftAsyncTests (#1197) Co-authored-by: Janos Gub --- .../src/test/java/org/apache/ratis/RaftAsyncTests.java | 8 ++++---- 1 file changed, 4 insertions(+), 4 deletions(-) diff --git a/ratis-server/src/test/java/org/apache/ratis/RaftAsyncTests.java b/ratis-server/src/test/java/org/apache/ratis/RaftAsyncTests.java index 925b8bbadb..3a760a8065 100644 --- a/ratis-server/src/test/java/org/apache/ratis/RaftAsyncTests.java +++ b/ratis-server/src/test/java/org/apache/ratis/RaftAsyncTests.java @@ -281,18 +281,18 @@ public void testStaleReadAsync() throws Exception { } void runTestStaleReadAsync(CLUSTER cluster) throws Exception { - final int numMesssages = 10; + final int numMessages = 10; try (RaftClient client = cluster.createClient()) { RaftTestUtil.waitForLeader(cluster); // submit some messages final List> futures = new ArrayList<>(); - for (int i = 0; i < numMesssages; i++) { + for (int i = 0; i < numMessages; i++) { final String s = "" + i; LOG.info("sendAsync " + s); futures.add(client.async().send(new SimpleMessage(s))); } - Assert.assertEquals(numMesssages, futures.size()); + Assert.assertEquals(numMessages, futures.size()); final List replies = new ArrayList<>(); for (CompletableFuture f : futures) { final RaftClientReply r = f.join(); @@ -322,7 +322,7 @@ void runTestStaleReadAsync(CLUSTER cluster) throws Exception { StateMachineException.class, IndexOutOfBoundsException.class); // test sendStaleReadAsync - for (int i = 0; i < numMesssages; i++) { + for (int i = 0; i < numMessages; i++) { final RaftClientReply reply = replies.get(i); final String query = "" + i; LOG.info("query=" + query + ", reply=" + reply); From 2b9f873dcfa2981ff70bab77c05bb6df91c3fafd Mon Sep 17 00:00:00 2001 From: "Doroszlai, Attila" <6454655+adoroszlai@users.noreply.github.com> Date: Sun, 22 Dec 2024 17:11:13 +0100 Subject: [PATCH 123/397] RATIS-2222. Remove copy-rename-maven-plugin usage (#1194) --- pom.xml | 6 ------ ratis-examples/pom.xml | 19 +------------------ ratis-experiments/pom.xml | 19 +------------------ 3 files changed, 2 insertions(+), 42 deletions(-) diff --git a/pom.xml b/pom.xml index 109c467285..ac3d237f1d 100644 --- a/pom.xml +++ b/pom.xml @@ -186,7 +186,6 @@ 2.2.0 0.6.1 - 1.0 2.8.0 4.2.1 @@ -473,11 +472,6 @@ license-maven-plugin ${license-maven-plugin.version} - - com.coderplus.maven.plugins - copy-rename-maven-plugin - ${copy-rename-maven-plugin.version} - org.apache.maven.plugins diff --git a/ratis-examples/pom.xml b/ratis-examples/pom.xml index c3d0eee057..efd3ba67d6 100644 --- a/ratis-examples/pom.xml +++ b/ratis-examples/pom.xml @@ -151,7 +151,7 @@ false - true + false org.apache.ratis.examples.common.Runner @@ -184,23 +184,6 @@ - - com.coderplus.maven.plugins - copy-rename-maven-plugin - - - copy-file - package - - copy - - - target/${project.artifactId}-${project.version}-shaded.jar - target/${project.artifactId}-${project.version}.jar - - - - diff --git a/ratis-experiments/pom.xml b/ratis-experiments/pom.xml index 4c3bc9ccd3..86ed7985f7 100644 --- a/ratis-experiments/pom.xml +++ b/ratis-experiments/pom.xml @@ -121,7 +121,7 @@ false - true + false *:* @@ -140,23 +140,6 @@ - - com.coderplus.maven.plugins - copy-rename-maven-plugin - - - copy-file - package - - copy - - - target/${project.artifactId}-${project.version}-shaded.jar - target/${project.artifactId}-${project.version}.jar - - - - From b712f58d8d767250c3d2044fe43dc5910093b7a4 Mon Sep 17 00:00:00 2001 From: "Doroszlai, Attila" <6454655+adoroszlai@users.noreply.github.com> Date: Mon, 23 Dec 2024 20:16:42 +0100 Subject: [PATCH 124/397] RATIS-2219. Remove duplicate test case from TestRaftWithGrpc (#1192) --- .../apache/ratis/grpc/TestRaftWithGrpc.java | 23 ++++++++++--------- 1 file changed, 12 insertions(+), 11 deletions(-) diff --git a/ratis-test/src/test/java/org/apache/ratis/grpc/TestRaftWithGrpc.java b/ratis-test/src/test/java/org/apache/ratis/grpc/TestRaftWithGrpc.java index b93621137d..5821b7f227 100644 --- a/ratis-test/src/test/java/org/apache/ratis/grpc/TestRaftWithGrpc.java +++ b/ratis-test/src/test/java/org/apache/ratis/grpc/TestRaftWithGrpc.java @@ -31,11 +31,10 @@ import org.apache.ratis.util.JavaUtils; import org.apache.ratis.util.TimeDuration; import org.junit.jupiter.api.Assertions; +import org.junit.jupiter.api.Disabled; import org.junit.jupiter.params.ParameterizedTest; -import org.junit.jupiter.params.provider.MethodSource; +import org.junit.jupiter.params.provider.ValueSource; -import java.util.Arrays; -import java.util.Collection; import java.util.concurrent.CompletableFuture; import java.util.concurrent.TimeUnit; @@ -50,28 +49,30 @@ public class TestRaftWithGrpc SimpleStateMachine4Testing.class, StateMachine.class); } - public static Collection data() { - return Arrays.asList((new Boolean[][] {{Boolean.FALSE}, {Boolean.TRUE}})); + @Disabled + @Override + public void testWithLoad() { + // skip testWithLoad() from parent, called from parameterized testWithLoad(boolean) } @ParameterizedTest - @MethodSource("data") - public void testWithLoad(Boolean separateHeartbeat) throws Exception { + @ValueSource(booleans = {true, false}) + public void testWithLoad(boolean separateHeartbeat) throws Exception { GrpcConfigKeys.Server.setHeartbeatChannel(getProperties(), separateHeartbeat); super.testWithLoad(); BlockRequestHandlingInjection.getInstance().unblockAll(); } @ParameterizedTest - @MethodSource("data") - public void testRequestTimeout(Boolean separateHeartbeat) throws Exception { + @ValueSource(booleans = {true, false}) + public void testRequestTimeout(boolean separateHeartbeat) throws Exception { GrpcConfigKeys.Server.setHeartbeatChannel(getProperties(), separateHeartbeat); runWithNewCluster(NUM_SERVERS, cluster -> testRequestTimeout(false, cluster, LOG)); } @ParameterizedTest - @MethodSource("data") - public void testUpdateViaHeartbeat(Boolean separateHeartbeat) throws Exception { + @ValueSource(booleans = {true, false}) + public void testUpdateViaHeartbeat(boolean separateHeartbeat) throws Exception { GrpcConfigKeys.Server.setHeartbeatChannel(getProperties(), separateHeartbeat); runWithNewCluster(NUM_SERVERS, this::runTestUpdateViaHeartbeat); } From e3633dd78f3641c937acaea6aa6e2376251b2684 Mon Sep 17 00:00:00 2001 From: "Doroszlai, Attila" <6454655+adoroszlai@users.noreply.github.com> Date: Tue, 24 Dec 2024 19:22:44 +0100 Subject: [PATCH 125/397] RATIS-2210. Reduce duplication in CI workflow (#1195) --- .github/workflows/check.yml | 203 +++++++++++++++++++++++ .github/workflows/ci.yml | 141 ++++++++++++++++ .github/workflows/post-commit.yml | 261 ++---------------------------- dev-support/checks/compile.sh | 31 ++++ 4 files changed, 385 insertions(+), 251 deletions(-) create mode 100644 .github/workflows/check.yml create mode 100644 .github/workflows/ci.yml create mode 100755 dev-support/checks/compile.sh diff --git a/.github/workflows/check.yml b/.github/workflows/check.yml new file mode 100644 index 0000000000..b4498927ae --- /dev/null +++ b/.github/workflows/check.yml @@ -0,0 +1,203 @@ +# 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. + +# This reusable workflow executes a single check from `dev-support/checks/`. +# Before and after the check, it performs various steps based on workflow inputs. + +name: ci-check + +on: + workflow_call: + inputs: + # REQUIRED + script: + type: string + description: "Test script to run from dev-support/checks, without .sh extension" + required: true + + # OPTIONAL (ordered alphabetically) + java-version: + type: string + description: "Java version to set up (default: 8)" + default: '8' + required: false + + needs-binary-tarball: + type: boolean + description: "Whether to download Ratis binary tarball created by build (default: no)" + default: false + required: false + + needs-maven-repo: + type: boolean + description: "Whether to download Ratis jars created by build (default: no)" + default: false + required: false + + needs-source-tarball: + type: boolean + description: "Whether to download Ratis source tarball created by build (default: no)" + default: false + required: false + + runner: + type: string + description: "GitHub Actions runner to use" + default: 'ubuntu-20.04' + required: false + + script-args: + type: string + description: "Arguments for the test script" + default: '' + required: false + + split: + type: string + description: "Name of split for matrix jobs, only used in display name" + default: '' + required: false + + timeout-minutes: + type: number + description: "Job timeout in minutes (default: 30)" + default: 30 + required: false + +env: + MAVEN_ARGS: --batch-mode --show-version + MAVEN_OPTS: -Dhttp.keepAlive=false -Dmaven.wagon.http.pool=false -Dmaven.wagon.http.retryHandler.class=standard -Dmaven.wagon.http.retryHandler.count=3 + WITH_COVERAGE: true + +jobs: + check: + name: ${{ (inputs.split && format('{0} ({1})', inputs.script, inputs.split)) || inputs.script }} + runs-on: ${{ inputs.runner }} + timeout-minutes: ${{ inputs.timeout-minutes }} + steps: + - name: Checkout project + if: ${{ !inputs.needs-source-tarball }} + uses: actions/checkout@v4 + + - name: Download source tarball + if: ${{ inputs.needs-source-tarball }} + uses: actions/download-artifact@v4 + with: + name: ratis-src + + - name: Extract source tarball + if: ${{ inputs.needs-source-tarball }} + run: | + tar --strip-components 1 -xzvf ratis*-src.tar.gz + + - name: Create cache for Maven dependencies + if: ${{ inputs.script == 'build' }} + uses: actions/cache@v4 + with: + path: | + ~/.m2/repository/*/*/* + !~/.m2/repository/org/apache/ratis + key: maven-repo-${{ hashFiles('**/pom.xml') }} + restore-keys: | + maven-repo- + + - name: Restore cache for Maven dependencies + if: ${{ inputs.script != 'build' }} + uses: actions/cache/restore@v4 + with: + path: | + ~/.m2/repository/*/*/* + !~/.m2/repository/org/apache/ratis + key: maven-repo-${{ hashFiles('**/pom.xml') }} + restore-keys: | + maven-repo- + + - name: Download Maven repo + id: download-maven-repo + if: ${{ inputs.needs-maven-repo }} + uses: actions/download-artifact@v4 + with: + name: maven-repo + path: | + ~/.m2/repository/org/apache/ratis + + - name: Download binary tarball + if: ${{ inputs.needs-binary-tarball }} + uses: actions/download-artifact@v4 + with: + name: ratis-bin + + - name: Extract binary tarball + if: ${{ inputs.needs-binary-tarball }} + run: | + mkdir -p ratis-assembly/target + tar xzvf ratis-*-bin.tar.gz -C ratis-assembly/target + + - name: Setup java ${{ inputs.java-version }} + if: ${{ inputs.java-version }} + uses: actions/setup-java@v4 + with: + distribution: 'temurin' + java-version: ${{ inputs.java-version }} + + - name: Execute tests + run: | + dev-support/checks/${{ inputs.script }}.sh ${{ inputs.script-args }} + env: + WITH_COVERAGE: ${{ inputs.with-coverage }} + + - name: Summary of failures + if: ${{ failure() }} + run: | + if [[ -s "target/${{ inputs.script }}/summary.txt" ]]; then + cat target/${{ inputs.script }}/summary.txt + fi + + - name: Archive build results + if: ${{ !cancelled() }} + uses: actions/upload-artifact@v4 + with: + name: ${{ (inputs.split && format('{0}-{1}', inputs.script, inputs.split)) || inputs.script }} + path: target/${{ inputs.script }} + continue-on-error: true + + # The following steps are hard-coded to be run only for 'build' check, + # to avoid the need for 3 more inputs. + - name: Store binaries for tests + if: ${{ inputs.script == 'build' && !cancelled() }} + uses: actions/upload-artifact@v4 + with: + name: ratis-bin + path: | + ratis-assembly/target/ratis-assembly-*-bin.tar.gz + retention-days: 1 + + - name: Store source tarball for compilation + if: ${{ inputs.script == 'build' && !cancelled() }} + uses: actions/upload-artifact@v4 + with: + name: ratis-src + path: | + ratis-assembly/target/ratis-assembly-*-src.tar.gz + retention-days: 1 + + - name: Store Maven repo for tests + if: ${{ inputs.script == 'build' && !cancelled() }} + uses: actions/upload-artifact@v4 + with: + name: maven-repo + path: | + ~/.m2/repository/org/apache/ratis + retention-days: 1 diff --git a/.github/workflows/ci.yml b/.github/workflows/ci.yml new file mode 100644 index 0000000000..11fb1bd515 --- /dev/null +++ b/.github/workflows/ci.yml @@ -0,0 +1,141 @@ +# Licensed to the Apache Software Foundation (ASF) under one or more +# contributor license agreements. See the NOTICE file distributed with +# this work for additional information regarding copyright ownership. +# The ASF licenses this file to You under the Apache License, Version 2.0 +# (the "License"); you may not use this file except in compliance with +# the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. + +name: CI + +on: + workflow_call: + inputs: + ref: + type: string + description: Ratis git ref (branch, tag or commit hash) + default: '' + required: false + +jobs: + build: + uses: ./.github/workflows/check.yml + with: + script: build + script-args: -Prelease + timeout-minutes: 30 + secrets: inherit + + compile: + needs: + - build + strategy: + matrix: + java: [ 11, 17, 21 ] + fail-fast: false + uses: ./.github/workflows/check.yml + with: + java-version: ${{ matrix.java }} + needs-source-tarball: true + script: compile + script-args: -Djavac.version=${{ matrix.java }} + split: ${{ matrix.java }} + timeout-minutes: 30 + secrets: inherit + + repro: + needs: + - build + uses: ./.github/workflows/check.yml + with: + needs-maven-repo: true + script: repro + script-args: -Prelease + timeout-minutes: 30 + secrets: inherit + + basic: + strategy: + matrix: + check: + - author + - checkstyle + - findbugs + - rat + fail-fast: false + uses: ./.github/workflows/check.yml + with: + script: ${{ matrix.check }} + timeout-minutes: 30 + secrets: inherit + + unit: + strategy: + matrix: + profile: + - grpc + - server + - misc + fail-fast: false + uses: ./.github/workflows/check.yml + with: + script: unit + script-args: -P${{ matrix.profile }}-tests + split: ${{ matrix.profile }} + timeout-minutes: 60 + secrets: inherit + + coverage: + needs: + - build + - unit + runs-on: ubuntu-20.04 + timeout-minutes: 30 + if: (github.repository == 'apache/ratis' || github.repository == 'apache/incubator-ratis') && github.event_name != 'pull_request' + steps: + - name: Checkout project + uses: actions/checkout@v4 + with: + fetch-depth: 0 + - name: Cache for maven dependencies + uses: actions/cache/restore@v4 + with: + path: | + ~/.m2/repository + !~/.m2/repository/org/apache/ratis + key: maven-repo-${{ hashFiles('**/pom.xml') }} + restore-keys: | + maven-repo- + - name: Setup java 17 + uses: actions/setup-java@v4 + with: + distribution: 'temurin' + java-version: 17 + - name: Download artifacts + uses: actions/download-artifact@v4 + with: + path: target/artifacts + - name: Untar binaries + run: | + mkdir -p ratis-assembly/target + tar xzvf target/artifacts/ratis-bin/ratis-assembly-*.tar.gz -C ratis-assembly/target + - name: Calculate combined coverage + run: ./dev-support/checks/coverage.sh + - name: Upload coverage to Sonar + run: ./dev-support/checks/sonar.sh + env: + SONAR_TOKEN: ${{ secrets.SONARCLOUD_TOKEN }} + GITHUB_TOKEN: ${{ secrets.GITHUB_TOKEN }} + - name: Archive build results + uses: actions/upload-artifact@v4 + if: always() + with: + name: ${{ github.job }} + path: target/${{ github.job }} diff --git a/.github/workflows/post-commit.yml b/.github/workflows/post-commit.yml index 5fee9462ae..9b0df4ce40 100644 --- a/.github/workflows/post-commit.yml +++ b/.github/workflows/post-commit.yml @@ -12,7 +12,9 @@ # 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. + name: build-branch + on: push: branches-ignore: @@ -20,255 +22,12 @@ on: tags: - '**' pull_request: -env: - WITH_COVERAGE: true + +concurrency: + group: ci-${{ github.event.pull_request.number || github.sha }} + cancel-in-progress: ${{ github.event_name == 'pull_request' }} + jobs: - build: - runs-on: ubuntu-20.04 - timeout-minutes: 30 - steps: - - name: Checkout project - uses: actions/checkout@v4 - - name: Cache for maven dependencies - uses: actions/cache@v4 - with: - path: | - ~/.m2/repository - !~/.m2/repository/org/apache/ratis - key: maven-repo-${{ hashFiles('**/pom.xml') }} - restore-keys: | - maven-repo- - - name: Setup java - uses: actions/setup-java@v4 - with: - distribution: 'temurin' - java-version: 8 - - name: Run a full build - run: ./dev-support/checks/build.sh -Prelease - - name: Store binaries for tests - uses: actions/upload-artifact@v4 - with: - name: ratis-bin - path: | - ratis-assembly/target/ratis-assembly-*.tar.gz - !ratis-assembly/target/ratis-assembly-*-src.tar.gz - retention-days: 1 - - name: Store source tarball for compilation - uses: actions/upload-artifact@v4 - with: - name: ratis-src - path: ratis-assembly/target/ratis-assembly-*-src.tar.gz - retention-days: 1 - compile: - needs: - - build - runs-on: ubuntu-20.04 - timeout-minutes: 30 - strategy: - matrix: - java: [ 11, 17, 21 ] - fail-fast: false - steps: - - name: Download source tarball - uses: actions/download-artifact@v4 - with: - name: ratis-src - - name: Untar sources - run: | - tar --strip-components 1 -xzvf ratis-assembly-*-src.tar.gz - - name: Cache for maven dependencies - uses: actions/cache/restore@v4 - with: - path: | - ~/.m2/repository - !~/.m2/repository/org/apache/ratis - key: maven-repo-${{ hashFiles('**/pom.xml') }} - restore-keys: | - maven-repo- - - name: Setup java - uses: actions/setup-java@v4 - with: - distribution: 'temurin' - java-version: ${{ matrix.java }} - - name: Run a full build - run: ./dev-support/checks/build.sh -Djavac.version=${{ matrix.java }} - - name: Test reproducibility - run: ./dev-support/checks/repro.sh -Djavac.version=${{ matrix.java }} - rat: - name: rat - runs-on: ubuntu-20.04 - timeout-minutes: 15 - steps: - - name: Checkout project - uses: actions/checkout@v4 - - name: Cache for maven dependencies - uses: actions/cache/restore@v4 - with: - path: | - ~/.m2/repository - !~/.m2/repository/org/apache/ratis - key: maven-repo-${{ hashFiles('**/pom.xml') }} - restore-keys: | - maven-repo- - - name: Run tests - run: ./dev-support/checks/rat.sh - - name: Upload results - uses: actions/upload-artifact@v4 - if: always() - with: - name: rat - path: target/rat - author: - name: author - runs-on: ubuntu-20.04 - timeout-minutes: 15 - steps: - - name: Checkout project - uses: actions/checkout@v4 - - name: Run tests - run: ./dev-support/checks/author.sh - - name: Upload results - uses: actions/upload-artifact@v4 - if: always() - with: - name: author - path: target/author - unit: - name: unit - runs-on: ubuntu-20.04 - timeout-minutes: 60 - strategy: - matrix: - profile: - - grpc - - server - - misc - fail-fast: false - steps: - - name: Checkout project - uses: actions/checkout@v4 - - name: Cache for maven dependencies - uses: actions/cache/restore@v4 - with: - path: | - ~/.m2/repository - !~/.m2/repository/org/apache/ratis - key: maven-repo-${{ hashFiles('**/pom.xml') }} - restore-keys: | - maven-repo- - - name: Setup java - uses: actions/setup-java@v4 - with: - distribution: 'temurin' - java-version: 8 - - name: Run tests - run: ./dev-support/checks/unit.sh -P${{ matrix.profile }}-tests - - name: Summary of failures - run: cat target/${{ github.job }}/summary.txt - if: ${{ !cancelled() }} - - name: Upload results - uses: actions/upload-artifact@v4 - if: ${{ !cancelled() }} - with: - name: unit-${{ matrix.profile }} - path: target/unit - checkstyle: - name: checkstyle - runs-on: ubuntu-20.04 - timeout-minutes: 15 - steps: - - name: Checkout project - uses: actions/checkout@v4 - - name: Cache for maven dependencies - uses: actions/cache/restore@v4 - with: - path: | - ~/.m2/repository - !~/.m2/repository/org/apache/ratis - key: maven-repo-${{ hashFiles('**/pom.xml') }} - restore-keys: | - maven-repo- - - name: Run tests - run: ./dev-support/checks/checkstyle.sh - - name: Upload results - uses: actions/upload-artifact@v4 - if: always() - with: - name: checkstyle - path: target/checkstyle - findbugs: - name: findbugs - runs-on: ubuntu-20.04 - timeout-minutes: 30 - steps: - - name: Setup java - uses: actions/setup-java@v4 - with: - distribution: 'temurin' - java-version: 8 - - name: Checkout project - uses: actions/checkout@v4 - - name: Cache for maven dependencies - uses: actions/cache/restore@v4 - with: - path: | - ~/.m2/repository - !~/.m2/repository/org/apache/ratis - key: maven-repo-${{ hashFiles('**/pom.xml') }} - restore-keys: | - maven-repo- - - name: Run tests - run: ./dev-support/checks/findbugs.sh - - name: Upload results - uses: actions/upload-artifact@v4 - if: always() - with: - name: findbugs - path: target/findbugs - coverage: - needs: - - build - - unit - runs-on: ubuntu-20.04 - timeout-minutes: 30 - if: (github.repository == 'apache/ratis' || github.repository == 'apache/incubator-ratis') && github.event_name != 'pull_request' - steps: - - name: Checkout project - uses: actions/checkout@v4 - with: - fetch-depth: 0 - - name: Cache for maven dependencies - uses: actions/cache/restore@v4 - with: - path: | - ~/.m2/repository - !~/.m2/repository/org/apache/ratis - key: maven-repo-${{ hashFiles('**/pom.xml') }} - restore-keys: | - maven-repo- - - name: Setup java 17 - uses: actions/setup-java@v4 - with: - distribution: 'temurin' - java-version: 17 - - name: Download artifacts - uses: actions/download-artifact@v4 - with: - path: target/artifacts - - name: Untar binaries - run: | - mkdir -p ratis-assembly/target - tar xzvf target/artifacts/ratis-bin/ratis-assembly-*.tar.gz -C ratis-assembly/target - - name: Calculate combined coverage - run: ./dev-support/checks/coverage.sh - - name: Upload coverage to Sonar - run: ./dev-support/checks/sonar.sh - env: - SONAR_TOKEN: ${{ secrets.SONARCLOUD_TOKEN }} - GITHUB_TOKEN: ${{ secrets.GITHUB_TOKEN }} - - name: Archive build results - uses: actions/upload-artifact@v4 - if: always() - with: - name: ${{ github.job }} - path: target/${{ github.job }} + CI: + uses: ./.github/workflows/ci.yml + secrets: inherit diff --git a/dev-support/checks/compile.sh b/dev-support/checks/compile.sh new file mode 100755 index 0000000000..fba4394dec --- /dev/null +++ b/dev-support/checks/compile.sh @@ -0,0 +1,31 @@ +#!/usr/bin/env bash +# 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. +DIR="$( cd "$( dirname "${BASH_SOURCE[0]}" )" >/dev/null 2>&1 && pwd )" +cd "$DIR/../.." || exit 1 + +source "${DIR}/../find_maven.sh" + +: ${WITH_COVERAGE:="false"} + +MAVEN_OPTIONS='-V -B -Dmaven.javadoc.skip=true -DskipTests' + +if [[ "${WITH_COVERAGE}" != "true" ]]; then + MAVEN_OPTIONS="${MAVEN_OPTIONS} -Djacoco.skip" +fi + +export MAVEN_OPTS="-Xmx4096m" +${MVN} ${MAVEN_OPTIONS} clean verify "$@" +exit $? From 0a40f06da4b6ccac4b8efecb9362430c090c0dfb Mon Sep 17 00:00:00 2001 From: "Doroszlai, Attila" <6454655+adoroszlai@users.noreply.github.com> Date: Tue, 24 Dec 2024 21:30:26 +0100 Subject: [PATCH 126/397] RATIS-2226. Enable Develocity local build cache (#1196) --- .gitignore | 1 + .mvn/develocity.xml | 44 ++++++++++++++++++++++++++++++++++++++++++++ .mvn/extensions.xml | 34 ++++++++++++++++++++++++++++++++++ 3 files changed, 79 insertions(+) create mode 100644 .mvn/develocity.xml create mode 100644 .mvn/extensions.xml diff --git a/.gitignore b/.gitignore index 9379453102..ecaf1f2df9 100644 --- a/.gitignore +++ b/.gitignore @@ -11,6 +11,7 @@ .hugo_build.lock .idea .classpath +.mvn/.develocity/ .project .settings target diff --git a/.mvn/develocity.xml b/.mvn/develocity.xml new file mode 100644 index 0000000000..1fa63e6f21 --- /dev/null +++ b/.mvn/develocity.xml @@ -0,0 +1,44 @@ + + + + + + false + false + false + false + + + false + + + + + #{isFalse(env['GITHUB_ACTIONS'])} + + + false + + + diff --git a/.mvn/extensions.xml b/.mvn/extensions.xml new file mode 100644 index 0000000000..549a1cddcd --- /dev/null +++ b/.mvn/extensions.xml @@ -0,0 +1,34 @@ + + + + + com.gradle + develocity-maven-extension + 1.23 + + + com.gradle + common-custom-user-data-maven-extension + 2.0.1 + + From e607a95145205fbabddc6c7bf9feb3c591b45ef8 Mon Sep 17 00:00:00 2001 From: Symious <14933944+symious@users.noreply.github.com> Date: Thu, 26 Dec 2024 02:57:36 +0800 Subject: [PATCH 127/397] RATIS-2197. Clean remote stream to resolve direct memory leak (#1179) --- .../client/impl/DataStreamClientImpl.java | 3 ++ .../netty/server/DataStreamManagement.java | 35 ++++++++++++------- 2 files changed, 26 insertions(+), 12 deletions(-) diff --git a/ratis-client/src/main/java/org/apache/ratis/client/impl/DataStreamClientImpl.java b/ratis-client/src/main/java/org/apache/ratis/client/impl/DataStreamClientImpl.java index ba91866d71..313131cbda 100644 --- a/ratis-client/src/main/java/org/apache/ratis/client/impl/DataStreamClientImpl.java +++ b/ratis-client/src/main/java/org/apache/ratis/client/impl/DataStreamClientImpl.java @@ -153,6 +153,9 @@ private CompletableFuture combineHeader(CompletableFuture writeAsyncImpl(Object data, long length, Iterable options) { if (isClosed()) { + if (data instanceof ByteBuf) { + ((ByteBuf) data).release(); + } return JavaUtils.completeExceptionally(new AlreadyClosedException( clientId + ": stream already closed, request=" + header)); } diff --git a/ratis-netty/src/main/java/org/apache/ratis/netty/server/DataStreamManagement.java b/ratis-netty/src/main/java/org/apache/ratis/netty/server/DataStreamManagement.java index 74d5cd7fde..7110631858 100644 --- a/ratis-netty/src/main/java/org/apache/ratis/netty/server/DataStreamManagement.java +++ b/ratis-netty/src/main/java/org/apache/ratis/netty/server/DataStreamManagement.java @@ -211,6 +211,12 @@ private Set getSuccessors(RaftPeerId peerId) { return Collections.emptySet(); } + + void cleanUp(ClientInvocationId invocationId) { + getDivision().getDataStreamMap().remove(invocationId); + getLocal().cleanUp(); + applyToRemotes(remote -> remote.out.closeAsync()); + } } private final RaftServer server; @@ -301,6 +307,9 @@ static long writeTo(ByteBuf buf, Iterable options, final DataChannel channel = stream.getDataChannel(); long byteWritten = 0; for (ByteBuffer buffer : buf.nioBuffers()) { + if (buffer.remaining() == 0) { + continue; + } final ReferenceCountedObject wrapped = ReferenceCountedObject.wrap( buffer, buf::retain, ignored -> buf.release()); try(UncheckedAutoCloseable ignore = wrapped.retainAndReleaseOnClose()) { @@ -389,9 +398,7 @@ static void sendDataStreamException(Throwable throwable, DataStreamRequestByteBu void cleanUp(Set ids) { for (ClientInvocationId clientInvocationId : ids) { - Optional.ofNullable(streams.remove(clientInvocationId)) - .map(StreamInfo::getLocal) - .ifPresent(LocalStream::cleanUp); + removeDataStream(clientInvocationId); } } @@ -411,19 +418,16 @@ void read(DataStreamRequestByteBuf request, ChannelHandlerContext ctx, readImpl(request, ctx, getStreams); } catch (Throwable t) { replyDataStreamException(t, request, ctx); - removeDataStream(ClientInvocationId.valueOf(request.getClientId(), request.getStreamId()), null); + removeDataStream(ClientInvocationId.valueOf(request.getClientId(), request.getStreamId())); } } - private void removeDataStream(ClientInvocationId invocationId, StreamInfo info) { + private StreamInfo removeDataStream(ClientInvocationId invocationId) { final StreamInfo removed = streams.remove(invocationId); - if (info == null) { - info = removed; - } - if (info != null) { - info.getDivision().getDataStreamMap().remove(invocationId); - info.getLocal().cleanUp(); + if (removed != null) { + removed.cleanUp(invocationId); } + return removed; } private void readImpl(DataStreamRequestByteBuf request, ChannelHandlerContext ctx, @@ -479,7 +483,14 @@ private void readImpl(DataStreamRequestByteBuf request, ChannelHandlerContext ct try { if (exception != null) { replyDataStreamException(server, exception, info.getRequest(), request, ctx); - removeDataStream(key, info); + final StreamInfo removed = removeDataStream(key); + if (removed != null) { + Preconditions.assertSame(info, removed, "removed"); + } else { + info.cleanUp(key); + } + } else if (close) { + info.applyToRemotes(remote -> remote.out.closeAsync()); } } finally { request.release(); From b3818664debfba95e9c75a9093b8f137d3372b15 Mon Sep 17 00:00:00 2001 From: kiruthiga1793 <64447332+kiruthiga1793@users.noreply.github.com> Date: Thu, 26 Dec 2024 02:29:38 -0500 Subject: [PATCH 128/397] RATIS-2200. Make Ratis example log at INFO level (#1189) --- ratis-examples/src/main/resources/log4j.properties | 4 +--- 1 file changed, 1 insertion(+), 3 deletions(-) diff --git a/ratis-examples/src/main/resources/log4j.properties b/ratis-examples/src/main/resources/log4j.properties index f5b4baf67c..eedefe287e 100644 --- a/ratis-examples/src/main/resources/log4j.properties +++ b/ratis-examples/src/main/resources/log4j.properties @@ -16,10 +16,8 @@ log4j.rootLogger=INFO, stdout -log4j.logger.org.apache.ratis.server.impl.RatisServerImpl=DEBUG -log4j.logger.org.apache.ratis.client.RaftClient=DEBUG log4j.appender.stdout=org.apache.log4j.ConsoleAppender log4j.appender.stdout.Target=System.out log4j.appender.stdout.layout=org.apache.log4j.PatternLayout -log4j.appender.stdout.layout.ConversionPattern=%d{yyyy-MM-dd HH:mm:ss} %-5p %c{1}:%L - %m%n \ No newline at end of file +log4j.appender.stdout.layout.ConversionPattern=%d{yyyy-MM-dd HH:mm:ss} %-5p %c{1}:%L - %m%n From d7f29a3fb0af9eabedbe1a2bbafc42ad58a5c6a5 Mon Sep 17 00:00:00 2001 From: Potato Date: Fri, 27 Dec 2024 21:05:46 +0800 Subject: [PATCH 129/397] RATIS-2231. Bump ratis-thirdparty to 1.0.8 (#1202) --- pom.xml | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/pom.xml b/pom.xml index ac3d237f1d..9ccba3d598 100644 --- a/pom.xml +++ b/pom.xml @@ -206,11 +206,11 @@ 3.3.9 - 1.0.7 + 1.0.8 - 3.24.4 - 1.58.0 + 3.25.5 + 1.69.0 true From e92e1f2aa1a465311cfd4b3c4e447a3fd20df286 Mon Sep 17 00:00:00 2001 From: William Song <48054931+SzyWilliam@users.noreply.github.com> Date: Fri, 3 Jan 2025 03:03:12 +0800 Subject: [PATCH 130/397] RATIS-2232. Improve log for debugging on RaftLog / TransactionManager (#1203) --- .../apache/ratis/server/raftlog/RaftLogSequentialOps.java | 4 +++- .../org/apache/ratis/server/impl/TransactionManager.java | 8 +------- 2 files changed, 4 insertions(+), 8 deletions(-) diff --git a/ratis-server-api/src/main/java/org/apache/ratis/server/raftlog/RaftLogSequentialOps.java b/ratis-server-api/src/main/java/org/apache/ratis/server/raftlog/RaftLogSequentialOps.java index 7b9f42b6bd..5e8bd6d784 100644 --- a/ratis-server-api/src/main/java/org/apache/ratis/server/raftlog/RaftLogSequentialOps.java +++ b/ratis-server-api/src/main/java/org/apache/ratis/server/raftlog/RaftLogSequentialOps.java @@ -77,8 +77,10 @@ OUTPUT runSequentially( // The current thread is already the runner. return operation.get(); } else { + final Throwable cause = new Throwable("The thread already running: " + previous); + cause.setStackTrace(previous.getStackTrace()); throw new IllegalStateException( - name + ": Already running a method by " + previous + ", current=" + current); + name + ": Already running a method by " + previous + ", current=" + current, cause); } } } diff --git a/ratis-server/src/main/java/org/apache/ratis/server/impl/TransactionManager.java b/ratis-server/src/main/java/org/apache/ratis/server/impl/TransactionManager.java index c33bc26bce..cba310b34d 100644 --- a/ratis-server/src/main/java/org/apache/ratis/server/impl/TransactionManager.java +++ b/ratis-server/src/main/java/org/apache/ratis/server/impl/TransactionManager.java @@ -71,12 +71,6 @@ void remove(TermIndex termIndex) { @Override public String toString() { - if (contexts.isEmpty()) { - return name + " "; - } - - final StringBuilder b = new StringBuilder(name); - contexts.forEach((k, v) -> b.append("\n ").append(k).append(": initialized? ").append(v.isInitialized())); - return b.toString(); + return name + ":size=" + contexts.size(); } } \ No newline at end of file From b5db2ca1ad1238c9a5b73feaf25e667036e1fc99 Mon Sep 17 00:00:00 2001 From: William Song <48054931+SzyWilliam@users.noreply.github.com> Date: Fri, 3 Jan 2025 03:23:19 +0800 Subject: [PATCH 131/397] RATIS-2234. Remove lock race between heartbeat and append log channels (#1205) --- .../java/org/apache/ratis/util/AutoCloseableLock.java | 8 ++++++++ .../org/apache/ratis/server/raftlog/RaftLogBase.java | 9 ++++++++- 2 files changed, 16 insertions(+), 1 deletion(-) diff --git a/ratis-common/src/main/java/org/apache/ratis/util/AutoCloseableLock.java b/ratis-common/src/main/java/org/apache/ratis/util/AutoCloseableLock.java index 8a5409bafe..9581e925a5 100644 --- a/ratis-common/src/main/java/org/apache/ratis/util/AutoCloseableLock.java +++ b/ratis-common/src/main/java/org/apache/ratis/util/AutoCloseableLock.java @@ -17,6 +17,7 @@ */ package org.apache.ratis.util; +import java.util.Objects; import java.util.concurrent.atomic.AtomicBoolean; import java.util.concurrent.locks.Lock; @@ -45,6 +46,13 @@ public static AutoCloseableLock acquire(final Lock lock, Runnable preUnlock) { return new AutoCloseableLock(lock, preUnlock); } + public static AutoCloseableLock tryAcquire(final Lock lock, Runnable preUnlock, TimeDuration timeout) + throws InterruptedException { + Objects.requireNonNull(timeout, "timeout == null"); + final boolean locked = lock.tryLock(timeout.getDuration(), timeout.getUnit()); + return locked? new AutoCloseableLock(lock, preUnlock): null; + } + private final Lock underlying; private final AtomicBoolean closed = new AtomicBoolean(false); private final Runnable preUnlock; diff --git a/ratis-server/src/main/java/org/apache/ratis/server/raftlog/RaftLogBase.java b/ratis-server/src/main/java/org/apache/ratis/server/raftlog/RaftLogBase.java index 9b0367213a..b37d40c28a 100644 --- a/ratis-server/src/main/java/org/apache/ratis/server/raftlog/RaftLogBase.java +++ b/ratis-server/src/main/java/org/apache/ratis/server/raftlog/RaftLogBase.java @@ -119,7 +119,7 @@ public boolean isOpened() { @Override public boolean updateCommitIndex(long majorityIndex, long currentTerm, boolean isLeader) { - try(AutoCloseableLock writeLock = writeLock()) { + try(AutoCloseableLock writeLock = tryWriteLock(TimeDuration.ONE_SECOND)) { final long oldCommittedIndex = getLastCommittedIndex(); final long newCommitIndex = Math.min(majorityIndex, getFlushIndex()); if (oldCommittedIndex < newCommitIndex) { @@ -133,6 +133,9 @@ public boolean updateCommitIndex(long majorityIndex, long currentTerm, boolean i return commitIndex.updateIncreasingly(newCommitIndex, traceIndexChange); } } + } catch (InterruptedException e) { + LOG.warn("{}: Interrupted to updateCommitIndex: majorityIndex={}, currentTerm={}, isLeader={}", + getName(), majorityIndex, currentTerm, isLeader, e); } return false; } @@ -375,6 +378,10 @@ public AutoCloseableLock writeLock() { return AutoCloseableLock.acquire(lock.writeLock()); } + public AutoCloseableLock tryWriteLock(TimeDuration timeout) throws InterruptedException { + return AutoCloseableLock.tryAcquire(lock.writeLock(), null, timeout); + } + public boolean hasWriteLock() { return this.lock.isWriteLockedByCurrentThread(); } From 90100c6e621316dc55362a79ccb1fdcaa3554b5f Mon Sep 17 00:00:00 2001 From: Tsz-Wo Nicholas Sze Date: Fri, 3 Jan 2025 09:55:48 -0800 Subject: [PATCH 132/397] RATIS-2229. Do not print the same conf values multiple times. (#1200) --- .../java/org/apache/ratis/conf/ConfUtils.java | 18 ++++++++++- .../ratis/server/simulation/SimulatedRpc.java | 6 +++- .../org/apache/ratis/conf/TestConfUtils.java | 32 +++++++++++++++++++ 3 files changed, 54 insertions(+), 2 deletions(-) diff --git a/ratis-common/src/main/java/org/apache/ratis/conf/ConfUtils.java b/ratis-common/src/main/java/org/apache/ratis/conf/ConfUtils.java index c1fb9268c9..3f7678a0bc 100644 --- a/ratis-common/src/main/java/org/apache/ratis/conf/ConfUtils.java +++ b/ratis-common/src/main/java/org/apache/ratis/conf/ConfUtils.java @@ -33,6 +33,8 @@ import java.net.InetSocketAddress; import java.util.Arrays; import java.util.List; +import java.util.concurrent.ConcurrentHashMap; +import java.util.concurrent.ConcurrentMap; import java.util.function.BiConsumer; import java.util.function.BiFunction; import java.util.function.Consumer; @@ -41,8 +43,22 @@ public interface ConfUtils { Logger LOG = LoggerFactory.getLogger(ConfUtils.class); + class Utils { + private static final ConcurrentMap CACHE = new ConcurrentHashMap<>(); + + private static boolean isNew(String key, T value) { + if (value == null) { + final Object previous = CACHE.remove(key); + return previous != null; + } else { + final Object previous = CACHE.put(key, value); + return !value.equals(previous); + } + } + } + static void logGet(String key, T value, T defaultValue, Consumer logger) { - if (logger != null) { + if (logger != null && Utils.isNew(key, value)) { logger.accept(String.format("%s = %s (%s)", key, value, Objects.equal(value, defaultValue)? "default": "custom")); } diff --git a/ratis-server/src/test/java/org/apache/ratis/server/simulation/SimulatedRpc.java b/ratis-server/src/test/java/org/apache/ratis/server/simulation/SimulatedRpc.java index 0399b414bf..e570c35af0 100644 --- a/ratis-server/src/test/java/org/apache/ratis/server/simulation/SimulatedRpc.java +++ b/ratis-server/src/test/java/org/apache/ratis/server/simulation/SimulatedRpc.java @@ -28,9 +28,13 @@ import java.util.Objects; -class SimulatedRpc implements RpcType { +public class SimulatedRpc implements RpcType { static final SimulatedRpc INSTANCE = new SimulatedRpc(); + public static SimulatedRpc get() { + return INSTANCE; + } + @Override public String name() { return getClass().getName(); diff --git a/ratis-test/src/test/java/org/apache/ratis/conf/TestConfUtils.java b/ratis-test/src/test/java/org/apache/ratis/conf/TestConfUtils.java index 67c02cd5f6..859c597b52 100644 --- a/ratis-test/src/test/java/org/apache/ratis/conf/TestConfUtils.java +++ b/ratis-test/src/test/java/org/apache/ratis/conf/TestConfUtils.java @@ -22,10 +22,42 @@ import org.apache.ratis.client.RaftClientConfigKeys; import org.apache.ratis.grpc.GrpcConfigKeys; import org.apache.ratis.netty.NettyConfigKeys; +import org.apache.ratis.rpc.RpcType; import org.apache.ratis.server.RaftServerConfigKeys; +import org.apache.ratis.server.simulation.SimulatedRpc; +import org.junit.jupiter.api.Assertions; import org.junit.jupiter.api.Test; +import java.util.concurrent.atomic.AtomicInteger; +import java.util.function.Consumer; + public class TestConfUtils extends BaseTest { + @Test + public void testLogging() { + final AtomicInteger count = new AtomicInteger(); + final Consumer logger = s -> { + System.out.println("log: " + s); + count.incrementAndGet(); + }; + + final RaftProperties properties = new RaftProperties(); + final RpcType simulated = SimulatedRpc.get(); + + // get a value the first time + final RpcType defaultType = RaftConfigKeys.Rpc.type(properties, logger); + Assertions.assertEquals(1, count.get()); + Assertions.assertNotEquals(defaultType, simulated); + + // get the same value the second time + RaftConfigKeys.Rpc.type(properties, logger); + Assertions.assertEquals(1, count.get()); + + // get a different value + RaftConfigKeys.Rpc.setType(properties, SimulatedRpc.get()); + RaftConfigKeys.Rpc.type(properties, logger); + Assertions.assertEquals(2, count.get()); + } + @Test public void testRaftConfigKeys() { ConfUtils.printAll(RaftConfigKeys.class); From 02cc17b0aefce2a06cd87dda367bbb8519088256 Mon Sep 17 00:00:00 2001 From: William Song <48054931+SzyWilliam@users.noreply.github.com> Date: Sat, 4 Jan 2025 01:58:36 +0800 Subject: [PATCH 133/397] RATIS-2233. make NOPROGRESS timeout configurable (#1204) --- ratis-docs/src/site/markdown/configurations.md | 10 ++++++++++ .../apache/ratis/server/RaftServerConfigKeys.java | 15 +++++++++++++++ .../apache/ratis/server/impl/LeaderStateImpl.java | 4 +++- 3 files changed, 28 insertions(+), 1 deletion(-) diff --git a/ratis-docs/src/site/markdown/configurations.md b/ratis-docs/src/site/markdown/configurations.md index 71eae7d3d1..4caf11b30e 100644 --- a/ratis-docs/src/site/markdown/configurations.md +++ b/ratis-docs/src/site/markdown/configurations.md @@ -104,6 +104,16 @@ When bootstrapping a new peer, If the gap between the match index of the peer and the leader's latest committed index is less than this gap, we treat the peer as caught-up. Increase this number when write throughput is high. +--------------------------------------------------------------------------------- +| **Property** | `raft.server.staging.timeout` | +|:----------------|:-----------------------------------------| +| **Description** | timeout of bootstrapping a new peer | +| **Type** | TimeDuration | +| **Default** | 3 times of `raft.server.rpc.timeout.max` | + +During the initialization of a new peer, the leader will classify the bootstrap process as "NO PROGRESS" +if it fails to receive any RPC responses from this peer within this specified timeout period. + --------------------------------------------------------------------------------- ### ThreadPool - Configurations related to server thread pools. diff --git a/ratis-server-api/src/main/java/org/apache/ratis/server/RaftServerConfigKeys.java b/ratis-server-api/src/main/java/org/apache/ratis/server/RaftServerConfigKeys.java index 8e1d92b754..849597433a 100644 --- a/ratis-server-api/src/main/java/org/apache/ratis/server/RaftServerConfigKeys.java +++ b/ratis-server-api/src/main/java/org/apache/ratis/server/RaftServerConfigKeys.java @@ -116,6 +116,21 @@ static void setStagingCatchupGap(RaftProperties properties, int stagingCatchupGa setInt(properties::setInt, STAGING_CATCHUP_GAP_KEY, stagingCatchupGap); } + String STAGING_TIMEOUT_KEY = PREFIX + ".staging.timeout"; + + TimeDuration STAGING_TIMEOUT_DEFAULT = null; + + static TimeDuration stagingTimeout(RaftProperties properties) { + final TimeDuration fallbackStagingTimeout = Rpc.timeoutMax(properties, null).multiply(3); + return getTimeDuration(properties.getTimeDuration(fallbackStagingTimeout.getUnit()), + STAGING_TIMEOUT_KEY, STAGING_TIMEOUT_DEFAULT, + Rpc.TIMEOUT_MAX_KEY, fallbackStagingTimeout, getDefaultLog()); + } + static void setStagingTimeout(RaftProperties properties, TimeDuration stagingTimeout) { + setTimeDuration(properties::setTimeDuration, STAGING_TIMEOUT_KEY, stagingTimeout); + } + + interface ThreadPool { String PREFIX = RaftServerConfigKeys.PREFIX + ".threadpool"; diff --git a/ratis-server/src/main/java/org/apache/ratis/server/impl/LeaderStateImpl.java b/ratis-server/src/main/java/org/apache/ratis/server/impl/LeaderStateImpl.java index 0dfbf263df..5870f51a68 100644 --- a/ratis-server/src/main/java/org/apache/ratis/server/impl/LeaderStateImpl.java +++ b/ratis-server/src/main/java/org/apache/ratis/server/impl/LeaderStateImpl.java @@ -350,6 +350,7 @@ boolean isApplied() { private final boolean logMetadataEnabled; private final int stagingCatchupGap; + private final TimeDuration stagingTimeout; private final RaftServerMetricsImpl raftServerMetrics; private final LogAppenderMetrics logAppenderMetrics; private final long followerMaxGapThreshold; @@ -364,6 +365,7 @@ boolean isApplied() { final RaftProperties properties = server.getRaftServer().getProperties(); stagingCatchupGap = RaftServerConfigKeys.stagingCatchupGap(properties); + stagingTimeout = RaftServerConfigKeys.stagingTimeout(properties); final ServerState state = server.getState(); this.raftLog = state.getLog(); @@ -789,7 +791,7 @@ public void run() { private BootStrapProgress checkProgress(FollowerInfo follower, long committed) { Preconditions.assertTrue(!isCaughtUp(follower)); final Timestamp progressTime = Timestamp.currentTime().addTimeMs(-server.getMaxTimeoutMs()); - final Timestamp timeoutTime = Timestamp.currentTime().addTimeMs(-3L * server.getMaxTimeoutMs()); + final Timestamp timeoutTime = Timestamp.currentTime().addTimeMs(-stagingTimeout.toLong(TimeUnit.MILLISECONDS)); if (follower.getLastRpcResponseTime().compareTo(timeoutTime) < 0) { LOG.debug("{} detects a follower {} timeout ({}ms) for bootstrapping", this, follower, follower.getLastRpcResponseTime().elapsedTimeMs()); From fd7d4d141bfcd1adc6a4c6bd464e9c11a937c3af Mon Sep 17 00:00:00 2001 From: William Song <48054931+SzyWilliam@users.noreply.github.com> Date: Tue, 7 Jan 2025 02:06:08 +0800 Subject: [PATCH 134/397] RATIS-2235. Allow only one thread to perform appendLog (#1206) --- .../apache/ratis/server/impl/RaftServerImpl.java | 14 +++++++++++--- 1 file changed, 11 insertions(+), 3 deletions(-) diff --git a/ratis-server/src/main/java/org/apache/ratis/server/impl/RaftServerImpl.java b/ratis-server/src/main/java/org/apache/ratis/server/impl/RaftServerImpl.java index 3ffcee0797..14b09a0235 100644 --- a/ratis-server/src/main/java/org/apache/ratis/server/impl/RaftServerImpl.java +++ b/ratis-server/src/main/java/org/apache/ratis/server/impl/RaftServerImpl.java @@ -132,6 +132,7 @@ import java.util.concurrent.ThreadLocalRandom; import java.util.concurrent.TimeUnit; import java.util.concurrent.atomic.AtomicBoolean; +import java.util.concurrent.atomic.AtomicReference; import java.util.function.Function; import java.util.function.Supplier; import java.util.stream.Collectors; @@ -249,6 +250,8 @@ public long[] getFollowerNextIndices() { private final AtomicBoolean firstElectionSinceStartup = new AtomicBoolean(true); private final ThreadGroup threadGroup; + private final AtomicReference> appendLogFuture; + RaftServerImpl(RaftGroup group, StateMachine stateMachine, RaftServerProxy proxy, RaftStorage.StartupOption option) throws IOException { final RaftPeerId id = proxy.getId(); @@ -282,6 +285,7 @@ public long[] getFollowerNextIndices() { this.transferLeadership = new TransferLeadership(this, properties); this.snapshotRequestHandler = new SnapshotManagementRequestHandler(this); this.snapshotInstallationHandler = new SnapshotInstallationHandler(this, properties); + this.appendLogFuture = new AtomicReference<>(CompletableFuture.completedFuture(null)); this.serverExecutor = ConcurrentUtils.newThreadPoolWithMax( RaftServerConfigKeys.ThreadPool.serverCached(properties), @@ -1585,9 +1589,9 @@ leaderId, getMemberId(), currentTerm, followerCommit, inconsistencyReplyNextInde state.updateConfiguration(entries); } future.join(); + final CompletableFuture appendLog = entries.isEmpty()? CompletableFuture.completedFuture(null) + : appendLog(entries); - final List> futures = entries.isEmpty() ? Collections.emptyList() - : state.getLog().append(entries); proto.getCommitInfosList().forEach(commitInfoCache::update); CodeInjectionForTesting.execute(LOG_SYNC, getId(), null); @@ -1601,7 +1605,7 @@ leaderId, getMemberId(), currentTerm, followerCommit, inconsistencyReplyNextInde final long commitIndex = effectiveCommitIndex(proto.getLeaderCommit(), previous, entries.size()); final long matchIndex = isHeartbeat? RaftLog.INVALID_LOG_INDEX: entries.get(entries.size() - 1).getIndex(); - return JavaUtils.allOf(futures).whenCompleteAsync((r, t) -> { + return appendLog.whenCompleteAsync((r, t) -> { followerState.ifPresent(fs -> fs.updateLastRpcTime(FollowerState.UpdateType.APPEND_COMPLETE)); timer.stop(); }, getServerExecutor()).thenApply(v -> { @@ -1618,6 +1622,10 @@ leaderId, getMemberId(), currentTerm, followerCommit, inconsistencyReplyNextInde return reply; }); } + private CompletableFuture appendLog(List entries) { + return appendLogFuture.updateAndGet(f -> f.thenCompose( + ignored -> JavaUtils.allOf(state.getLog().append(entries)))); + } private long checkInconsistentAppendEntries(TermIndex previous, List entries) { // Check if a snapshot installation through state machine is in progress. From 285c81b5187c3db5014f9b2d35299564a1c51575 Mon Sep 17 00:00:00 2001 From: "Doroszlai, Attila" <6454655+adoroszlai@users.noreply.github.com> Date: Wed, 8 Jan 2025 05:19:40 +0100 Subject: [PATCH 135/397] RATIS-2215. Bump maven-remote-resources-plugin to 3.3.0 (#1208) --- pom.xml | 6 ++++ .../src/main/resources/META-INF/LICENSE.vm | 32 +++++++++---------- .../src/main/resources/META-INF/NOTICE.vm | 24 +++++++------- 3 files changed, 34 insertions(+), 28 deletions(-) diff --git a/pom.xml b/pom.xml index 9ccba3d598..d7019f0dfb 100644 --- a/pom.xml +++ b/pom.xml @@ -172,6 +172,7 @@ 3.3.0 4.0.6 1.6.1 + 3.3.0 3.6.0 3.0.0-M4 3.5.3 @@ -699,6 +700,11 @@ org.apache.maven.plugins maven-antrun-plugin + + org.apache.maven.plugins + maven-remote-resources-plugin + ${maven-remote-resources-plugin.version} + org.apache.maven.plugins maven-site-plugin diff --git a/ratis-resource-bundle/src/main/resources/META-INF/LICENSE.vm b/ratis-resource-bundle/src/main/resources/META-INF/LICENSE.vm index 3d9f76a119..874d716c8b 100644 --- a/ratis-resource-bundle/src/main/resources/META-INF/LICENSE.vm +++ b/ratis-resource-bundle/src/main/resources/META-INF/LICENSE.vm @@ -221,10 +221,10 @@ under the License. limitations under the License. ## Special cases, for e.g. ASL2.0 licensed works that bundle additional third party works -#set($bundled-jquery = ${bundled-jquery.equalsIgnoreCase("true")}) -#set($bundled-logo = ${bundled-logo.equalsIgnoreCase("true")}) -#set($bundled-dependencies = ${bundled-dependencies.equalsIgnoreCase("true")}) -#if($bundled-jquery || $bundled-logo || $bundled-dependencies) +#set($bundled_jquery = ${bundled_jquery.equalsIgnoreCase("true")}) +#set($bundled_logo = ${bundled_logo.equalsIgnoreCase("true")}) +#set($bundled_dependencies = ${bundled_dependencies.equalsIgnoreCase("true")}) +#if($bundled_jquery || $bundled_logo || $bundled_dependencies) ==== ${project.name} contained works @@ -884,12 +884,12 @@ facade for Java, which can be obtained at: * HOMEPAGE: * http://www.slf4j.org/ #end -## Supplemental from commons-math +## Supplemental from commons_math #macro(commons_math_license) ---- APACHE COMMONS MATH DERIVATIVE WORKS: -The Apache commons-math library includes a number of subcomponents +The Apache commons_math library includes a number of subcomponents whose implementation is derived from original sources written in C or Fortran. License terms of the original sources are reproduced below. @@ -1178,7 +1178,7 @@ For the org.apache.hadoop.util.bloom.* classes: ## skip jquery ## skip backbone ## relocated jackson 2.4.0 is ASLv2 with no notice -## relocated commons-logging 1.1.1 is in NOTICE.vm +## relocated commons_logging 1.1.1 is in NOTICE.vm #end #macro (thrift_license) ## Thrift supplemental for libthrift is a no-op. @@ -1295,15 +1295,15 @@ You can redistribute it and/or modify it under either the terms of the PURPOSE. #end ## modules with bundled works in source -#if(${bundled-jquery}) +#if(${bundled_jquery}) #jquery_license() #end -#if(${bundled-logo}) +#if(${bundled_logo}) #orca_logo_license() #end ## when true, we're in a module that makes a binary dist with ## bundled works. -#if(${bundled-dependencies}) +#if(${bundled_dependencies}) ==== ## gather up CDDL licensed works #set($cddl_1_0 = []) @@ -1316,8 +1316,8 @@ You can redistribute it and/or modify it under either the terms of the #set($cpl = []) ## gather up EPL 1.0 works #set($epl = []) -## track commons-math -#set($commons-math = false) +## track commons_math +#set($commons_math = false) ## track if we need jruby additionals. #set($jruby = false) ## track hadoops @@ -1342,7 +1342,7 @@ g:${dep.groupId} AND a:${dep.artifactId} AND v:${dep.version} Until ratis-resource-bundle/src/main/resources/supplemental-models.xml is updated, the build should fail. #end -#if(${debug-print-included-work-info.equalsIgnoreCase("true")}) +#if(${debug_print_included_work_info.equalsIgnoreCase("true")}) ===== Check license for included work @@ -1364,8 +1364,8 @@ ${dep.scm.url} #end #set($aggregated=false) ## Check for our set of known dependencies that require manual LICENSE additions. -#if($dep.artifactId.equals("commons-math3")) -#set($commons-math=true) +#if($dep.artifactId.equals("commons_math3")) +#set($commons_math=true) #end #if($dep.artifactId.equals("jruby-complete")) #set($jruby=true) @@ -1465,7 +1465,7 @@ ${dep.scm.url} #if($hadoop) #hadoop_license() #end -#if($commons-math) +#if($commons_math) #commons_math_license() #end #if(!(${mit.isEmpty()})) diff --git a/ratis-resource-bundle/src/main/resources/META-INF/NOTICE.vm b/ratis-resource-bundle/src/main/resources/META-INF/NOTICE.vm index f3fca903b3..7d473b60ac 100644 --- a/ratis-resource-bundle/src/main/resources/META-INF/NOTICE.vm +++ b/ratis-resource-bundle/src/main/resources/META-INF/NOTICE.vm @@ -23,10 +23,10 @@ Copyright 2017-2019 The Apache Software Foundation This product includes software developed at The Apache Software Foundation (http://www.apache.org/). ## Specially handled included deps. e.g. ASL2.0 works that have additional bundled works but -#set($bundled-boostrap = ${bundled-boostrap.equalsIgnoreCase("true")}) -#set($bundled-logo = ${bundled-logo.equalsIgnoreCase("true")}) -#set($bundled-dependencies = ${bundled-dependencies.equalsIgnoreCase("true")}) -#if($bundled-bootstrap || $bundled-logo || $bundled-dependencies) +#set($bundled_boostrap = ${bundled_boostrap.equalsIgnoreCase("true")}) +#set($bundled_logo = ${bundled_logo.equalsIgnoreCase("true")}) +#set($bundled_dependencies = ${bundled_dependencies.equalsIgnoreCase("true")}) +#if($bundled_bootstrap || $bundled_logo || $bundled_dependencies) ==== ${project.name} contained works @@ -102,13 +102,13 @@ noted in the LICENSE file. #end -#macro(jetty_notice $jetty-include-unixcrypt) +#macro(jetty_notice $jetty_include_unixcrypt) -- This product includes portions of 'The Jetty Web Container' Copyright 1995-2016 Mort Bay Consulting Pty Ltd. -#if(${jetty-include-unixcrypt}) +#if(${jetty_include_unixcrypt}) ## UnixCrypt.java paragraph only in server The UnixCrypt.java code ~Implements the one way cryptography used by Unix systems for simple password protection. Copyright 1996 Aki Yoshida, @@ -207,13 +207,13 @@ under the Apache License 2.0 (see: StringUtils.containsWhitespace()) #end ## first bundled source -#if(${bundled-logo}) +#if(${bundled_logo}) #orca_logo_notice() #end -#if(${bundled-bootstrap}) +#if(${bundled_bootstrap}) #bootstrap_notice() #end -#if(${bundled-dependencies}) +#if(${bundled_dependencies}) #** Note that this will fail the build if we don't have a license. update supplemental-models via setting '-Dlicense.debug.print.included' and looking in the generated LICENSE file @@ -229,7 +229,7 @@ under the Apache License 2.0 (see: StringUtils.containsWhitespace()) #set($mpl_1_1=[]) ## track jettys #set($jetty=false) -#set($jetty-with-crypt=false) +#set($jetty_with_crypt=false) ## track jruby #set($jruby=false) #foreach( ${dep} in ${projects} ) @@ -244,7 +244,7 @@ under the Apache License 2.0 (see: StringUtils.containsWhitespace()) #if(${dep.artifactId.startsWith("jetty")}) #set($jetty=true) #if(${dep.artifactId.equals("jetty")}) -#set($jetty-with-crypt=true) +#set($jetty_with_crypt=true) #end #end #if(${dep.artifactId.equals("log4j")}) @@ -317,7 +317,7 @@ For source see '${dep.url}'. #end ## Print out jetty #if(${jetty}) -#jetty_notice(${jetty-with-crypt}) +#jetty_notice(${jetty_with_crypt}) #end ## Now go through all the lists of Category-B licensed works and make sure we ## name them and give a URL for the project's home page. From 0b540d7fccd43c1bf7c3b84c8d63eaac7af8d4cc Mon Sep 17 00:00:00 2001 From: Potato Date: Thu, 9 Jan 2025 01:17:10 +0800 Subject: [PATCH 136/397] RATIS-2236 Fixed bug where manual triggerSnapshot would never finish (#1207) --- .../java/org/apache/ratis/server/impl/StateMachineUpdater.java | 3 +++ 1 file changed, 3 insertions(+) diff --git a/ratis-server/src/main/java/org/apache/ratis/server/impl/StateMachineUpdater.java b/ratis-server/src/main/java/org/apache/ratis/server/impl/StateMachineUpdater.java index c1db1fd3cc..a919ca732c 100644 --- a/ratis-server/src/main/java/org/apache/ratis/server/impl/StateMachineUpdater.java +++ b/ratis-server/src/main/java/org/apache/ratis/server/impl/StateMachineUpdater.java @@ -216,6 +216,9 @@ private void waitForCommit() throws InterruptedException { // Thus it is possible to have applied > committed initially. final long applied = getLastAppliedIndex(); for(; applied >= raftLog.getLastCommittedIndex() && state == State.RUNNING && !shouldStop(); ) { + if (server.getSnapshotRequestHandler().shouldTriggerTakingSnapshot()) { + takeSnapshot(); + } if (awaitForSignal.await(100, TimeUnit.MILLISECONDS)) { return; } From 1c0d487d45a8f6778f353b235d0b5b03f4e9767e Mon Sep 17 00:00:00 2001 From: szywilliam Date: Fri, 10 Jan 2025 10:05:09 +0800 Subject: [PATCH 137/397] fix cherry-pick generated warnings --- .../main/java/org/apache/ratis/server/impl/RaftServerImpl.java | 1 - 1 file changed, 1 deletion(-) diff --git a/ratis-server/src/main/java/org/apache/ratis/server/impl/RaftServerImpl.java b/ratis-server/src/main/java/org/apache/ratis/server/impl/RaftServerImpl.java index 14b09a0235..02e038ef8f 100644 --- a/ratis-server/src/main/java/org/apache/ratis/server/impl/RaftServerImpl.java +++ b/ratis-server/src/main/java/org/apache/ratis/server/impl/RaftServerImpl.java @@ -118,7 +118,6 @@ import java.nio.file.NoSuchFileException; import java.util.ArrayList; import java.util.Collection; -import java.util.Collections; import java.util.Comparator; import java.util.List; import java.util.Map; From 13e6670b311e51a665c7f2cb975eb8d3480a5076 Mon Sep 17 00:00:00 2001 From: szywilliam Date: Fri, 10 Jan 2025 10:06:10 +0800 Subject: [PATCH 138/397] Change version for the version 3.1.3 -rc2 --- pom.xml | 4 ++-- ratis-assembly/pom.xml | 2 +- ratis-client/pom.xml | 2 +- ratis-common/pom.xml | 2 +- ratis-docs/pom.xml | 2 +- ratis-examples/pom.xml | 2 +- ratis-experiments/pom.xml | 2 +- ratis-grpc/pom.xml | 2 +- ratis-metrics-api/pom.xml | 2 +- ratis-metrics-default/pom.xml | 2 +- ratis-metrics-dropwizard3/pom.xml | 2 +- ratis-netty/pom.xml | 2 +- ratis-proto/pom.xml | 2 +- ratis-replicated-map/pom.xml | 2 +- ratis-resource-bundle/pom.xml | 2 +- ratis-server-api/pom.xml | 2 +- ratis-server/pom.xml | 2 +- ratis-shell/pom.xml | 2 +- ratis-test/pom.xml | 2 +- ratis-tools/pom.xml | 2 +- 20 files changed, 21 insertions(+), 21 deletions(-) diff --git a/pom.xml b/pom.xml index d7019f0dfb..55b588cbfc 100644 --- a/pom.xml +++ b/pom.xml @@ -24,7 +24,7 @@ ratis org.apache.ratis - 3.1.2 + 3.1.3 Apache Ratis pom @@ -158,7 +158,7 @@ - 2024-11-04T07:06:57Z + 2025-01-10T02:06:09Z UTF-8 UTF-8 diff --git a/ratis-assembly/pom.xml b/ratis-assembly/pom.xml index 34e6e52c58..276de0a6b9 100644 --- a/ratis-assembly/pom.xml +++ b/ratis-assembly/pom.xml @@ -17,7 +17,7 @@ ratis org.apache.ratis - 3.1.2 + 3.1.3 ratis-assembly diff --git a/ratis-client/pom.xml b/ratis-client/pom.xml index 707db06fe2..9a8d228d3c 100644 --- a/ratis-client/pom.xml +++ b/ratis-client/pom.xml @@ -17,7 +17,7 @@ ratis org.apache.ratis - 3.1.2 + 3.1.3 ratis-client diff --git a/ratis-common/pom.xml b/ratis-common/pom.xml index 66af275adf..a09f446798 100644 --- a/ratis-common/pom.xml +++ b/ratis-common/pom.xml @@ -17,7 +17,7 @@ ratis org.apache.ratis - 3.1.2 + 3.1.3 ratis-common diff --git a/ratis-docs/pom.xml b/ratis-docs/pom.xml index 578bef6c54..79f9d79b22 100644 --- a/ratis-docs/pom.xml +++ b/ratis-docs/pom.xml @@ -20,7 +20,7 @@ https://maven.apache.org/xsd/maven-4.0.0.xsd"> ratis org.apache.ratis - 3.1.2 + 3.1.3 ratis-docs diff --git a/ratis-examples/pom.xml b/ratis-examples/pom.xml index efd3ba67d6..6b734a9651 100644 --- a/ratis-examples/pom.xml +++ b/ratis-examples/pom.xml @@ -17,7 +17,7 @@ ratis org.apache.ratis - 3.1.2 + 3.1.3 ratis-examples diff --git a/ratis-experiments/pom.xml b/ratis-experiments/pom.xml index 86ed7985f7..467ff08eb9 100644 --- a/ratis-experiments/pom.xml +++ b/ratis-experiments/pom.xml @@ -17,7 +17,7 @@ ratis org.apache.ratis - 3.1.2 + 3.1.3 ratis-experiments diff --git a/ratis-grpc/pom.xml b/ratis-grpc/pom.xml index 7a8f1b6c8f..536b2f70c1 100644 --- a/ratis-grpc/pom.xml +++ b/ratis-grpc/pom.xml @@ -17,7 +17,7 @@ ratis org.apache.ratis - 3.1.2 + 3.1.3 ratis-grpc diff --git a/ratis-metrics-api/pom.xml b/ratis-metrics-api/pom.xml index 2be02faa2e..3f30339bdf 100644 --- a/ratis-metrics-api/pom.xml +++ b/ratis-metrics-api/pom.xml @@ -18,7 +18,7 @@ ratis org.apache.ratis - 3.1.2 + 3.1.3 ratis-metrics-api diff --git a/ratis-metrics-default/pom.xml b/ratis-metrics-default/pom.xml index 9f8bc550be..7e45f42949 100644 --- a/ratis-metrics-default/pom.xml +++ b/ratis-metrics-default/pom.xml @@ -18,7 +18,7 @@ ratis org.apache.ratis - 3.1.2 + 3.1.3 ratis-metrics-default diff --git a/ratis-metrics-dropwizard3/pom.xml b/ratis-metrics-dropwizard3/pom.xml index bcb8a822f1..d9bf14ddff 100644 --- a/ratis-metrics-dropwizard3/pom.xml +++ b/ratis-metrics-dropwizard3/pom.xml @@ -18,7 +18,7 @@ ratis org.apache.ratis - 3.1.2 + 3.1.3 ratis-metrics-dropwizard3 diff --git a/ratis-netty/pom.xml b/ratis-netty/pom.xml index 58605ca912..bbd6cbe945 100644 --- a/ratis-netty/pom.xml +++ b/ratis-netty/pom.xml @@ -17,7 +17,7 @@ ratis org.apache.ratis - 3.1.2 + 3.1.3 ratis-netty diff --git a/ratis-proto/pom.xml b/ratis-proto/pom.xml index e6c16473e6..c5fc03ac90 100644 --- a/ratis-proto/pom.xml +++ b/ratis-proto/pom.xml @@ -17,7 +17,7 @@ ratis org.apache.ratis - 3.1.2 + 3.1.3 ratis-proto diff --git a/ratis-replicated-map/pom.xml b/ratis-replicated-map/pom.xml index b45b735745..a0c4f94c1d 100644 --- a/ratis-replicated-map/pom.xml +++ b/ratis-replicated-map/pom.xml @@ -17,7 +17,7 @@ ratis org.apache.ratis - 3.1.2 + 3.1.3 ratis-replicated-map diff --git a/ratis-resource-bundle/pom.xml b/ratis-resource-bundle/pom.xml index df59a55fd1..3403fab8f8 100644 --- a/ratis-resource-bundle/pom.xml +++ b/ratis-resource-bundle/pom.xml @@ -23,7 +23,7 @@ ratis org.apache.ratis - 3.1.2 + 3.1.3 .. diff --git a/ratis-server-api/pom.xml b/ratis-server-api/pom.xml index ceeaeb1b24..7de592030f 100644 --- a/ratis-server-api/pom.xml +++ b/ratis-server-api/pom.xml @@ -17,7 +17,7 @@ ratis org.apache.ratis - 3.1.2 + 3.1.3 ratis-server-api diff --git a/ratis-server/pom.xml b/ratis-server/pom.xml index 70098160a3..22041f0ee0 100644 --- a/ratis-server/pom.xml +++ b/ratis-server/pom.xml @@ -17,7 +17,7 @@ ratis org.apache.ratis - 3.1.2 + 3.1.3 ratis-server diff --git a/ratis-shell/pom.xml b/ratis-shell/pom.xml index 608e1e8f19..38eed1b980 100644 --- a/ratis-shell/pom.xml +++ b/ratis-shell/pom.xml @@ -17,7 +17,7 @@ ratis org.apache.ratis - 3.1.2 + 3.1.3 ratis-shell diff --git a/ratis-test/pom.xml b/ratis-test/pom.xml index 5c0cefb259..69470bf0db 100644 --- a/ratis-test/pom.xml +++ b/ratis-test/pom.xml @@ -17,7 +17,7 @@ ratis org.apache.ratis - 3.1.2 + 3.1.3 ratis-test diff --git a/ratis-tools/pom.xml b/ratis-tools/pom.xml index 03ca14bf44..49f36f0b7d 100644 --- a/ratis-tools/pom.xml +++ b/ratis-tools/pom.xml @@ -17,7 +17,7 @@ ratis org.apache.ratis - 3.1.2 + 3.1.3 ratis-tools From a4b471a61d9847797eba637c2c61add88c05954e Mon Sep 17 00:00:00 2001 From: jianghuazhu <740087514@qq.com> Date: Sat, 5 Oct 2024 05:16:44 +0800 Subject: [PATCH 139/397] RATIS-2167. Add default value in TermIndex (#1161) --- .../org/apache/ratis/server/protocol/TermIndex.java | 13 +++++++++++++ .../ratis/statemachine/impl/BaseStateMachine.java | 2 +- 2 files changed, 14 insertions(+), 1 deletion(-) diff --git a/ratis-server-api/src/main/java/org/apache/ratis/server/protocol/TermIndex.java b/ratis-server-api/src/main/java/org/apache/ratis/server/protocol/TermIndex.java index 7def686bb5..dac1a51d23 100644 --- a/ratis-server-api/src/main/java/org/apache/ratis/server/protocol/TermIndex.java +++ b/ratis-server-api/src/main/java/org/apache/ratis/server/protocol/TermIndex.java @@ -19,12 +19,25 @@ import org.apache.ratis.proto.RaftProtos.LogEntryProto; import org.apache.ratis.proto.RaftProtos.TermIndexProto; +import org.apache.ratis.server.raftlog.RaftLog; import java.util.Comparator; import java.util.Optional; /** The term and the log index defined in the Raft consensus algorithm. */ public interface TermIndex extends Comparable { + /** + * The initial value. + * When a new Raft group starts, + * all the servers has term 0 and index -1 (= {@link RaftLog#INVALID_LOG_INDEX}). + * Note that term is incremented during leader election + * and index is incremented when writing to the {@link RaftLog}. + * The least term and index possibly written to the {@link RaftLog} + * are respectively 1 and 0 (= {@link RaftLog#LEAST_VALID_LOG_INDEX}). + */ + TermIndex INITIAL_VALUE = valueOf(0, RaftLog.INVALID_LOG_INDEX); + + /** An empty {@link TermIndex} array. */ TermIndex[] EMPTY_ARRAY = {}; /** @return the term. */ diff --git a/ratis-server/src/main/java/org/apache/ratis/statemachine/impl/BaseStateMachine.java b/ratis-server/src/main/java/org/apache/ratis/statemachine/impl/BaseStateMachine.java index 657c6a0fd3..bb7e9856bc 100644 --- a/ratis-server/src/main/java/org/apache/ratis/statemachine/impl/BaseStateMachine.java +++ b/ratis-server/src/main/java/org/apache/ratis/statemachine/impl/BaseStateMachine.java @@ -59,7 +59,7 @@ public class BaseStateMachine implements StateMachine, StateMachine.DataApi, private final SortedMap> transactionFutures = new TreeMap<>(); public BaseStateMachine() { - setLastAppliedTermIndex(TermIndex.valueOf(0, -1)); + setLastAppliedTermIndex(TermIndex.INITIAL_VALUE); } public RaftPeerId getId() { From 8722e3412b7ae5cd416c23c17db52534cb3a95de Mon Sep 17 00:00:00 2001 From: Tsz-Wo Nicholas Sze Date: Mon, 7 Oct 2024 12:09:21 -0700 Subject: [PATCH 140/397] RATIS-2169. mvnw test: Corrupted STDOUT by directly writing to native stream in forked JVM 1 (#1162) --- pom.xml | 8 ++++---- 1 file changed, 4 insertions(+), 4 deletions(-) diff --git a/pom.xml b/pom.xml index 55b588cbfc..48343fa939 100644 --- a/pom.xml +++ b/pom.xml @@ -220,8 +220,8 @@ 4 2.0.7 - 5.10.1 - 0.8.11 + 5.11.2 + 0.8.12 @@ -424,7 +424,7 @@ org.junit junit-bom - ${junit.jupiter.version} + ${junit-bom.version} pom import @@ -640,7 +640,7 @@ all 600 - -Xmx2048m -XX:+HeapDumpOnOutOfMemoryError @{argLine} + -Xmx2g -XX:+HeapDumpOnOutOfMemoryError @{argLine} ${project.build.directory}/log ${project.build.directory}/tmp From 805c38e1e89e41189406a1c1b40345a615893604 Mon Sep 17 00:00:00 2001 From: Tsz-Wo Nicholas Sze Date: Wed, 16 Oct 2024 18:06:59 -0700 Subject: [PATCH 141/397] RATIS-2168. Support custom gRPC services. (#1169) --- ratis-grpc/pom.xml | 4 + .../org/apache/ratis/grpc/GrpcConfigKeys.java | 29 ++- .../org/apache/ratis/grpc/GrpcFactory.java | 19 +- .../ratis/grpc/server/GrpcLogAppender.java | 6 +- .../ratis/grpc/server/GrpcServices.java | 56 +++++ ...GrpcService.java => GrpcServicesImpl.java} | 43 ++-- .../ratis/grpc/MiniRaftClusterWithGrpc.java | 4 +- .../ratis/grpc/TestCustomGrpcServices.java | 205 ++++++++++++++++++ .../{ => server}/TestGrpcMessageMetrics.java | 8 +- 9 files changed, 336 insertions(+), 38 deletions(-) create mode 100644 ratis-grpc/src/main/java/org/apache/ratis/grpc/server/GrpcServices.java rename ratis-grpc/src/main/java/org/apache/ratis/grpc/server/{GrpcService.java => GrpcServicesImpl.java} (91%) create mode 100644 ratis-test/src/test/java/org/apache/ratis/grpc/TestCustomGrpcServices.java rename ratis-test/src/test/java/org/apache/ratis/grpc/{ => server}/TestGrpcMessageMetrics.java (91%) diff --git a/ratis-grpc/pom.xml b/ratis-grpc/pom.xml index 536b2f70c1..e0736de195 100644 --- a/ratis-grpc/pom.xml +++ b/ratis-grpc/pom.xml @@ -53,6 +53,10 @@ test test-jar + + org.apache.ratis + ratis-server-api + ratis-server org.apache.ratis diff --git a/ratis-grpc/src/main/java/org/apache/ratis/grpc/GrpcConfigKeys.java b/ratis-grpc/src/main/java/org/apache/ratis/grpc/GrpcConfigKeys.java index 8caacfeeb5..a603960f17 100644 --- a/ratis-grpc/src/main/java/org/apache/ratis/grpc/GrpcConfigKeys.java +++ b/ratis-grpc/src/main/java/org/apache/ratis/grpc/GrpcConfigKeys.java @@ -19,6 +19,7 @@ import org.apache.ratis.conf.Parameters; import org.apache.ratis.conf.RaftProperties; +import org.apache.ratis.grpc.server.GrpcServices; import org.apache.ratis.server.RaftServerConfigKeys; import org.apache.ratis.util.SizeInBytes; import org.apache.ratis.util.TimeDuration; @@ -230,15 +231,6 @@ static void setAsyncRequestThreadPoolSize(RaftProperties properties, int port) { setInt(properties::setInt, ASYNC_REQUEST_THREAD_POOL_SIZE_KEY, port); } - String TLS_CONF_PARAMETER = PREFIX + ".tls.conf"; - Class TLS_CONF_CLASS = TLS.CONF_CLASS; - static GrpcTlsConfig tlsConf(Parameters parameters) { - return parameters != null ? parameters.get(TLS_CONF_PARAMETER, TLS_CONF_CLASS): null; - } - static void setTlsConf(Parameters parameters, GrpcTlsConfig conf) { - parameters.put(TLS_CONF_PARAMETER, conf, TLS_CONF_CLASS); - } - String LEADER_OUTSTANDING_APPENDS_MAX_KEY = PREFIX + ".leader.outstanding.appends.max"; int LEADER_OUTSTANDING_APPENDS_MAX_DEFAULT = 8; static int leaderOutstandingAppendsMax(RaftProperties properties) { @@ -292,6 +284,25 @@ static void setLogMessageBatchDuration(RaftProperties properties, setTimeDuration(properties::setTimeDuration, LOG_MESSAGE_BATCH_DURATION_KEY, logMessageBatchDuration); } + + String SERVICES_CUSTOMIZER_PARAMETER = PREFIX + ".services.customizer"; + Class SERVICES_CUSTOMIZER_CLASS = GrpcServices.Customizer.class; + static GrpcServices.Customizer servicesCustomizer(Parameters parameters) { + return parameters == null ? null + : parameters.get(SERVICES_CUSTOMIZER_PARAMETER, SERVICES_CUSTOMIZER_CLASS); + } + static void setServicesCustomizer(Parameters parameters, GrpcServices.Customizer customizer) { + parameters.put(SERVICES_CUSTOMIZER_PARAMETER, customizer, SERVICES_CUSTOMIZER_CLASS); + } + + String TLS_CONF_PARAMETER = PREFIX + ".tls.conf"; + Class TLS_CONF_CLASS = TLS.CONF_CLASS; + static GrpcTlsConfig tlsConf(Parameters parameters) { + return parameters != null ? parameters.get(TLS_CONF_PARAMETER, TLS_CONF_CLASS): null; + } + static void setTlsConf(Parameters parameters, GrpcTlsConfig conf) { + parameters.put(TLS_CONF_PARAMETER, conf, TLS_CONF_CLASS); + } } String MESSAGE_SIZE_MAX_KEY = PREFIX + ".message.size.max"; diff --git a/ratis-grpc/src/main/java/org/apache/ratis/grpc/GrpcFactory.java b/ratis-grpc/src/main/java/org/apache/ratis/grpc/GrpcFactory.java index 75eb34a2d1..331d1a8585 100644 --- a/ratis-grpc/src/main/java/org/apache/ratis/grpc/GrpcFactory.java +++ b/ratis-grpc/src/main/java/org/apache/ratis/grpc/GrpcFactory.java @@ -22,7 +22,8 @@ import org.apache.ratis.conf.RaftProperties; import org.apache.ratis.grpc.client.GrpcClientRpc; import org.apache.ratis.grpc.server.GrpcLogAppender; -import org.apache.ratis.grpc.server.GrpcService; +import org.apache.ratis.grpc.server.GrpcServices; +import org.apache.ratis.grpc.server.GrpcServicesImpl; import org.apache.ratis.protocol.ClientId; import org.apache.ratis.rpc.SupportedRpcType; import org.apache.ratis.server.RaftServer; @@ -64,6 +65,8 @@ static boolean checkPooledByteBufAllocatorUseCacheForAllThreads(Consumer return value; } + private final GrpcServices.Customizer servicesCustomizer; + private final GrpcTlsConfig tlsConfig; private final GrpcTlsConfig adminTlsConfig; private final GrpcTlsConfig clientTlsConfig; @@ -76,7 +79,7 @@ public static Parameters newRaftParameters(GrpcTlsConfig conf) { } public GrpcFactory(Parameters parameters) { - this( + this(GrpcConfigKeys.Server.servicesCustomizer(parameters), GrpcConfigKeys.TLS.conf(parameters), GrpcConfigKeys.Admin.tlsConf(parameters), GrpcConfigKeys.Client.tlsConf(parameters), @@ -85,11 +88,14 @@ public GrpcFactory(Parameters parameters) { } public GrpcFactory(GrpcTlsConfig tlsConfig) { - this(tlsConfig, null, null, null); + this(null, tlsConfig, null, null, null); } - private GrpcFactory(GrpcTlsConfig tlsConfig, GrpcTlsConfig adminTlsConfig, + private GrpcFactory(GrpcServices.Customizer servicesCustomizer, + GrpcTlsConfig tlsConfig, GrpcTlsConfig adminTlsConfig, GrpcTlsConfig clientTlsConfig, GrpcTlsConfig serverTlsConfig) { + this.servicesCustomizer = servicesCustomizer; + this.tlsConfig = tlsConfig; this.adminTlsConfig = adminTlsConfig; this.clientTlsConfig = clientTlsConfig; @@ -123,10 +129,11 @@ public LogAppender newLogAppender(RaftServer.Division server, LeaderState state, } @Override - public GrpcService newRaftServerRpc(RaftServer server) { + public GrpcServices newRaftServerRpc(RaftServer server) { checkPooledByteBufAllocatorUseCacheForAllThreads(LOG::info); - return GrpcService.newBuilder() + return GrpcServicesImpl.newBuilder() .setServer(server) + .setCustomizer(servicesCustomizer) .setAdminTlsConfig(getAdminTlsConfig()) .setServerTlsConfig(getServerTlsConfig()) .setClientTlsConfig(getClientTlsConfig()) diff --git a/ratis-grpc/src/main/java/org/apache/ratis/grpc/server/GrpcLogAppender.java b/ratis-grpc/src/main/java/org/apache/ratis/grpc/server/GrpcLogAppender.java index ce7bd315c5..2b14f86f61 100644 --- a/ratis-grpc/src/main/java/org/apache/ratis/grpc/server/GrpcLogAppender.java +++ b/ratis-grpc/src/main/java/org/apache/ratis/grpc/server/GrpcLogAppender.java @@ -192,8 +192,8 @@ public GrpcLogAppender(RaftServer.Division server, LeaderState leaderState, Foll } @Override - public GrpcService getServerRpc() { - return (GrpcService)super.getServerRpc(); + public GrpcServicesImpl getServerRpc() { + return (GrpcServicesImpl)super.getServerRpc(); } private GrpcServerProtocolClient getClient() throws IOException { @@ -419,7 +419,7 @@ private static void sleep(TimeDuration waitTime, boolean heartbeat) private void sendRequest(AppendEntriesRequest request, AppendEntriesRequestProto proto) throws InterruptedIOException { - CodeInjectionForTesting.execute(GrpcService.GRPC_SEND_SERVER_REQUEST, + CodeInjectionForTesting.execute(GrpcServicesImpl.GRPC_SEND_SERVER_REQUEST, getServer().getId(), null, proto); resetHeartbeatTrigger(); diff --git a/ratis-grpc/src/main/java/org/apache/ratis/grpc/server/GrpcServices.java b/ratis-grpc/src/main/java/org/apache/ratis/grpc/server/GrpcServices.java new file mode 100644 index 0000000000..663fd6d743 --- /dev/null +++ b/ratis-grpc/src/main/java/org/apache/ratis/grpc/server/GrpcServices.java @@ -0,0 +1,56 @@ +/* + * 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.ratis.grpc.server; + +import org.apache.ratis.server.RaftServerRpc; +import org.apache.ratis.thirdparty.io.grpc.netty.NettyServerBuilder; + +import java.util.EnumSet; + +/** The gRPC services extending {@link RaftServerRpc}. */ +public interface GrpcServices extends RaftServerRpc { + /** The type of the services. */ + enum Type {ADMIN, CLIENT, SERVER} + + /** + * To customize the services. + * For example, add a custom service. + */ + interface Customizer { + /** The default NOOP {@link Customizer}. */ + class Default implements Customizer { + private static final Default INSTANCE = new Default(); + + @Override + public NettyServerBuilder customize(NettyServerBuilder builder, EnumSet types) { + return builder; + } + } + + static Customizer getDefaultInstance() { + return Default.INSTANCE; + } + + /** + * Customize the given builder for the given types. + * + * @return the customized builder. + */ + NettyServerBuilder customize(NettyServerBuilder builder, EnumSet types); + } +} diff --git a/ratis-grpc/src/main/java/org/apache/ratis/grpc/server/GrpcService.java b/ratis-grpc/src/main/java/org/apache/ratis/grpc/server/GrpcServicesImpl.java similarity index 91% rename from ratis-grpc/src/main/java/org/apache/ratis/grpc/server/GrpcService.java rename to ratis-grpc/src/main/java/org/apache/ratis/grpc/server/GrpcServicesImpl.java index 864c0166a8..853a420d1c 100644 --- a/ratis-grpc/src/main/java/org/apache/ratis/grpc/server/GrpcService.java +++ b/ratis-grpc/src/main/java/org/apache/ratis/grpc/server/GrpcServicesImpl.java @@ -21,6 +21,7 @@ import org.apache.ratis.grpc.GrpcConfigKeys; import org.apache.ratis.grpc.GrpcTlsConfig; import org.apache.ratis.grpc.GrpcUtil; +import org.apache.ratis.grpc.metrics.MessageMetrics; import org.apache.ratis.grpc.metrics.intercept.server.MetricServerInterceptor; import org.apache.ratis.protocol.AdminAsynchronousProtocol; import org.apache.ratis.protocol.RaftGroupId; @@ -48,6 +49,7 @@ import java.io.IOException; import java.net.InetSocketAddress; +import java.util.EnumSet; import java.util.HashMap; import java.util.Map; import java.util.concurrent.CompletableFuture; @@ -57,11 +59,12 @@ import static org.apache.ratis.thirdparty.io.netty.handler.ssl.SslProvider.OPENSSL; /** A grpc implementation of {@link org.apache.ratis.server.RaftServerRpc}. */ -public final class GrpcService extends RaftServerRpcWithProxy> { - static final Logger LOG = LoggerFactory.getLogger(GrpcService.class); +public final class GrpcServicesImpl + extends RaftServerRpcWithProxy> + implements GrpcServices { + static final Logger LOG = LoggerFactory.getLogger(GrpcServicesImpl.class); public static final String GRPC_SEND_SERVER_REQUEST = - JavaUtils.getClassSimpleName(GrpcService.class) + ".sendRequest"; + JavaUtils.getClassSimpleName(GrpcServicesImpl.class) + ".sendRequest"; class AsyncService implements RaftServerAsynchronousProtocol { @@ -99,6 +102,7 @@ public void onCompleted() { public static final class Builder { private RaftServer server; + private Customizer customizer; private String adminHost; private int adminPort; @@ -145,6 +149,11 @@ public Builder setServer(RaftServer raftServer) { return this; } + public Builder setCustomizer(Customizer customizer) { + this.customizer = customizer != null? customizer : Customizer.getDefaultInstance(); + return this; + } + private GrpcServerProtocolClient newGrpcServerProtocolClient(RaftPeer target) { return new GrpcServerProtocolClient(target, flowControlWindow.getSizeInt(), requestTimeoutDuration, serverTlsConfig, separateHeartbeatChannel); @@ -172,6 +181,10 @@ private MetricServerInterceptor newMetricServerInterceptor() { JavaUtils.getClassSimpleName(getClass()) + "_" + serverPort); } + Server buildServer(NettyServerBuilder builder, EnumSet types) { + return customizer.customize(builder, types).build(); + } + private NettyServerBuilder newNettyServerBuilderForServer() { return newNettyServerBuilder(serverHost, serverPort, serverTlsConfig); } @@ -218,21 +231,24 @@ private boolean separateClientServer() { } Server newServer(GrpcClientProtocolService client, ServerInterceptor interceptor) { + final EnumSet types = EnumSet.of(GrpcServices.Type.SERVER); final NettyServerBuilder serverBuilder = newNettyServerBuilderForServer(); final GrpcServerProtocolService service = newGrpcServerProtocolService(); serverBuilder.addService(ServerInterceptors.intercept(service, interceptor)); if (!separateAdminServer()) { + types.add(GrpcServices.Type.ADMIN); addAdminService(serverBuilder, server, interceptor); } if (!separateClientServer()) { + types.add(GrpcServices.Type.CLIENT); addClientService(serverBuilder, client, interceptor); } - return serverBuilder.build(); + return buildServer(serverBuilder, types); } - public GrpcService build() { - return new GrpcService(this); + public GrpcServicesImpl build() { + return new GrpcServicesImpl(this); } public Builder setAdminTlsConfig(GrpcTlsConfig config) { @@ -267,11 +283,7 @@ public static Builder newBuilder() { private final MetricServerInterceptor serverInterceptor; - public MetricServerInterceptor getServerInterceptor() { - return serverInterceptor; - } - - private GrpcService(Builder b) { + private GrpcServicesImpl(Builder b) { super(b.server::getId, id -> new PeerProxyMap<>(id.toString(), b::newGrpcServerProtocolClient)); this.executor = b.newExecutor(); @@ -285,7 +297,7 @@ private GrpcService(Builder b) { if (b.separateAdminServer()) { final NettyServerBuilder builder = b.newNettyServerBuilderForAdmin(); addAdminService(builder, b.server, serverInterceptor); - final Server adminServer = builder.build(); + final Server adminServer = b.buildServer(builder, EnumSet.of(GrpcServices.Type.ADMIN)); servers.put(GrpcAdminProtocolService.class.getName(), adminServer); adminServerAddressSupplier = newAddressSupplier(b.adminPort, adminServer); } else { @@ -295,7 +307,7 @@ private GrpcService(Builder b) { if (b.separateClientServer()) { final NettyServerBuilder builder = b.newNettyServerBuilderForClient(); addClientService(builder, clientProtocolService, serverInterceptor); - final Server clientServer = builder.build(); + final Server clientServer = b.buildServer(builder, EnumSet.of(GrpcServices.Type.CLIENT)); servers.put(GrpcClientProtocolService.class.getName(), clientServer); clientServerAddressSupplier = newAddressSupplier(b.clientPort, clientServer); } else { @@ -411,4 +423,7 @@ public StartLeaderElectionReplyProto startLeaderElection(StartLeaderElectionRequ return getProxies().getProxy(target).startLeaderElection(request); } + MessageMetrics getMessageMetrics() { + return serverInterceptor.getMetrics(); + } } diff --git a/ratis-grpc/src/test/java/org/apache/ratis/grpc/MiniRaftClusterWithGrpc.java b/ratis-grpc/src/test/java/org/apache/ratis/grpc/MiniRaftClusterWithGrpc.java index 84cb72bbe9..0e4eb55544 100644 --- a/ratis-grpc/src/test/java/org/apache/ratis/grpc/MiniRaftClusterWithGrpc.java +++ b/ratis-grpc/src/test/java/org/apache/ratis/grpc/MiniRaftClusterWithGrpc.java @@ -21,7 +21,7 @@ import org.apache.ratis.RaftTestUtil; import org.apache.ratis.conf.Parameters; import org.apache.ratis.conf.RaftProperties; -import org.apache.ratis.grpc.server.GrpcService; +import org.apache.ratis.grpc.server.GrpcServicesImpl; import org.apache.ratis.protocol.RaftGroup; import org.apache.ratis.protocol.RaftPeer; import org.apache.ratis.protocol.RaftPeerId; @@ -53,7 +53,7 @@ default Factory getFactory() { } public static final DelayLocalExecutionInjection SEND_SERVER_REQUEST_INJECTION = - new DelayLocalExecutionInjection(GrpcService.GRPC_SEND_SERVER_REQUEST); + new DelayLocalExecutionInjection(GrpcServicesImpl.GRPC_SEND_SERVER_REQUEST); public MiniRaftClusterWithGrpc(String[] ids, RaftProperties properties, Parameters parameters) { this(ids, new String[0], properties, parameters); diff --git a/ratis-test/src/test/java/org/apache/ratis/grpc/TestCustomGrpcServices.java b/ratis-test/src/test/java/org/apache/ratis/grpc/TestCustomGrpcServices.java new file mode 100644 index 0000000000..13c4a59fba --- /dev/null +++ b/ratis-test/src/test/java/org/apache/ratis/grpc/TestCustomGrpcServices.java @@ -0,0 +1,205 @@ +/* + * 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.ratis.grpc; + +import org.apache.ratis.BaseTest; +import org.apache.ratis.RaftTestUtil; +import org.apache.ratis.client.RaftClient; +import org.apache.ratis.conf.Parameters; +import org.apache.ratis.conf.RaftProperties; +import org.apache.ratis.grpc.server.GrpcServices; +import org.apache.ratis.protocol.RaftClientReply; +import org.apache.ratis.server.RaftServerRpc; +import org.apache.ratis.test.proto.GreeterGrpc; +import org.apache.ratis.test.proto.HelloReply; +import org.apache.ratis.test.proto.HelloRequest; +import org.apache.ratis.thirdparty.io.grpc.ManagedChannel; +import org.apache.ratis.thirdparty.io.grpc.ManagedChannelBuilder; +import org.apache.ratis.thirdparty.io.grpc.netty.NettyServerBuilder; +import org.apache.ratis.thirdparty.io.grpc.stub.StreamObserver; +import org.apache.ratis.util.IOUtils; +import org.apache.ratis.util.NetUtils; +import org.junit.jupiter.api.Assertions; +import org.junit.jupiter.api.Test; + +import java.io.Closeable; +import java.io.IOException; +import java.util.EnumSet; +import java.util.Objects; +import java.util.Queue; +import java.util.concurrent.CompletableFuture; +import java.util.concurrent.ConcurrentLinkedQueue; +import java.util.concurrent.TimeUnit; + +import static org.apache.ratis.RaftTestUtil.waitForLeader; + +public class TestCustomGrpcServices extends BaseTest { + /** Add two different greeter services for client and admin. */ + class MyCustomizer implements GrpcServices.Customizer { + final GreeterImpl clientGreeter = new GreeterImpl("Hello"); + final GreeterImpl adminGreeter = new GreeterImpl("Hi"); + + @Override + public NettyServerBuilder customize(NettyServerBuilder builder, EnumSet types) { + if (types.contains(GrpcServices.Type.CLIENT)) { + return builder.addService(clientGreeter); + } + if (types.contains(GrpcServices.Type.ADMIN)) { + return builder.addService(adminGreeter); + } + return builder; + } + } + + class GreeterImpl extends GreeterGrpc.GreeterImplBase { + private final String prefix; + + GreeterImpl(String prefix) { + this.prefix = prefix; + } + + String toReply(String request) { + return prefix + " " + request; + } + + @Override + public StreamObserver hello(StreamObserver responseObserver) { + return new StreamObserver() { + @Override + public void onNext(HelloRequest helloRequest) { + final String reply = toReply(helloRequest.getName()); + responseObserver.onNext(HelloReply.newBuilder().setMessage(reply).build()); + } + + @Override + public void onError(Throwable throwable) { + LOG.error("onError", throwable); + } + + @Override + public void onCompleted() { + responseObserver.onCompleted(); + } + }; + } + } + + class GreeterClient implements Closeable { + private final ManagedChannel channel; + private final StreamObserver requestHandler; + private final Queue> replies = new ConcurrentLinkedQueue<>(); + + GreeterClient(int port) { + this.channel = ManagedChannelBuilder.forAddress(NetUtils.LOCALHOST, port) + .usePlaintext() + .build(); + + final StreamObserver responseHandler = new StreamObserver() { + @Override + public void onNext(HelloReply helloReply) { + Objects.requireNonNull(replies.poll(), "queue is empty") + .complete(helloReply.getMessage()); + } + + @Override + public void onError(Throwable throwable) { + LOG.info("onError", throwable); + completeExceptionally(throwable); + } + + @Override + public void onCompleted() { + LOG.info("onCompleted"); + completeExceptionally(new IllegalStateException("onCompleted")); + } + + void completeExceptionally(Throwable throwable) { + replies.forEach(f -> f.completeExceptionally(throwable)); + replies.clear(); + } + }; + this.requestHandler = GreeterGrpc.newStub(channel).hello(responseHandler); + } + + @Override + public void close() throws IOException { + try { + /* After the request handler is cancelled, no more life-cycle hooks are allowed, + * see {@link org.apache.ratis.thirdparty.io.grpc.ClientCall.Listener#cancel(String, Throwable)} */ + // requestHandler.onCompleted(); + channel.shutdown().awaitTermination(5, TimeUnit.SECONDS); + } catch (InterruptedException e) { + Thread.currentThread().interrupt(); + throw IOUtils.toInterruptedIOException("Failed to close", e); + } + } + + CompletableFuture send(String name) { + LOG.info("send: {}", name); + final HelloRequest request = HelloRequest.newBuilder().setName(name).build(); + final CompletableFuture f = new CompletableFuture<>(); + try { + requestHandler.onNext(request); + replies.offer(f); + } catch (IllegalStateException e) { + // already closed + f.completeExceptionally(e); + } + return f.whenComplete((r, e) -> LOG.info("reply: {}", r)); + } + } + + @Test + public void testCustomServices() throws Exception { + final String[] ids = {"s0"}; + final RaftProperties properties = new RaftProperties(); + + final Parameters parameters = new Parameters(); + final MyCustomizer customizer = new MyCustomizer(); + GrpcConfigKeys.Server.setServicesCustomizer(parameters, customizer); + + try(MiniRaftClusterWithGrpc cluster = new MiniRaftClusterWithGrpc(ids, properties, parameters)) { + cluster.start(); + final RaftServerRpc server = waitForLeader(cluster).getRaftServer().getServerRpc(); + + // test Raft service + try (RaftClient client = cluster.createClient()) { + final RaftClientReply reply = client.io().send(new RaftTestUtil.SimpleMessage("abc")); + Assertions.assertTrue(reply.isSuccess()); + } + + // test custom client service + final int clientPort = server.getClientServerAddress().getPort(); + try (GreeterClient client = new GreeterClient(clientPort)) { + sendAndAssertReply("world", client, customizer.clientGreeter); + } + + // test custom admin service + final int adminPort = server.getAdminServerAddress().getPort(); + try (GreeterClient admin = new GreeterClient(adminPort)) { + sendAndAssertReply("admin", admin, customizer.adminGreeter); + } + } + } + + static void sendAndAssertReply(String name, GreeterClient client, GreeterImpl greeter) { + final String computed = client.send(name).join(); + final String expected = greeter.toReply(name); + Assertions.assertEquals(expected, computed); + } +} diff --git a/ratis-test/src/test/java/org/apache/ratis/grpc/TestGrpcMessageMetrics.java b/ratis-test/src/test/java/org/apache/ratis/grpc/server/TestGrpcMessageMetrics.java similarity index 91% rename from ratis-test/src/test/java/org/apache/ratis/grpc/TestGrpcMessageMetrics.java rename to ratis-test/src/test/java/org/apache/ratis/grpc/server/TestGrpcMessageMetrics.java index 812c691e20..8094069cf5 100644 --- a/ratis-test/src/test/java/org/apache/ratis/grpc/TestGrpcMessageMetrics.java +++ b/ratis-test/src/test/java/org/apache/ratis/grpc/server/TestGrpcMessageMetrics.java @@ -15,13 +15,13 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.ratis.grpc; +package org.apache.ratis.grpc.server; import org.apache.ratis.BaseTest; +import org.apache.ratis.grpc.MiniRaftClusterWithGrpc; import org.apache.ratis.server.impl.MiniRaftCluster; import org.apache.ratis.RaftTestUtil; import org.apache.ratis.client.RaftClient; -import org.apache.ratis.grpc.server.GrpcService; import org.apache.ratis.metrics.impl.JvmMetrics; import org.apache.ratis.metrics.RatisMetricRegistry; import org.apache.ratis.protocol.RaftClientReply; @@ -66,8 +66,8 @@ static void sendMessages(MiniRaftCluster cluster) throws Exception { static void assertMessageCount(RaftServer.Division server) { String serverId = server.getId().toString(); - GrpcService service = (GrpcService) RaftServerTestUtil.getServerRpc(server); - RatisMetricRegistry registry = service.getServerInterceptor().getMetrics().getRegistry(); + final GrpcServicesImpl services = (GrpcServicesImpl) RaftServerTestUtil.getServerRpc(server); + final RatisMetricRegistry registry = services.getMessageMetrics().getRegistry(); String counter_prefix = serverId + "_" + "ratis.grpc.RaftServerProtocolService"; Assertions.assertTrue( registry.counter(counter_prefix + "_" + "requestVote" + "_OK_completed_total").getCount() > 0); From ab8e17b0a695e15706bbabe2e44c74df84520681 Mon Sep 17 00:00:00 2001 From: jianghuazhu <740087514@qq.com> Date: Thu, 31 Oct 2024 11:16:15 +0800 Subject: [PATCH 142/397] RATIS-2182. Fix some Objects.requireNonNull messages (#1172) --- .../main/java/org/apache/ratis/util/SizeInBytes.java | 4 ++-- .../main/java/org/apache/ratis/util/TimeDuration.java | 4 ++-- .../apache/ratis/statemachine/TransactionContext.java | 2 +- .../org/apache/ratis/server/impl/PendingRequest.java | 2 +- .../ratis/server/impl/RaftConfigurationImpl.java | 10 +++++----- 5 files changed, 11 insertions(+), 11 deletions(-) diff --git a/ratis-common/src/main/java/org/apache/ratis/util/SizeInBytes.java b/ratis-common/src/main/java/org/apache/ratis/util/SizeInBytes.java index 683f0da628..0756f4feee 100644 --- a/ratis-common/src/main/java/org/apache/ratis/util/SizeInBytes.java +++ b/ratis-common/src/main/java/org/apache/ratis/util/SizeInBytes.java @@ -33,14 +33,14 @@ public static SizeInBytes valueOf(long size) { } public static SizeInBytes valueOf(long n, TraditionalBinaryPrefix prefix) { - final long size = Objects.requireNonNull(prefix, "prefix = null").toLong(n); + final long size = Objects.requireNonNull(prefix, "prefix == null").toLong(n); final String input = n + " " + prefix.getSymbol(); final String description = input + " (=" + size + ")"; return new SizeInBytes(size, input, description); } public static SizeInBytes valueOf(String input) { - input = Objects.requireNonNull(input, "input = null").trim(); + input = Objects.requireNonNull(input, "input == null").trim(); final int last = input.length() - 1; final String s = "b".equalsIgnoreCase(input.substring(last))? diff --git a/ratis-common/src/main/java/org/apache/ratis/util/TimeDuration.java b/ratis-common/src/main/java/org/apache/ratis/util/TimeDuration.java index 2abdfdfaca..2a520083e0 100644 --- a/ratis-common/src/main/java/org/apache/ratis/util/TimeDuration.java +++ b/ratis-common/src/main/java/org/apache/ratis/util/TimeDuration.java @@ -144,7 +144,7 @@ public static long parse(String timeString, TimeUnit targetUnit) { * @return a {@link TimeDuration} in the target unit. */ public static TimeDuration valueOf(String timeString, TimeUnit defaultUnit) { - Objects.requireNonNull(timeString, "timeString = null"); + Objects.requireNonNull(timeString, "timeString == null"); final String lower = timeString.trim().replace("_", "").toLowerCase(); for(Abbreviation a : Abbreviation.values()) { for(String s : a.getSymbols()) { @@ -172,7 +172,7 @@ public static TimeDuration valueOf(long duration, TimeUnit unit) { private TimeDuration(long duration, TimeUnit unit) { this.duration = duration; - this.unit = Objects.requireNonNull(unit, "unit = null"); + this.unit = Objects.requireNonNull(unit, "unit == null"); } /** @return the duration value. */ diff --git a/ratis-server-api/src/main/java/org/apache/ratis/statemachine/TransactionContext.java b/ratis-server-api/src/main/java/org/apache/ratis/statemachine/TransactionContext.java index 3821b058c9..35a40efb57 100644 --- a/ratis-server-api/src/main/java/org/apache/ratis/statemachine/TransactionContext.java +++ b/ratis-server-api/src/main/java/org/apache/ratis/statemachine/TransactionContext.java @@ -194,7 +194,7 @@ public TransactionContext build() { return newTransactionContext(stateMachine, clientRequest, stateMachineLogEntry, logData, stateMachineData, stateMachineContext); } else { - Objects.requireNonNull(logEntry, "logEntry MUST NOT be null since clientRequest == null"); + Objects.requireNonNull(logEntry, "logEntry must not be null since clientRequest == null"); Preconditions.assertTrue(logEntry.hasStateMachineLogEntry(), () -> "Unexpected logEntry: stateMachineLogEntry not found, logEntry=" + logEntry); return newTransactionContext(serverRole, stateMachine, logEntry); diff --git a/ratis-server/src/main/java/org/apache/ratis/server/impl/PendingRequest.java b/ratis-server/src/main/java/org/apache/ratis/server/impl/PendingRequest.java index 06a3a7b3ce..ed13b10113 100644 --- a/ratis-server/src/main/java/org/apache/ratis/server/impl/PendingRequest.java +++ b/ratis-server/src/main/java/org/apache/ratis/server/impl/PendingRequest.java @@ -64,7 +64,7 @@ RaftClientReply convert(RaftClientRequest q, RaftClientReply p) { } TermIndex getTermIndex() { - return Objects.requireNonNull(termIndex, "termIndex"); + return Objects.requireNonNull(termIndex, "termIndex == null"); } RaftClientRequest getRequest() { diff --git a/ratis-server/src/main/java/org/apache/ratis/server/impl/RaftConfigurationImpl.java b/ratis-server/src/main/java/org/apache/ratis/server/impl/RaftConfigurationImpl.java index 15a114cdb4..2e5cd58120 100644 --- a/ratis-server/src/main/java/org/apache/ratis/server/impl/RaftConfigurationImpl.java +++ b/ratis-server/src/main/java/org/apache/ratis/server/impl/RaftConfigurationImpl.java @@ -58,7 +58,7 @@ static final class Builder { private Builder() {} Builder setConf(PeerConfiguration conf) { - Objects.requireNonNull(conf); + Objects.requireNonNull(conf, "PeerConfiguration == null"); Preconditions.assertTrue(this.conf == null, "conf is already set."); this.conf = conf; return this; @@ -73,7 +73,7 @@ Builder setConf(Iterable peers, Iterable listeners) { } Builder setConf(RaftConfigurationImpl transitionalConf) { - Objects.requireNonNull(transitionalConf); + Objects.requireNonNull(transitionalConf, "transitionalConf == null"); Preconditions.assertTrue(transitionalConf.isTransitional()); Preconditions.assertTrue(!forceTransitional); @@ -83,7 +83,7 @@ Builder setConf(RaftConfigurationImpl transitionalConf) { Builder setOldConf(PeerConfiguration oldConf) { - Objects.requireNonNull(oldConf); + Objects.requireNonNull(oldConf, "oldConf == null"); Preconditions.assertTrue(this.oldConf == null, "oldConf is already set."); this.oldConf = oldConf; return this; @@ -94,7 +94,7 @@ Builder setOldConf(Iterable oldPeers, Iterable oldListeners) } Builder setOldConf(RaftConfigurationImpl stableConf) { - Objects.requireNonNull(stableConf); + Objects.requireNonNull(stableConf, "stableConf == null"); Preconditions.assertTrue(stableConf.isStable()); Preconditions.assertTrue(!forceStable); @@ -133,7 +133,7 @@ RaftConfigurationImpl build() { private RaftConfigurationImpl(PeerConfiguration conf, PeerConfiguration oldConf, long logEntryIndex) { - this.conf = Objects.requireNonNull(conf); + this.conf = Objects.requireNonNull(conf, "PeerConfiguration == null"); this.oldConf = oldConf; this.logEntryIndex = logEntryIndex; } From 73579fd17a76894ce3f5f4c3e23f880c0ffd924d Mon Sep 17 00:00:00 2001 From: Potato Date: Mon, 13 Jan 2025 09:52:28 +0800 Subject: [PATCH 143/397] RATIS-2237. Add getFollowerMatchIndices interface for DivisionInfo --- .../main/java/org/apache/ratis/server/DivisionInfo.java | 6 ++++++ .../org/apache/ratis/server/impl/LeaderStateImpl.java | 4 ++++ .../java/org/apache/ratis/server/impl/RaftServerImpl.java | 8 ++++++++ 3 files changed, 18 insertions(+) diff --git a/ratis-server-api/src/main/java/org/apache/ratis/server/DivisionInfo.java b/ratis-server-api/src/main/java/org/apache/ratis/server/DivisionInfo.java index c07317bd69..78073a21c5 100644 --- a/ratis-server-api/src/main/java/org/apache/ratis/server/DivisionInfo.java +++ b/ratis-server-api/src/main/java/org/apache/ratis/server/DivisionInfo.java @@ -80,4 +80,10 @@ default boolean isAlive() { * otherwise, return null. */ long[] getFollowerNextIndices(); + + /** + * @return an array of match indices of the followers if this server division is the leader; + * otherwise, return null. + */ + long[] getFollowerMatchIndices(); } diff --git a/ratis-server/src/main/java/org/apache/ratis/server/impl/LeaderStateImpl.java b/ratis-server/src/main/java/org/apache/ratis/server/impl/LeaderStateImpl.java index 5870f51a68..66d2c1a1c3 100644 --- a/ratis-server/src/main/java/org/apache/ratis/server/impl/LeaderStateImpl.java +++ b/ratis-server/src/main/java/org/apache/ratis/server/impl/LeaderStateImpl.java @@ -1212,6 +1212,10 @@ long[] getFollowerNextIndices() { return getLogAppenders().mapToLong(s -> s.getFollower().getNextIndex()).toArray(); } + long[] getFollowerMatchIndices() { + return getLogAppenders().mapToLong(s -> s.getFollower().getMatchIndex()).toArray(); + } + static Map newMap(Collection peers, String str) { Objects.requireNonNull(peers, () -> str + " == null"); final Map map = new HashMap<>(); diff --git a/ratis-server/src/main/java/org/apache/ratis/server/impl/RaftServerImpl.java b/ratis-server/src/main/java/org/apache/ratis/server/impl/RaftServerImpl.java index 02e038ef8f..f7a0623c2a 100644 --- a/ratis-server/src/main/java/org/apache/ratis/server/impl/RaftServerImpl.java +++ b/ratis-server/src/main/java/org/apache/ratis/server/impl/RaftServerImpl.java @@ -204,6 +204,14 @@ public long[] getFollowerNextIndices() { .map(LeaderStateImpl::getFollowerNextIndices) .orElse(null); } + + @Override + public long[] getFollowerMatchIndices() { + return role.getLeaderState() + .filter(leader -> isLeader()) + .map(LeaderStateImpl::getFollowerMatchIndices) + .orElse(null); + } } private final RaftServerProxy proxy; From 7fba2f20371a052580d6c80f2d21f9714942eb07 Mon Sep 17 00:00:00 2001 From: "Doroszlai, Attila" <6454655+adoroszlai@users.noreply.github.com> Date: Fri, 17 Jan 2025 07:52:55 +0100 Subject: [PATCH 144/397] RATIS-2239. Publish build scans to develocity.apache.org (#1212) --- .github/workflows/check.yml | 1 + .mvn/develocity.xml | 17 +++++++++++++---- .mvn/extensions.xml | 2 +- 3 files changed, 15 insertions(+), 5 deletions(-) diff --git a/.github/workflows/check.yml b/.github/workflows/check.yml index b4498927ae..78a4be2ad3 100644 --- a/.github/workflows/check.yml +++ b/.github/workflows/check.yml @@ -156,6 +156,7 @@ jobs: run: | dev-support/checks/${{ inputs.script }}.sh ${{ inputs.script-args }} env: + DEVELOCITY_ACCESS_KEY: ${{ secrets.DEVELOCITY_ACCESS_KEY }} WITH_COVERAGE: ${{ inputs.with-coverage }} - name: Summary of failures diff --git a/.mvn/develocity.xml b/.mvn/develocity.xml index 1fa63e6f21..3bef395946 100644 --- a/.mvn/develocity.xml +++ b/.mvn/develocity.xml @@ -22,16 +22,25 @@ + ratis + + https://develocity.apache.org + false + - false - false - false + true + true + true false + #{isFalse(env['GITHUB_ACTIONS'])} - false + + + #{{'0.0.0.0'}} + diff --git a/.mvn/extensions.xml b/.mvn/extensions.xml index 549a1cddcd..8ceede33b9 100644 --- a/.mvn/extensions.xml +++ b/.mvn/extensions.xml @@ -24,7 +24,7 @@ com.gradle develocity-maven-extension - 1.23 + 1.22.2 com.gradle From 1938f8ad30119753db14c1d5adc8c9a84403fb25 Mon Sep 17 00:00:00 2001 From: William Song <48054931+SzyWilliam@users.noreply.github.com> Date: Thu, 23 Jan 2025 00:38:00 +0800 Subject: [PATCH 145/397] RATIS-2240. updatePurgeIndex doesn't need to hold RaftLogBase.writeLock (#1214) --- .../java/org/apache/ratis/server/raftlog/RaftLogBase.java | 6 ++---- 1 file changed, 2 insertions(+), 4 deletions(-) diff --git a/ratis-server/src/main/java/org/apache/ratis/server/raftlog/RaftLogBase.java b/ratis-server/src/main/java/org/apache/ratis/server/raftlog/RaftLogBase.java index b37d40c28a..710dede480 100644 --- a/ratis-server/src/main/java/org/apache/ratis/server/raftlog/RaftLogBase.java +++ b/ratis-server/src/main/java/org/apache/ratis/server/raftlog/RaftLogBase.java @@ -141,10 +141,8 @@ public boolean updateCommitIndex(long majorityIndex, long currentTerm, boolean i } protected void updatePurgeIndex(Long purged) { - try (AutoCloseableLock writeLock = writeLock()) { - if (purged != null) { - purgeIndex.updateToMax(purged, infoIndexChange); - } + if (purged != null) { + purgeIndex.updateToMax(purged, infoIndexChange); } } From 826608e6c41ec60f0cc5186e569ff047a3277638 Mon Sep 17 00:00:00 2001 From: venkatsambath Date: Sun, 2 Feb 2025 23:13:34 -0500 Subject: [PATCH 146/397] RATIS-2244. Reduce the number of log messages during bootstrap (#1217) --- .../ratis/client/impl/OrderedAsync.java | 2 +- .../org/apache/ratis/util/BatchLogger.java | 14 ++++++------- .../ratis/grpc/server/GrpcLogAppender.java | 10 ++++++---- .../server/GrpcServerProtocolService.java | 20 +++++++++++++++++-- .../impl/SnapshotInstallationHandler.java | 20 +++++++++++-------- 5 files changed, 44 insertions(+), 22 deletions(-) diff --git a/ratis-client/src/main/java/org/apache/ratis/client/impl/OrderedAsync.java b/ratis-client/src/main/java/org/apache/ratis/client/impl/OrderedAsync.java index 09c6cd4ac9..1e21b171b3 100644 --- a/ratis-client/src/main/java/org/apache/ratis/client/impl/OrderedAsync.java +++ b/ratis-client/src/main/java/org/apache/ratis/client/impl/OrderedAsync.java @@ -213,7 +213,7 @@ private void sendRequestWithRetry(PendingOrderedRequest pending) { final Throwable exception = e; final String key = client.getId() + "-" + request.getCallId() + "-" + exception; final Consumer op = suffix -> LOG.error("{} {}: Failed* {}", suffix, client.getId(), request, exception); - BatchLogger.warn(BatchLogKey.SEND_REQUEST_EXCEPTION, key, op); + BatchLogger.print(BatchLogKey.SEND_REQUEST_EXCEPTION, key, op); handleException(pending, request, e); return null; }); diff --git a/ratis-common/src/main/java/org/apache/ratis/util/BatchLogger.java b/ratis-common/src/main/java/org/apache/ratis/util/BatchLogger.java index 38dad5c499..b57bed704c 100644 --- a/ratis-common/src/main/java/org/apache/ratis/util/BatchLogger.java +++ b/ratis-common/src/main/java/org/apache/ratis/util/BatchLogger.java @@ -45,9 +45,9 @@ default TimeDuration getBatchDuration() { private static final class UniqueId { private final Key key; - private final String name; + private final Object name; - private UniqueId(Key key, String name) { + private UniqueId(Key key, Object name) { this.key = Objects.requireNonNull(key, "key == null"); this.name = name; } @@ -99,15 +99,15 @@ private synchronized boolean tryStartBatch(Consumer op) { private static final TimeoutExecutor SCHEDULER = TimeoutExecutor.getInstance(); private static final ConcurrentMap LOG_CACHE = new ConcurrentHashMap<>(); - public static void warn(Key key, String name, Consumer op) { - warn(key, name, op, key.getBatchDuration(), true); + public static void print(Key key, Object name, Consumer op) { + print(key, name, op, key.getBatchDuration(), true); } - public static void warn(Key key, String name, Consumer op, TimeDuration batchDuration) { - warn(key, name, op, batchDuration, true); + public static void print(Key key, Object name, Consumer op, TimeDuration batchDuration) { + print(key, name, op, batchDuration, true); } - public static void warn(Key key, String name, Consumer op, TimeDuration batchDuration, boolean shouldBatch) { + public static void print(Key key, Object name, Consumer op, TimeDuration batchDuration, boolean shouldBatch) { if (!shouldBatch || batchDuration.isNonPositive()) { op.accept(""); return; diff --git a/ratis-grpc/src/main/java/org/apache/ratis/grpc/server/GrpcLogAppender.java b/ratis-grpc/src/main/java/org/apache/ratis/grpc/server/GrpcLogAppender.java index 2b14f86f61..3bff1fb35b 100644 --- a/ratis-grpc/src/main/java/org/apache/ratis/grpc/server/GrpcLogAppender.java +++ b/ratis-grpc/src/main/java/org/apache/ratis/grpc/server/GrpcLogAppender.java @@ -66,6 +66,7 @@ public class GrpcLogAppender extends LogAppenderBase { private enum BatchLogKey implements BatchLogger.Key { RESET_CLIENT, + INCONSISTENCY_REPLY, APPEND_LOG_RESPONSE_HANDLER_ON_ERROR } @@ -217,7 +218,7 @@ private void resetClient(AppendEntriesRequest request, Event event) { .orElseGet(f::getMatchIndex); if (event.isError() && request == null) { final long followerNextIndex = f.getNextIndex(); - BatchLogger.warn(BatchLogKey.RESET_CLIENT, f.getId() + "-" + followerNextIndex, suffix -> + BatchLogger.print(BatchLogKey.RESET_CLIENT, f.getId() + "-" + followerNextIndex, suffix -> LOG.warn("{}: Follower failed (request=null, errorCount={}); keep nextIndex ({}) unchanged and retry.{}", this, errorCount, followerNextIndex, suffix), logMessageBatchDuration); return; @@ -516,8 +517,9 @@ private void onNextImpl(AppendEntriesRequest request, AppendEntriesReplyProto re break; case INCONSISTENCY: grpcServerMetrics.onRequestInconsistency(getFollowerId().toString()); - LOG.warn("{}: received {} reply with nextIndex {}, errorCount={}, request={}", - this, reply.getResult(), reply.getNextIndex(), errorCount, request); + BatchLogger.print(BatchLogKey.INCONSISTENCY_REPLY, getFollower().getName() + "_" + reply.getNextIndex(), + suffix -> LOG.warn("{}: received {} reply with nextIndex {}, errorCount={}, request={} {}", + this, reply.getResult(), reply.getNextIndex(), errorCount, request, suffix)); final long requestFirstIndex = request != null? request.getFirstIndex(): RaftLog.INVALID_LOG_INDEX; updateNextIndex(getNextIndexForInconsistency(requestFirstIndex, reply.getNextIndex())); break; @@ -537,7 +539,7 @@ public void onError(Throwable t) { LOG.info("{} is already stopped", GrpcLogAppender.this); return; } - BatchLogger.warn(BatchLogKey.APPEND_LOG_RESPONSE_HANDLER_ON_ERROR, AppendLogResponseHandler.this.name, + BatchLogger.print(BatchLogKey.APPEND_LOG_RESPONSE_HANDLER_ON_ERROR, AppendLogResponseHandler.this.name, suffix -> GrpcUtil.warn(LOG, () -> this + ": Failed appendEntries" + suffix, t), logMessageBatchDuration, t instanceof StatusRuntimeException); grpcServerMetrics.onRequestRetry(); // Update try counter diff --git a/ratis-grpc/src/main/java/org/apache/ratis/grpc/server/GrpcServerProtocolService.java b/ratis-grpc/src/main/java/org/apache/ratis/grpc/server/GrpcServerProtocolService.java index 77d79e64f3..75362fcf88 100644 --- a/ratis-grpc/src/main/java/org/apache/ratis/grpc/server/GrpcServerProtocolService.java +++ b/ratis-grpc/src/main/java/org/apache/ratis/grpc/server/GrpcServerProtocolService.java @@ -29,6 +29,8 @@ import org.apache.ratis.thirdparty.io.grpc.stub.StreamObserver; import org.apache.ratis.proto.RaftProtos.*; import org.apache.ratis.proto.grpc.RaftServerProtocolServiceGrpc.RaftServerProtocolServiceImplBase; +import org.apache.ratis.util.BatchLogger; +import org.apache.ratis.util.MemoizedSupplier; import org.apache.ratis.util.ProtoUtils; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -43,6 +45,11 @@ class GrpcServerProtocolService extends RaftServerProtocolServiceImplBase { public static final Logger LOG = LoggerFactory.getLogger(GrpcServerProtocolService.class); + private enum BatchLogKey implements BatchLogger.Key { + COMPLETED_REQUEST, + COMPLETED_REPLY + } + static class PendingServerRequest { private final REQUEST request; private final CompletableFuture future = new CompletableFuture<>(); @@ -62,6 +69,7 @@ CompletableFuture getFuture() { abstract class ServerRequestStreamObserver implements StreamObserver { private final RaftServer.Op op; + private final Supplier nameSupplier; private final StreamObserver responseObserver; /** For ordered {@link #onNext(Object)} requests. */ private final AtomicReference> previousOnNext = new AtomicReference<>(); @@ -72,9 +80,14 @@ abstract class ServerRequestStreamObserver implements StreamObse ServerRequestStreamObserver(RaftServer.Op op, StreamObserver responseObserver) { this.op = op; + this.nameSupplier = MemoizedSupplier.valueOf(() -> getId() + "_" + op); this.responseObserver = responseObserver; } + String getName() { + return nameSupplier.get(); + } + private String getPreviousRequestString() { return Optional.ofNullable(previousOnNext.get()) .map(PendingServerRequest::getRequest) @@ -154,9 +167,12 @@ public void onNext(REQUEST request) { @Override public void onCompleted() { if (isClosed.compareAndSet(false, true)) { - LOG.info("{}: Completed {}, lastRequest: {}", getId(), op, getPreviousRequestString()); + BatchLogger.print(BatchLogKey.COMPLETED_REQUEST, getName(), + suffix -> LOG.info("{}: Completed {}, lastRequest: {} {}", + getId(), op, getPreviousRequestString(), suffix)); requestFuture.get().thenAccept(reply -> { - LOG.info("{}: Completed {}, lastReply: {}", getId(), op, reply); + BatchLogger.print(BatchLogKey.COMPLETED_REPLY, getName(), + suffix -> LOG.info("{}: Completed {}, lastReply: {} {}", getId(), op, reply, suffix)); responseObserver.onCompleted(); }); } diff --git a/ratis-server/src/main/java/org/apache/ratis/server/impl/SnapshotInstallationHandler.java b/ratis-server/src/main/java/org/apache/ratis/server/impl/SnapshotInstallationHandler.java index 70027e6dda..4f1ac4177f 100644 --- a/ratis-server/src/main/java/org/apache/ratis/server/impl/SnapshotInstallationHandler.java +++ b/ratis-server/src/main/java/org/apache/ratis/server/impl/SnapshotInstallationHandler.java @@ -36,6 +36,7 @@ import org.apache.ratis.server.protocol.TermIndex; import org.apache.ratis.server.raftlog.LogProtoUtils; import org.apache.ratis.server.util.ServerStringUtils; +import org.apache.ratis.util.BatchLogger; import org.apache.ratis.util.CodeInjectionForTesting; import org.apache.ratis.util.LifeCycle; import org.apache.ratis.util.Preconditions; @@ -59,6 +60,11 @@ class SnapshotInstallationHandler { static final Logger LOG = LoggerFactory.getLogger(SnapshotInstallationHandler.class); + private enum BatchLogKey implements BatchLogger.Key { + INSTALL_SNAPSHOT_REQUEST, + INSTALL_SNAPSHOT_REPLY + } + static final TermIndex INVALID_TERM_INDEX = TermIndex.valueOf(0, INVALID_LOG_INDEX); private final RaftServerImpl server; @@ -93,10 +99,9 @@ long getInProgressInstallSnapshotIndex() { } InstallSnapshotReplyProto installSnapshot(InstallSnapshotRequestProto request) throws IOException { - if (LOG.isInfoEnabled()) { - LOG.info("{}: receive installSnapshot: {}", getMemberId(), - ServerStringUtils.toInstallSnapshotRequestString(request)); - } + BatchLogger.print(BatchLogKey.INSTALL_SNAPSHOT_REQUEST, getMemberId(), + suffix -> LOG.info("{}: receive installSnapshot: {} {}", + getMemberId(), ServerStringUtils.toInstallSnapshotRequestString(request), suffix)); final InstallSnapshotReplyProto reply; try { reply = installSnapshotImpl(request); @@ -104,10 +109,9 @@ InstallSnapshotReplyProto installSnapshot(InstallSnapshotRequestProto request) t LOG.error("{}: installSnapshot failed", getMemberId(), e); throw e; } - if (LOG.isInfoEnabled()) { - LOG.info("{}: reply installSnapshot: {}", getMemberId(), - ServerStringUtils.toInstallSnapshotReplyString(reply)); - } + BatchLogger.print(BatchLogKey.INSTALL_SNAPSHOT_REPLY, getMemberId(), + suffix -> LOG.info("{}: reply installSnapshot: {} {}", + getMemberId(), ServerStringUtils.toInstallSnapshotReplyString(reply), suffix)); return reply; } From cf7407aa807fa43bf4975e72c30f7b62dd51a423 Mon Sep 17 00:00:00 2001 From: "Doroszlai, Attila" <6454655+adoroszlai@users.noreply.github.com> Date: Tue, 11 Feb 2025 08:04:26 +0100 Subject: [PATCH 147/397] RATIS-2247. Bump GitHub Actions runner to ubuntu-24.04 (#1219) --- .github/workflows/check.yml | 2 +- .github/workflows/ci.yml | 2 +- .github/workflows/repeat-test.yml | 6 +++--- 3 files changed, 5 insertions(+), 5 deletions(-) diff --git a/.github/workflows/check.yml b/.github/workflows/check.yml index 78a4be2ad3..be5718f7ef 100644 --- a/.github/workflows/check.yml +++ b/.github/workflows/check.yml @@ -55,7 +55,7 @@ on: runner: type: string description: "GitHub Actions runner to use" - default: 'ubuntu-20.04' + default: 'ubuntu-24.04' required: false script-args: diff --git a/.github/workflows/ci.yml b/.github/workflows/ci.yml index 11fb1bd515..986bc7128c 100644 --- a/.github/workflows/ci.yml +++ b/.github/workflows/ci.yml @@ -96,7 +96,7 @@ jobs: needs: - build - unit - runs-on: ubuntu-20.04 + runs-on: ubuntu-24.04 timeout-minutes: 30 if: (github.repository == 'apache/ratis' || github.repository == 'apache/incubator-ratis') && github.event_name != 'pull_request' steps: diff --git a/.github/workflows/repeat-test.yml b/.github/workflows/repeat-test.yml index f431b1a0fe..66fec90f16 100644 --- a/.github/workflows/repeat-test.yml +++ b/.github/workflows/repeat-test.yml @@ -50,7 +50,7 @@ env: run-name: ${{ github.event_name == 'workflow_dispatch' && format('{0}#{1}[{2}]-{3}x{4}', inputs.test-class, inputs.test-method, inputs.ref, inputs.splits, inputs.iterations) || '' }} jobs: prepare: - runs-on: ubuntu-20.04 + runs-on: ubuntu-24.04 outputs: matrix: ${{ steps.generate.outputs.matrix }} test-spec: ${{ steps.test-spec.outputs.test-spec }} @@ -79,7 +79,7 @@ jobs: if: ${{ always() }} needs: - prepare - runs-on: ubuntu-20.04 + runs-on: ubuntu-24.04 env: TEST_SPEC: ${{ needs.prepare.outputs.test-spec }} strategy: @@ -121,7 +121,7 @@ jobs: count-failures: if: ${{ failure() }} needs: test - runs-on: ubuntu-20.04 + runs-on: ubuntu-24.04 steps: - name: Download build results uses: actions/download-artifact@v4 From 89cddb8aebb7fff8993178bd1261051bc4870f09 Mon Sep 17 00:00:00 2001 From: "Doroszlai, Attila" <6454655+adoroszlai@users.noreply.github.com> Date: Tue, 11 Feb 2025 19:41:08 +0100 Subject: [PATCH 148/397] RATIS-2248. Rename CI workflows to .yaml (#1220) --- .github/workflows/{check.yml => check.yaml} | 0 .github/workflows/{ci.yml => ci.yaml} | 10 +++++----- .../workflows/{post-commit.yml => post-commit.yaml} | 2 +- .../workflows/{repeat-test.yml => repeat-test.yaml} | 0 4 files changed, 6 insertions(+), 6 deletions(-) rename .github/workflows/{check.yml => check.yaml} (100%) rename .github/workflows/{ci.yml => ci.yaml} (94%) rename .github/workflows/{post-commit.yml => post-commit.yaml} (96%) rename .github/workflows/{repeat-test.yml => repeat-test.yaml} (100%) diff --git a/.github/workflows/check.yml b/.github/workflows/check.yaml similarity index 100% rename from .github/workflows/check.yml rename to .github/workflows/check.yaml diff --git a/.github/workflows/ci.yml b/.github/workflows/ci.yaml similarity index 94% rename from .github/workflows/ci.yml rename to .github/workflows/ci.yaml index 986bc7128c..55d55336b5 100644 --- a/.github/workflows/ci.yml +++ b/.github/workflows/ci.yaml @@ -26,7 +26,7 @@ on: jobs: build: - uses: ./.github/workflows/check.yml + uses: ./.github/workflows/check.yaml with: script: build script-args: -Prelease @@ -40,7 +40,7 @@ jobs: matrix: java: [ 11, 17, 21 ] fail-fast: false - uses: ./.github/workflows/check.yml + uses: ./.github/workflows/check.yaml with: java-version: ${{ matrix.java }} needs-source-tarball: true @@ -53,7 +53,7 @@ jobs: repro: needs: - build - uses: ./.github/workflows/check.yml + uses: ./.github/workflows/check.yaml with: needs-maven-repo: true script: repro @@ -70,7 +70,7 @@ jobs: - findbugs - rat fail-fast: false - uses: ./.github/workflows/check.yml + uses: ./.github/workflows/check.yaml with: script: ${{ matrix.check }} timeout-minutes: 30 @@ -84,7 +84,7 @@ jobs: - server - misc fail-fast: false - uses: ./.github/workflows/check.yml + uses: ./.github/workflows/check.yaml with: script: unit script-args: -P${{ matrix.profile }}-tests diff --git a/.github/workflows/post-commit.yml b/.github/workflows/post-commit.yaml similarity index 96% rename from .github/workflows/post-commit.yml rename to .github/workflows/post-commit.yaml index 9b0df4ce40..583eb7bcf5 100644 --- a/.github/workflows/post-commit.yml +++ b/.github/workflows/post-commit.yaml @@ -29,5 +29,5 @@ concurrency: jobs: CI: - uses: ./.github/workflows/ci.yml + uses: ./.github/workflows/ci.yaml secrets: inherit diff --git a/.github/workflows/repeat-test.yml b/.github/workflows/repeat-test.yaml similarity index 100% rename from .github/workflows/repeat-test.yml rename to .github/workflows/repeat-test.yaml From fd6be6c9c6373e3600610a19c5f7003c60bea7c6 Mon Sep 17 00:00:00 2001 From: "Doroszlai, Attila" <6454655+adoroszlai@users.noreply.github.com> Date: Tue, 11 Feb 2025 21:14:33 +0100 Subject: [PATCH 149/397] RATIS-2249. Remove unused shellcheck.sh (#1221) --- dev-support/checks/shellcheck.sh | 37 -------------------------------- 1 file changed, 37 deletions(-) delete mode 100755 dev-support/checks/shellcheck.sh diff --git a/dev-support/checks/shellcheck.sh b/dev-support/checks/shellcheck.sh deleted file mode 100755 index feb307a2db..0000000000 --- a/dev-support/checks/shellcheck.sh +++ /dev/null @@ -1,37 +0,0 @@ -#!/usr/bin/env bash -# 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. -DIR="$( cd "$( dirname "${BASH_SOURCE[0]}" )" >/dev/null 2>&1 && pwd )" -cd "$DIR/../.." || exit 1 - -REPORT_DIR=${OUTPUT_DIR:-"$DIR/../../target/shellcheck"} -mkdir -p "$REPORT_DIR" -REPORT_FILE="$REPORT_DIR/summary.txt" - -echo "" > "$OUTPUT_FILE" -if [[ "$(uname -s)" = "Darwin" ]]; then - find . -type f -perm '-500' -else - find . -type f -executable -fi \ - | grep -v -e target/ -e node_modules/ -e '\.\(ico\|py\|yml\)$' \ - | xargs -n1 shellcheck \ - | tee "$REPORT_FILE" - -wc -l "$REPORT_FILE" | awk '{print $1}'> "$REPORT_DIR/failures" - -if [[ -s "${REPORT_FILE}" ]]; then - exit 1 -fi From 4ffa60ac33c0d29fd27bd264d856f00a11190eed Mon Sep 17 00:00:00 2001 From: Swaminathan Balachandran <47532440+swamirishi@users.noreply.github.com> Date: Wed, 19 Feb 2025 15:06:25 -0800 Subject: [PATCH 150/397] RATIS-2245. Ratis should wait for all apply transaction futures before taking snapshot and group remove (#1218) --- .../server/impl/StateMachineUpdater.java | 38 +++--- .../impl/StateMachineShutdownTests.java | 128 ++++++++++++++---- 2 files changed, 119 insertions(+), 47 deletions(-) diff --git a/ratis-server/src/main/java/org/apache/ratis/server/impl/StateMachineUpdater.java b/ratis-server/src/main/java/org/apache/ratis/server/impl/StateMachineUpdater.java index a919ca732c..3dfe5e0aa9 100644 --- a/ratis-server/src/main/java/org/apache/ratis/server/impl/StateMachineUpdater.java +++ b/ratis-server/src/main/java/org/apache/ratis/server/impl/StateMachineUpdater.java @@ -37,8 +37,6 @@ import org.slf4j.LoggerFactory; import java.io.IOException; -import java.util.ArrayList; -import java.util.List; import java.util.Objects; import java.util.Optional; import java.util.concurrent.CompletableFuture; @@ -182,19 +180,20 @@ public String toString() { @Override public void run() { + CompletableFuture applyLogFutures = CompletableFuture.completedFuture(null); for(; state != State.STOP; ) { try { - waitForCommit(); + waitForCommit(applyLogFutures); if (state == State.RELOAD) { reload(); } - final MemoizedSupplier>> futures = applyLog(); - checkAndTakeSnapshot(futures); + applyLogFutures = applyLog(applyLogFutures); + checkAndTakeSnapshot(applyLogFutures); if (shouldStop()) { - checkAndTakeSnapshot(futures); + applyLogFutures.get(); stop(); } } catch (Throwable t) { @@ -210,14 +209,14 @@ public void run() { } } - private void waitForCommit() throws InterruptedException { + private void waitForCommit(CompletableFuture applyLogFutures) throws InterruptedException, ExecutionException { // When a peer starts, the committed is initialized to 0. // It will be updated only after the leader contacts other peers. // Thus it is possible to have applied > committed initially. final long applied = getLastAppliedIndex(); for(; applied >= raftLog.getLastCommittedIndex() && state == State.RUNNING && !shouldStop(); ) { if (server.getSnapshotRequestHandler().shouldTriggerTakingSnapshot()) { - takeSnapshot(); + takeSnapshot(applyLogFutures); } if (awaitForSignal.await(100, TimeUnit.MILLISECONDS)) { return; @@ -239,8 +238,7 @@ private void reload() throws IOException { state = State.RUNNING; } - private MemoizedSupplier>> applyLog() throws RaftLogIOException { - final MemoizedSupplier>> futures = MemoizedSupplier.valueOf(ArrayList::new); + private CompletableFuture applyLog(CompletableFuture applyLogFutures) throws RaftLogIOException { final long committed = raftLog.getLastCommittedIndex(); for(long applied; (applied = getLastAppliedIndex()) < committed && state == State.RUNNING && !shouldStop(); ) { final long nextIndex = applied + 1; @@ -256,7 +254,12 @@ private MemoizedSupplier>> applyLog() throws Raf final long incremented = appliedIndex.incrementAndGet(debugIndexChange); Preconditions.assertTrue(incremented == nextIndex); if (f != null) { - futures.get().add(f); + CompletableFuture exceptionHandledFuture = f.exceptionally(ex -> { + LOG.error("Exception while {}: applying txn index={}, nextLog={}", this, nextIndex, + LogProtoUtils.toLogEntryString(next), ex); + return null; + }); + applyLogFutures = applyLogFutures.thenCombine(exceptionHandledFuture, (v, message) -> null); f.thenAccept(m -> notifyAppliedIndex(incremented)); } else { notifyAppliedIndex(incremented); @@ -267,23 +270,20 @@ private MemoizedSupplier>> applyLog() throws Raf break; } } - return futures; + return applyLogFutures; } - private void checkAndTakeSnapshot(MemoizedSupplier>> futures) + private void checkAndTakeSnapshot(CompletableFuture futures) throws ExecutionException, InterruptedException { // check if need to trigger a snapshot if (shouldTakeSnapshot()) { - if (futures.isInitialized()) { - JavaUtils.allOf(futures.get()).get(); - } - - takeSnapshot(); + takeSnapshot(futures); } } - private void takeSnapshot() { + private void takeSnapshot(CompletableFuture applyLogFutures) throws ExecutionException, InterruptedException { final long i; + applyLogFutures.get(); try { try(UncheckedAutoCloseable ignored = Timekeeper.start(stateMachineMetrics.get().getTakeSnapshotTimer())) { i = stateMachine.takeSnapshot(); diff --git a/ratis-server/src/test/java/org/apache/ratis/server/impl/StateMachineShutdownTests.java b/ratis-server/src/test/java/org/apache/ratis/server/impl/StateMachineShutdownTests.java index 28f8e6ace9..c70464a18c 100644 --- a/ratis-server/src/test/java/org/apache/ratis/server/impl/StateMachineShutdownTests.java +++ b/ratis-server/src/test/java/org/apache/ratis/server/impl/StateMachineShutdownTests.java @@ -28,47 +28,106 @@ import org.apache.ratis.statemachine.impl.SimpleStateMachine4Testing; import org.apache.ratis.statemachine.StateMachine; import org.apache.ratis.statemachine.TransactionContext; -import org.junit.Assert; -import org.junit.Test; - -import java.util.concurrent.CompletableFuture; +import org.junit.*; +import org.mockito.MockedStatic; +import org.mockito.Mockito; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; +import java.util.*; +import java.util.concurrent.*; +import java.util.concurrent.atomic.AtomicLong; public abstract class StateMachineShutdownTests extends BaseTest implements MiniRaftCluster.Factory.Get { - + public static Logger LOG = LoggerFactory.getLogger(StateMachineUpdater.class); + private static MockedStatic mocked; protected static class StateMachineWithConditionalWait extends SimpleStateMachine4Testing { + boolean unblockAllTxns = false; + final Set blockTxns = ConcurrentHashMap.newKeySet(); + private final ExecutorService executor = Executors.newFixedThreadPool(10); + public static Map>> futures = new ConcurrentHashMap<>(); + public static Map numTxns = new ConcurrentHashMap<>(); + private final Map appliedTxns = new ConcurrentHashMap<>(); + + private synchronized void updateTxns() { + long appliedIndex = this.getLastAppliedTermIndex().getIndex() + 1; + Long appliedTerm = null; + while (appliedTxns.containsKey(appliedIndex)) { + appliedTerm = appliedTxns.remove(appliedIndex); + appliedIndex += 1; + } + if (appliedTerm != null) { + updateLastAppliedTermIndex(appliedTerm, appliedIndex - 1); + } + } - private final Long objectToWait = 0L; - volatile boolean blockOnApply = true; + @Override + public void notifyTermIndexUpdated(long term, long index) { + appliedTxns.put(index, term); + updateTxns(); + } @Override public CompletableFuture applyTransaction(TransactionContext trx) { - if (blockOnApply) { - synchronized (objectToWait) { - try { - objectToWait.wait(); - } catch (InterruptedException e) { - Thread.currentThread().interrupt(); - throw new RuntimeException(); + final RaftProtos.LogEntryProto entry = trx.getLogEntry(); + + CompletableFuture future = new CompletableFuture<>(); + futures.computeIfAbsent(Thread.currentThread().getId(), k -> new HashSet<>()).add(future); + executor.submit(() -> { + synchronized (blockTxns) { + if (!unblockAllTxns) { + blockTxns.add(entry.getIndex()); + } + while (!unblockAllTxns && blockTxns.contains(entry.getIndex())) { + try { + blockTxns.wait(10000); + } catch (InterruptedException e) { + throw new RuntimeException(e); + } } } + numTxns.computeIfAbsent(getId(), (k) -> new AtomicLong()).incrementAndGet(); + appliedTxns.put(entry.getIndex(), entry.getTerm()); + updateTxns(); + future.complete(new RaftTestUtil.SimpleMessage("done")); + }); + return future; + } + + public void unBlockApplyTxn(long txnId) { + synchronized (blockTxns) { + blockTxns.remove(txnId); + blockTxns.notifyAll(); } - RaftProtos.LogEntryProto entry = trx.getLogEntry(); - updateLastAppliedTermIndex(entry.getTerm(), entry.getIndex()); - return CompletableFuture.completedFuture(new RaftTestUtil.SimpleMessage("done")); } - public void unBlockApplyTxn() { - blockOnApply = false; - synchronized (objectToWait) { - objectToWait.notifyAll(); + public void unblockAllTxns() { + unblockAllTxns = true; + synchronized (blockTxns) { + for (Long txnId : blockTxns) { + blockTxns.remove(txnId); + } + blockTxns.notifyAll(); } } } + @Before + public void setup() { + mocked = Mockito.mockStatic(CompletableFuture.class, Mockito.CALLS_REAL_METHODS); + } + + @After + public void tearDownClass() { + if (mocked != null) { + mocked.close(); + } + + } + @Test public void testStateMachineShutdownWaitsForApplyTxn() throws Exception { final RaftProperties prop = getProperties(); @@ -82,10 +141,9 @@ public void testStateMachineShutdownWaitsForApplyTxn() throws Exception { //Unblock leader and one follower ((StateMachineWithConditionalWait)leader.getStateMachine()) - .unBlockApplyTxn(); + .unblockAllTxns(); ((StateMachineWithConditionalWait)cluster. - getFollowers().get(0).getStateMachine()).unBlockApplyTxn(); - + getFollowers().get(0).getStateMachine()).unblockAllTxns(); cluster.getLeaderAndSendFirstMessage(true); try (final RaftClient client = cluster.createClient(leaderId)) { @@ -107,16 +165,30 @@ public void testStateMachineShutdownWaitsForApplyTxn() throws Exception { final Thread t = new Thread(secondFollower::close); t.start(); - // The second follower should still be blocked in apply transaction - Assert.assertTrue(secondFollower.getInfo().getLastAppliedIndex() < logIndex); + // Now unblock the second follower - ((StateMachineWithConditionalWait) secondFollower.getStateMachine()) - .unBlockApplyTxn(); + long minIndex = ((StateMachineWithConditionalWait) secondFollower.getStateMachine()).blockTxns.stream() + .min(Comparator.naturalOrder()).get(); + Assert.assertEquals(2, StateMachineWithConditionalWait.numTxns.values().stream() + .filter(val -> val.get() == 3).count()); + // The second follower should still be blocked in apply transaction + Assert.assertTrue(secondFollower.getInfo().getLastAppliedIndex() < minIndex); + for (long index : ((StateMachineWithConditionalWait) secondFollower.getStateMachine()).blockTxns) { + if (minIndex != index) { + ((StateMachineWithConditionalWait) secondFollower.getStateMachine()).unBlockApplyTxn(index); + } + } + Assert.assertEquals(2, StateMachineWithConditionalWait.numTxns.values().stream() + .filter(val -> val.get() == 3).count()); + Assert.assertTrue(secondFollower.getInfo().getLastAppliedIndex() < minIndex); + ((StateMachineWithConditionalWait) secondFollower.getStateMachine()).unBlockApplyTxn(minIndex); // Now wait for the thread t.join(5000); Assert.assertEquals(logIndex, secondFollower.getInfo().getLastAppliedIndex()); + Assert.assertEquals(3, StateMachineWithConditionalWait.numTxns.values().stream() + .filter(val -> val.get() == 3).count()); cluster.shutdown(); } From e0dc9d5e3e6e517b325e4a073b1314b3da9b42c4 Mon Sep 17 00:00:00 2001 From: William Song <48054931+SzyWilliam@users.noreply.github.com> Date: Fri, 21 Feb 2025 03:38:50 +0800 Subject: [PATCH 151/397] RATIS-2242. change consistency criteria of heartbeat during appendLog (#1215) --- .../ratis/server/impl/RaftServerImpl.java | 15 ++- .../ratis/server/impl/RaftServerProxy.java | 3 +- .../ratis/server/impl/ServerImplUtils.java | 110 ++++++++++++++++++ 3 files changed, 123 insertions(+), 5 deletions(-) diff --git a/ratis-server/src/main/java/org/apache/ratis/server/impl/RaftServerImpl.java b/ratis-server/src/main/java/org/apache/ratis/server/impl/RaftServerImpl.java index f7a0623c2a..0186841efb 100644 --- a/ratis-server/src/main/java/org/apache/ratis/server/impl/RaftServerImpl.java +++ b/ratis-server/src/main/java/org/apache/ratis/server/impl/RaftServerImpl.java @@ -17,7 +17,6 @@ */ package org.apache.ratis.server.impl; -import java.util.concurrent.CountDownLatch; import org.apache.ratis.client.impl.ClientProtoUtils; import org.apache.ratis.conf.RaftProperties; import org.apache.ratis.metrics.Timekeeper; @@ -25,11 +24,11 @@ import org.apache.ratis.proto.RaftProtos.AppendEntriesReplyProto.AppendResult; import org.apache.ratis.proto.RaftProtos.AppendEntriesRequestProto; import org.apache.ratis.proto.RaftProtos.CommitInfoProto; -import org.apache.ratis.proto.RaftProtos.LogInfoProto; import org.apache.ratis.proto.RaftProtos.InstallSnapshotReplyProto; import org.apache.ratis.proto.RaftProtos.InstallSnapshotRequestProto; import org.apache.ratis.proto.RaftProtos.InstallSnapshotResult; import org.apache.ratis.proto.RaftProtos.LogEntryProto; +import org.apache.ratis.proto.RaftProtos.LogInfoProto; import org.apache.ratis.proto.RaftProtos.RaftClientRequestProto; import org.apache.ratis.proto.RaftProtos.RaftClientRequestProto.TypeCase; import org.apache.ratis.proto.RaftProtos.RaftConfigurationProto; @@ -82,6 +81,8 @@ import org.apache.ratis.server.RaftServerRpc; import org.apache.ratis.server.impl.LeaderElection.Phase; import org.apache.ratis.server.impl.RetryCacheImpl.CacheEntry; +import org.apache.ratis.server.impl.ServerImplUtils.ConsecutiveIndices; +import org.apache.ratis.server.impl.ServerImplUtils.NavigableIndices; import org.apache.ratis.server.leader.LeaderState; import org.apache.ratis.server.metrics.LeaderElectionMetrics; import org.apache.ratis.server.metrics.RaftServerMetricsImpl; @@ -126,6 +127,7 @@ import java.util.Set; import java.util.concurrent.CompletableFuture; import java.util.concurrent.CompletionException; +import java.util.concurrent.CountDownLatch; import java.util.concurrent.ExecutionException; import java.util.concurrent.ExecutorService; import java.util.concurrent.ThreadLocalRandom; @@ -258,6 +260,7 @@ public long[] getFollowerMatchIndices() { private final ThreadGroup threadGroup; private final AtomicReference> appendLogFuture; + private final NavigableIndices appendLogTermIndices = new NavigableIndices(); RaftServerImpl(RaftGroup group, StateMachine stateMachine, RaftServerProxy proxy, RaftStorage.StartupOption option) throws IOException { @@ -1630,8 +1633,12 @@ leaderId, getMemberId(), currentTerm, followerCommit, inconsistencyReplyNextInde }); } private CompletableFuture appendLog(List entries) { + final List entriesTermIndices = ConsecutiveIndices.convert(entries); + appendLogTermIndices.append(entriesTermIndices); + return appendLogFuture.updateAndGet(f -> f.thenCompose( - ignored -> JavaUtils.allOf(state.getLog().append(entries)))); + ignored -> JavaUtils.allOf(state.getLog().append(entries)))) + .whenComplete((v, e) -> appendLogTermIndices.removeExisting(entriesTermIndices)); } private long checkInconsistentAppendEntries(TermIndex previous, List entries) { @@ -1658,7 +1665,7 @@ private long checkInconsistentAppendEntries(TermIndex previous, List appendEntriesAsync(AppendEntriesRequestProto request) { final RaftGroupId groupId = ProtoUtils.toRaftGroupId(request.getServerRequest().getRaftGroupId()); return getImplFuture(groupId) - .thenCompose(impl -> impl.executeSubmitServerRequestAsync(() -> impl.appendEntriesAsync(request))); + .thenCompose(impl -> JavaUtils.callAsUnchecked( + () -> impl.appendEntriesAsync(request), CompletionException::new)); } @Override diff --git a/ratis-server/src/main/java/org/apache/ratis/server/impl/ServerImplUtils.java b/ratis-server/src/main/java/org/apache/ratis/server/impl/ServerImplUtils.java index e26c6e0ab1..c5010a5346 100644 --- a/ratis-server/src/main/java/org/apache/ratis/server/impl/ServerImplUtils.java +++ b/ratis-server/src/main/java/org/apache/ratis/server/impl/ServerImplUtils.java @@ -40,11 +40,121 @@ import org.apache.ratis.util.TimeDuration; import java.io.IOException; +import java.util.ArrayList; +import java.util.Collections; import java.util.List; +import java.util.Map; +import java.util.NavigableMap; +import java.util.TreeMap; import java.util.concurrent.TimeUnit; /** Server utilities for internal use. */ public final class ServerImplUtils { + /** The consecutive indices within the same term. */ + static class ConsecutiveIndices { + /** Convert the given entries to a list of {@link ConsecutiveIndices} */ + static List convert(List entries) { + if (entries == null || entries.isEmpty()) { + return Collections.emptyList(); + } + + List indices = null; + + LogEntryProto previous = entries.get(0); + long startIndex = previous.getIndex(); + int count = 1; + + for (int i = 1; i < entries.size(); i++) { + final LogEntryProto current = entries.get(i); + // validate if the indices are consecutive + Preconditions.assertSame(previous.getIndex() + 1, current.getIndex(), "index"); + + if (current.getTerm() == previous.getTerm()) { + count++; + } else { + // validate if the terms are increasing + Preconditions.assertTrue(previous.getTerm() < current.getTerm(), "term"); + if (indices == null) { + indices = new ArrayList<>(); + } + indices.add(new ConsecutiveIndices(previous.getTerm(), startIndex, count)); + + startIndex = current.getIndex(); + count = 1; + } + previous = current; + } + + final ConsecutiveIndices last = new ConsecutiveIndices(previous.getTerm(), startIndex, count); + if (indices == null) { + return Collections.singletonList(last); + } else { + indices.add(last); + return indices; + } + } + + private final long term; + private final long startIndex; + private final int count; + + ConsecutiveIndices(long term, long startIndex, int count) { + Preconditions.assertTrue(count > 0, () -> "count = " + count + " <= 0 "); + this.term = term; + this.startIndex = startIndex; + this.count = count; + } + + long getNextIndex() { + return startIndex + count; + } + + Long getTerm(long index) { + final long diff = index - startIndex; + return diff < 0 || diff >= count ? null: term; + } + } + + /** A data structure to support the {@link #contains(TermIndex)} method. */ + static class NavigableIndices { + private final NavigableMap map = new TreeMap<>(); + + boolean contains(TermIndex ti) { + final Long term = getTerm(ti.getIndex()); + return term != null && term == ti.getTerm(); + } + + synchronized Long getTerm(long index) { + if (map.isEmpty()) { + return null; + } + + final Map.Entry floorEntry = map.floorEntry(index); + if (floorEntry == null) { + return null; + } + return floorEntry.getValue().getTerm(index); + } + + synchronized void append(List entriesTermIndices) { + for(ConsecutiveIndices indices : entriesTermIndices) { + // validate startIndex + final Map.Entry lastEntry = map.lastEntry(); + if (lastEntry != null) { + Preconditions.assertSame(lastEntry.getValue().getNextIndex(), indices.startIndex, "startIndex"); + } + map.put(indices.startIndex, indices); + } + } + + synchronized void removeExisting(List entriesTermIndices) { + for(ConsecutiveIndices indices : entriesTermIndices) { + final ConsecutiveIndices removed = map.remove(indices.startIndex); + Preconditions.assertSame(indices, removed, "removed"); + } + } + } + private ServerImplUtils() { //Never constructed } From 48ac97e9057ac49883fc0f2aa577b33f7d246189 Mon Sep 17 00:00:00 2001 From: Tsz-Wo Nicholas Sze Date: Wed, 26 Feb 2025 00:15:55 -0800 Subject: [PATCH 152/397] RATIS-2252. Replace JUnitRunListener with JUnit5TestExecutionListener (#1228) --- pom.xml | 6 -- .../ratis/JUnit5TestExecutionListener.java | 51 +++++++++++ .../org/apache/ratis/JUnitRunListener.java | 91 ------------------- ...it.platform.launcher.TestExecutionListener | 18 ++++ 4 files changed, 69 insertions(+), 97 deletions(-) create mode 100644 ratis-common/src/test/java/org/apache/ratis/JUnit5TestExecutionListener.java delete mode 100644 ratis-common/src/test/java/org/apache/ratis/JUnitRunListener.java create mode 100644 ratis-common/src/test/resources/META-INF/services/org.junit.platform.launcher.TestExecutionListener diff --git a/pom.xml b/pom.xml index 48343fa939..29b2137e42 100644 --- a/pom.xml +++ b/pom.xml @@ -654,12 +654,6 @@ **/Test*$*.java ${test.exclude.pattern} - - - listener - org.apache.ratis.JUnitRunListener - - diff --git a/ratis-common/src/test/java/org/apache/ratis/JUnit5TestExecutionListener.java b/ratis-common/src/test/java/org/apache/ratis/JUnit5TestExecutionListener.java new file mode 100644 index 0000000000..535d8968cb --- /dev/null +++ b/ratis-common/src/test/java/org/apache/ratis/JUnit5TestExecutionListener.java @@ -0,0 +1,51 @@ +/* + * 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.ratis; + +import org.apache.ratis.util.JavaUtils; +import org.junit.platform.engine.TestExecutionResult; +import org.junit.platform.launcher.TestExecutionListener; +import org.junit.platform.launcher.TestIdentifier; + +import java.io.PrintStream; +import java.util.concurrent.TimeoutException; + +/** + * A {@link TestExecutionListener} to dump all threads after a test timeout failure. + */ +public class JUnit5TestExecutionListener implements TestExecutionListener { + private final PrintStream out = System.out; + + @Override + public void executionFinished(TestIdentifier id, TestExecutionResult result) { + final Throwable timeoutException = getTimeoutException(result); + if (timeoutException != null) { + out.format("%n%s %s failed%n", JavaUtils.date(), id.getDisplayName()); + timeoutException.printStackTrace(out); + JavaUtils.dumpAllThreads(out::println); + } + } + + private static Throwable getTimeoutException(TestExecutionResult result) { + if (result == null) { + return null; + } + final Throwable throwable = result.getThrowable().orElse(null); + return throwable instanceof TimeoutException? throwable : null; + } +} diff --git a/ratis-common/src/test/java/org/apache/ratis/JUnitRunListener.java b/ratis-common/src/test/java/org/apache/ratis/JUnitRunListener.java deleted file mode 100644 index 144c8069c6..0000000000 --- a/ratis-common/src/test/java/org/apache/ratis/JUnitRunListener.java +++ /dev/null @@ -1,91 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ -package org.apache.ratis; - -import org.apache.ratis.util.JavaUtils; -import org.junit.internal.runners.statements.FailOnTimeout; -import org.junit.runner.notification.Failure; -import org.junit.runner.notification.RunListener; -import org.junit.runners.model.Statement; -import org.junit.runners.model.TestTimedOutException; - -import java.io.PrintStream; -import java.util.concurrent.TimeUnit; - -/** - * A {@link RunListener} to dump all threads after a test timeout failure. - */ -public class JUnitRunListener extends RunListener { - private static final Throwable TIMEOUT_EXCEPTION = getTimeoutException(); - private static final String TIMEOUT_EXCEPTION_PREFIX; - - private static Throwable getTimeoutException() { - final FailOnTimeout f = FailOnTimeout.builder().withTimeout(1, TimeUnit.NANOSECONDS).build(new Statement() { - @Override - public void evaluate() throws InterruptedException { - Thread.sleep(1000); - } - }); - try { - f.evaluate(); - } catch(Throwable throwable) { - return throwable; - } - throw new IllegalStateException("Failed to getTimeoutException"); - } - - static { - final String message = JUnitRunListener.TIMEOUT_EXCEPTION.getMessage(); - TIMEOUT_EXCEPTION_PREFIX = message.substring(0, message.indexOf('1')); - } - - private final PrintStream out = System.out; - - @Override - public void testFailure(Failure failure) { - final Throwable timeoutException = getTimeoutException(failure); - if (timeoutException != null) { - out.format("%n%s ", JavaUtils.date()); - timeoutException.printStackTrace(out); - JavaUtils.dumpAllThreads(out::println); - } - } - - private static Throwable getTimeoutException(Failure failure) { - if (failure == null) { - return null; - } - final Throwable throwable = failure.getException(); - if (throwable.getClass() != TIMEOUT_EXCEPTION.getClass()) { - return null; - } - final String message = throwable.getMessage(); - if (message == null || !message.startsWith(TIMEOUT_EXCEPTION_PREFIX)) { - return null; - } - return throwable; - } - - public static void main(String[] args) { - final JUnitRunListener listener = new JUnitRunListener(); - listener.out.println("TIMEOUT_EXCEPTION_PREFIX = '" + TIMEOUT_EXCEPTION_PREFIX + "'"); - TIMEOUT_EXCEPTION.printStackTrace(listener.out); - - listener.testFailure(new Failure(null, new TestTimedOutException(999, TimeUnit.MILLISECONDS))); - } -} diff --git a/ratis-common/src/test/resources/META-INF/services/org.junit.platform.launcher.TestExecutionListener b/ratis-common/src/test/resources/META-INF/services/org.junit.platform.launcher.TestExecutionListener new file mode 100644 index 0000000000..71636f29a8 --- /dev/null +++ b/ratis-common/src/test/resources/META-INF/services/org.junit.platform.launcher.TestExecutionListener @@ -0,0 +1,18 @@ +# 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. + +org.apache.ratis.JUnit5TestExecutionListener \ No newline at end of file From 9616b3a9dc6617d2fbe221eca7defe5d3d9892f6 Mon Sep 17 00:00:00 2001 From: "Doroszlai, Attila" <6454655+adoroszlai@users.noreply.github.com> Date: Wed, 26 Feb 2025 18:40:39 +0100 Subject: [PATCH 153/397] RATIS-2217. Automatically re-try flaky tests in CI (#1229) --- .github/workflows/ci.yaml | 1 + dev-support/checks/unit.sh | 4 ++ pom.xml | 18 ++++++++ ratis-assembly/pom.xml | 2 + ratis-client/pom.xml | 5 +++ .../java/org/apache/ratis/test/tag/Flaky.java | 41 +++++++++++++++++++ .../org/apache/ratis/test/tag/FlakyTest.java | 28 +++++++++++++ ratis-docs/pom.xml | 6 +++ .../org/apache/ratis/TestMultiRaftGroup.java | 4 ++ ratis-experiments/pom.xml | 5 +++ ratis-metrics-api/pom.xml | 5 +++ ratis-netty/pom.xml | 5 +++ ratis-proto/pom.xml | 2 + ratis-replicated-map/pom.xml | 5 +++ ratis-server-api/pom.xml | 5 +++ .../server/impl/LeaderElectionTests.java | 6 ++- ratis-shell/pom.xml | 5 +++ .../apache/ratis/grpc/TestRaftWithGrpc.java | 2 + ratis-tools/pom.xml | 5 +++ 19 files changed, 152 insertions(+), 2 deletions(-) create mode 100644 ratis-common/src/test/java/org/apache/ratis/test/tag/Flaky.java create mode 100644 ratis-common/src/test/java/org/apache/ratis/test/tag/FlakyTest.java diff --git a/.github/workflows/ci.yaml b/.github/workflows/ci.yaml index 55d55336b5..cf230d3847 100644 --- a/.github/workflows/ci.yaml +++ b/.github/workflows/ci.yaml @@ -83,6 +83,7 @@ jobs: - grpc - server - misc + - flaky fail-fast: false uses: ./.github/workflows/check.yaml with: diff --git a/dev-support/checks/unit.sh b/dev-support/checks/unit.sh index c652136da7..c0369898e7 100755 --- a/dev-support/checks/unit.sh +++ b/dev-support/checks/unit.sh @@ -36,6 +36,10 @@ mkdir -p "$REPORT_DIR" export MAVEN_OPTS="-Xmx4096m" MAVEN_OPTIONS='-V -B' +if [[ "$@" =~ "-Pflaky-tests" ]]; then + MAVEN_OPTIONS="${MAVEN_OPTIONS} -Dsurefire.rerunFailingTestsCount=5 -Dsurefire.timeout=1200" +fi + if [[ "${FAIL_FAST}" == "true" ]]; then MAVEN_OPTIONS="${MAVEN_OPTIONS} --fail-fast -Dsurefire.skipAfterFailureCount=1" else diff --git a/pom.xml b/pom.xml index 29b2137e42..0a8b1c7319 100644 --- a/pom.xml +++ b/pom.xml @@ -222,6 +222,7 @@ 2.0.7 5.11.2 0.8.12 + flaky | org.apache.ratis.test.tag.FlakyTest @@ -1118,6 +1119,7 @@ org.apache.ratis.grpc.** + ${flaky-test-groups} @@ -1135,6 +1137,7 @@ org.apache.ratis.datastream.** org.apache.ratis.server.** + ${flaky-test-groups} @@ -1153,6 +1156,21 @@ org.apache.ratis.grpc.** org.apache.ratis.server.** + ${flaky-test-groups} + + + + + + + flaky-tests + + + + org.apache.maven.plugins + maven-surefire-plugin + + ${flaky-test-groups} diff --git a/ratis-assembly/pom.xml b/ratis-assembly/pom.xml index 276de0a6b9..11058eb329 100644 --- a/ratis-assembly/pom.xml +++ b/ratis-assembly/pom.xml @@ -33,6 +33,8 @@ ${project.build.directory}/test-classes ${project.build.directory}/test-classes true + + true diff --git a/ratis-client/pom.xml b/ratis-client/pom.xml index 9a8d228d3c..8a5008eafb 100644 --- a/ratis-client/pom.xml +++ b/ratis-client/pom.xml @@ -23,6 +23,11 @@ ratis-client Apache Ratis Client + + + true + + org.apache.ratis diff --git a/ratis-common/src/test/java/org/apache/ratis/test/tag/Flaky.java b/ratis-common/src/test/java/org/apache/ratis/test/tag/Flaky.java new file mode 100644 index 0000000000..2d8c63030a --- /dev/null +++ b/ratis-common/src/test/java/org/apache/ratis/test/tag/Flaky.java @@ -0,0 +1,41 @@ +/* + * 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.ratis.test.tag; + +import java.lang.annotation.ElementType; +import java.lang.annotation.Retention; +import java.lang.annotation.RetentionPolicy; +import java.lang.annotation.Target; + +import org.junit.jupiter.api.Tag; + +/** + * Annotation to mark JUnit5 test classes or methods that exhibit intermittent + * issues. These are run separately from the normal tests in CI. In case of + * failure they may be repeated a few times. + * Usage: @Flaky("RATIS-123") + */ +@Target({ ElementType.TYPE, ElementType.METHOD }) +@Retention(RetentionPolicy.RUNTIME) +@Tag("flaky") +public @interface Flaky { + /** + * The issue(s) tracking the flaky test. + */ + String[] value(); +} diff --git a/ratis-common/src/test/java/org/apache/ratis/test/tag/FlakyTest.java b/ratis-common/src/test/java/org/apache/ratis/test/tag/FlakyTest.java new file mode 100644 index 0000000000..f43bd16985 --- /dev/null +++ b/ratis-common/src/test/java/org/apache/ratis/test/tag/FlakyTest.java @@ -0,0 +1,28 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + *

+ * http://www.apache.org/licenses/LICENSE-2.0 + *

+ * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.ratis.test.tag; + +/** + * Interface to mark JUnit4 test classes or methods that exhibit intermittent + * issues. These are run separately from the normal tests in CI. In case of + * failure they may be repeated a few times. + * Usage: @Category(FlakyTest.class) @Flaky("RATIS-123") + */ +public interface FlakyTest { + // category marker +} diff --git a/ratis-docs/pom.xml b/ratis-docs/pom.xml index 79f9d79b22..08a86fb033 100644 --- a/ratis-docs/pom.xml +++ b/ratis-docs/pom.xml @@ -26,4 +26,10 @@ https://maven.apache.org/xsd/maven-4.0.0.xsd"> ratis-docs Apache Ratis Documentation jar + + + + true + + diff --git a/ratis-examples/src/test/java/org/apache/ratis/TestMultiRaftGroup.java b/ratis-examples/src/test/java/org/apache/ratis/TestMultiRaftGroup.java index 190f758589..7822b694df 100644 --- a/ratis-examples/src/test/java/org/apache/ratis/TestMultiRaftGroup.java +++ b/ratis-examples/src/test/java/org/apache/ratis/TestMultiRaftGroup.java @@ -26,7 +26,9 @@ import org.apache.ratis.server.impl.GroupManagementBaseTest; import org.apache.ratis.server.impl.MiniRaftCluster; import org.apache.ratis.util.Slf4jUtils; +import org.apache.ratis.test.tag.Flaky; import org.apache.ratis.util.function.CheckedBiConsumer; +import org.junit.jupiter.api.Timeout; import org.junit.jupiter.params.ParameterizedTest; import org.junit.jupiter.params.provider.MethodSource; import org.slf4j.event.Level; @@ -35,6 +37,8 @@ import java.util.Collection; import java.util.concurrent.atomic.AtomicInteger; +@Flaky("RATIS-2218") +@Timeout(value = 300) public class TestMultiRaftGroup extends BaseTest { static { Slf4jUtils.setLogLevel(RaftServer.Division.LOG, Level.DEBUG); diff --git a/ratis-experiments/pom.xml b/ratis-experiments/pom.xml index 467ff08eb9..5dc169c360 100644 --- a/ratis-experiments/pom.xml +++ b/ratis-experiments/pom.xml @@ -23,6 +23,11 @@ ratis-experiments Apache Ratis Experiments + + + true + + org.apache.ratis diff --git a/ratis-metrics-api/pom.xml b/ratis-metrics-api/pom.xml index 3f30339bdf..ba3b36b320 100644 --- a/ratis-metrics-api/pom.xml +++ b/ratis-metrics-api/pom.xml @@ -24,6 +24,11 @@ ratis-metrics-api Apache Ratis Metrics API + + + true + + ratis-common diff --git a/ratis-netty/pom.xml b/ratis-netty/pom.xml index bbd6cbe945..b5714a943e 100644 --- a/ratis-netty/pom.xml +++ b/ratis-netty/pom.xml @@ -23,6 +23,11 @@ ratis-netty Apache Ratis Netty Support + + + true + + org.apache.ratis diff --git a/ratis-proto/pom.xml b/ratis-proto/pom.xml index c5fc03ac90..9e65096e44 100644 --- a/ratis-proto/pom.xml +++ b/ratis-proto/pom.xml @@ -25,6 +25,8 @@ true + + true true diff --git a/ratis-replicated-map/pom.xml b/ratis-replicated-map/pom.xml index a0c4f94c1d..486b9cf9c5 100644 --- a/ratis-replicated-map/pom.xml +++ b/ratis-replicated-map/pom.xml @@ -23,6 +23,11 @@ ratis-replicated-map Apache Ratis Replicated Map + + + true + + org.apache.ratis diff --git a/ratis-server-api/pom.xml b/ratis-server-api/pom.xml index 7de592030f..812da9f023 100644 --- a/ratis-server-api/pom.xml +++ b/ratis-server-api/pom.xml @@ -23,6 +23,11 @@ ratis-server-api Apache Ratis Server API + + + true + + org.apache.ratis diff --git a/ratis-server/src/test/java/org/apache/ratis/server/impl/LeaderElectionTests.java b/ratis-server/src/test/java/org/apache/ratis/server/impl/LeaderElectionTests.java index b175ffe292..2129385431 100644 --- a/ratis-server/src/test/java/org/apache/ratis/server/impl/LeaderElectionTests.java +++ b/ratis-server/src/test/java/org/apache/ratis/server/impl/LeaderElectionTests.java @@ -39,13 +39,15 @@ import org.apache.ratis.server.protocol.TermIndex; import org.apache.ratis.server.raftlog.segmented.SegmentedRaftLogTestUtils; import org.apache.ratis.util.ExitUtils; +import org.apache.ratis.test.tag.Flaky; +import org.apache.ratis.thirdparty.com.codahale.metrics.Timer; +import org.apache.ratis.util.CodeInjectionForTesting; import org.apache.ratis.util.JavaUtils; import org.apache.ratis.util.LifeCycle; import org.apache.ratis.util.Slf4jUtils; import org.apache.ratis.util.TimeDuration; import org.apache.ratis.util.Timestamp; import org.apache.ratis.util.function.CheckedBiConsumer; -import org.apache.ratis.util.CodeInjectionForTesting; import org.junit.Assert; import org.junit.jupiter.api.Assertions; import org.junit.jupiter.api.Test; @@ -75,7 +77,6 @@ import static org.mockito.Mockito.mock; import static org.mockito.Mockito.when; -import org.apache.ratis.thirdparty.com.codahale.metrics.Timer; import org.slf4j.event.Level; public abstract class LeaderElectionTests @@ -743,6 +744,7 @@ void runTestLeaderLease(CLUSTER cluster, long leaseTimeoutMs) throws Exception { } } + @Flaky("RATIS-2108") @Test public void testLeaderLeaseDuringReconfiguration() throws Exception { // use a strict lease diff --git a/ratis-shell/pom.xml b/ratis-shell/pom.xml index 38eed1b980..7b4534a60c 100644 --- a/ratis-shell/pom.xml +++ b/ratis-shell/pom.xml @@ -23,6 +23,11 @@ ratis-shell Apache Ratis Shell + + + true + + ratis-client diff --git a/ratis-test/src/test/java/org/apache/ratis/grpc/TestRaftWithGrpc.java b/ratis-test/src/test/java/org/apache/ratis/grpc/TestRaftWithGrpc.java index 5821b7f227..c13544a782 100644 --- a/ratis-test/src/test/java/org/apache/ratis/grpc/TestRaftWithGrpc.java +++ b/ratis-test/src/test/java/org/apache/ratis/grpc/TestRaftWithGrpc.java @@ -28,6 +28,7 @@ import org.apache.ratis.server.raftlog.RaftLog; import org.apache.ratis.statemachine.impl.SimpleStateMachine4Testing; import org.apache.ratis.statemachine.StateMachine; +import org.apache.ratis.test.tag.Flaky; import org.apache.ratis.util.JavaUtils; import org.apache.ratis.util.TimeDuration; import org.junit.jupiter.api.Assertions; @@ -55,6 +56,7 @@ public void testWithLoad() { // skip testWithLoad() from parent, called from parameterized testWithLoad(boolean) } + @Flaky("RATIS-2253") @ParameterizedTest @ValueSource(booleans = {true, false}) public void testWithLoad(boolean separateHeartbeat) throws Exception { diff --git a/ratis-tools/pom.xml b/ratis-tools/pom.xml index 49f36f0b7d..0fe13f4023 100644 --- a/ratis-tools/pom.xml +++ b/ratis-tools/pom.xml @@ -23,6 +23,11 @@ ratis-tools Apache Ratis Tools + + + true + + org.apache.ratis From 337017f3270c0ef6a776efbc6447f7fb19e2991d Mon Sep 17 00:00:00 2001 From: Tsz-Wo Nicholas Sze Date: Wed, 26 Feb 2025 12:39:03 -0800 Subject: [PATCH 154/397] RATIS-2254. Replace junit 4 Timeout with junit 5. (#1230) --- .../test/java/org/apache/ratis/BaseTest.java | 50 +++++++------------ .../arithmetic/expression/TestExpression.java | 9 ++-- .../java/org/apache/ratis/RaftTestUtil.java | 10 ++++ .../server/impl/GroupManagementBaseTest.java | 2 + .../server/impl/LeaderElectionTests.java | 1 + .../apache/ratis/RaftLogTruncateTests.java | 7 +-- .../DataStreamAsyncClusterTests.java | 7 +-- .../apache/ratis/grpc/TestRaftWithGrpc.java | 2 + .../retry/TestMultipleLinearRandomRetry.java | 7 +-- .../apache/ratis/retry/TestRetryPolicy.java | 7 +-- 10 files changed, 45 insertions(+), 57 deletions(-) diff --git a/ratis-common/src/test/java/org/apache/ratis/BaseTest.java b/ratis-common/src/test/java/org/apache/ratis/BaseTest.java index e289784030..3e8e9a0fe0 100644 --- a/ratis-common/src/test/java/org/apache/ratis/BaseTest.java +++ b/ratis-common/src/test/java/org/apache/ratis/BaseTest.java @@ -18,7 +18,6 @@ package org.apache.ratis; import org.apache.ratis.conf.ConfUtils; -import org.apache.ratis.protocol.RaftPeer; import org.apache.ratis.util.ExitUtils; import org.apache.ratis.util.FileUtils; import org.apache.ratis.util.JavaUtils; @@ -27,9 +26,11 @@ import org.apache.ratis.util.TimeDuration; import org.apache.ratis.util.function.CheckedRunnable; import org.junit.After; +import org.junit.Before; import org.junit.Rule; import org.junit.jupiter.api.AfterEach; import org.junit.jupiter.api.Assertions; +import org.junit.jupiter.api.Assumptions; import org.junit.jupiter.api.BeforeEach; import org.junit.jupiter.api.TestInfo; import org.junit.jupiter.api.Timeout; @@ -40,8 +41,7 @@ import java.io.File; import java.io.IOException; -import java.util.ArrayList; -import java.util.List; +import java.lang.reflect.Method; import java.util.Objects; import java.util.concurrent.CompletableFuture; import java.util.concurrent.Future; @@ -73,30 +73,27 @@ public void setFirstException(Throwable e) { } } - public List getPeersWithPriority(List peers, RaftPeer suggestedLeader) { - List peersWithPriority = new ArrayList<>(); - for (int i = 0; i < peers.size(); i++) { - RaftPeer peer = peers.get(i); - final int priority = peer.equals(suggestedLeader)? 2: 1; - peersWithPriority.add( - RaftPeer.newBuilder(peer).setPriority(priority).build()); - } - return peersWithPriority; - } - - - /* - * Junit 4 reference will be removed and the code will be refactored once - * all the unit tests are migrated to Junit 5. - */ + // TODO: Junit 4 reference should be removed once all the unit tests are migrated to Junit 5. private String testCaseName; @BeforeEach public void setup(TestInfo testInfo) { - testCaseName = testInfo.getTestMethod() - .orElseThrow(() -> new RuntimeException("Exception while getting test name.")) - .getName(); + checkAssumptions(); + + final Method method = testInfo.getTestMethod().orElse(null); + testCaseName = testInfo.getTestClass().orElse(getClass()).getSimpleName() + + "." + (method == null? null : method.getName()); + } + + // @Before annotation is retained to support junit 4 tests. + @Before + public void checkAssumptions() { + final Throwable first = firstException.get(); + Assumptions.assumeTrue(first == null, () -> "Already failed with " + first); + + final Throwable exited = ExitUtils.getFirstExitException(); + Assumptions.assumeTrue(exited == null, () -> "Already exited with " + exited); } // @After annotation is retained to support junit 4 tests. @@ -111,19 +108,10 @@ public void assertNoFailures() { ExitUtils.assertNotTerminated(); } - // Retained to support junit 4 tests. - @Rule - public final org.junit.rules.Timeout globalTimeout = new org.junit.rules.Timeout( - getGlobalTimeoutSeconds(), TimeUnit.SECONDS ); - // Retained to support junit 4 tests. @Rule public final TestName testName = new TestName(); - public int getGlobalTimeoutSeconds() { - return 100; - } - private static final Supplier ROOT_TEST_DIR = JavaUtils.memoize( () -> JavaUtils.callAsUnchecked(() -> { final File dir = new File(System.getProperty("test.build.data", "target/test/data"), diff --git a/ratis-examples/src/test/java/org/apache/ratis/examples/arithmetic/expression/TestExpression.java b/ratis-examples/src/test/java/org/apache/ratis/examples/arithmetic/expression/TestExpression.java index 4cc81c6ed1..b512c617f4 100644 --- a/ratis-examples/src/test/java/org/apache/ratis/examples/arithmetic/expression/TestExpression.java +++ b/ratis-examples/src/test/java/org/apache/ratis/examples/arithmetic/expression/TestExpression.java @@ -1,4 +1,4 @@ -/** +/* * 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 @@ -21,16 +21,13 @@ import org.apache.ratis.BaseTest; import org.junit.jupiter.api.Assertions; import org.junit.jupiter.api.Test; +import org.junit.jupiter.api.Timeout; import java.util.Random; import java.util.concurrent.ThreadLocalRandom; +@Timeout(value = 1) public class TestExpression extends BaseTest { - @Override - public int getGlobalTimeoutSeconds() { - return 1; - } - @Test public void testArithmeticUtils() throws Exception { final Random ran = ThreadLocalRandom.current(); diff --git a/ratis-server/src/test/java/org/apache/ratis/RaftTestUtil.java b/ratis-server/src/test/java/org/apache/ratis/RaftTestUtil.java index d94d3a0326..b3a72a93a9 100644 --- a/ratis-server/src/test/java/org/apache/ratis/RaftTestUtil.java +++ b/ratis-server/src/test/java/org/apache/ratis/RaftTestUtil.java @@ -25,6 +25,7 @@ import org.apache.ratis.protocol.Message; import org.apache.ratis.protocol.RaftClientReply; import org.apache.ratis.protocol.RaftGroupId; +import org.apache.ratis.protocol.RaftPeer; import org.apache.ratis.protocol.RaftPeerId; import org.apache.ratis.server.RaftServer; import org.apache.ratis.server.RaftServerConfigKeys; @@ -462,6 +463,15 @@ static void delay(IntSupplier getDelayMs) throws InterruptedException { } } + static List getPeersWithPriority(List peers, RaftPeer suggestedLeader) { + List peersWithPriority = new ArrayList<>(); + for (RaftPeer peer : peers) { + final int priority = peer.equals(suggestedLeader) ? 2 : 1; + peersWithPriority.add(RaftPeer.newBuilder(peer).setPriority(priority).build()); + } + return peersWithPriority; + } + static RaftPeerId changeLeader(MiniRaftCluster cluster, RaftPeerId oldLeader) throws Exception { return changeLeader(cluster, oldLeader, AssumptionViolatedException::new); diff --git a/ratis-server/src/test/java/org/apache/ratis/server/impl/GroupManagementBaseTest.java b/ratis-server/src/test/java/org/apache/ratis/server/impl/GroupManagementBaseTest.java index 30f9f75797..a06336f912 100644 --- a/ratis-server/src/test/java/org/apache/ratis/server/impl/GroupManagementBaseTest.java +++ b/ratis-server/src/test/java/org/apache/ratis/server/impl/GroupManagementBaseTest.java @@ -54,6 +54,8 @@ import java.util.Random; import java.util.stream.Collectors; +import static org.apache.ratis.RaftTestUtil.getPeersWithPriority; + public abstract class GroupManagementBaseTest extends BaseTest { static final Logger LOG = LoggerFactory.getLogger(GroupManagementBaseTest.class); diff --git a/ratis-server/src/test/java/org/apache/ratis/server/impl/LeaderElectionTests.java b/ratis-server/src/test/java/org/apache/ratis/server/impl/LeaderElectionTests.java index 2129385431..edb0a01bd3 100644 --- a/ratis-server/src/test/java/org/apache/ratis/server/impl/LeaderElectionTests.java +++ b/ratis-server/src/test/java/org/apache/ratis/server/impl/LeaderElectionTests.java @@ -65,6 +65,7 @@ import java.util.concurrent.TimeUnit; import java.util.stream.Collectors; +import static org.apache.ratis.RaftTestUtil.getPeersWithPriority; import static org.apache.ratis.RaftTestUtil.waitForLeader; import static org.apache.ratis.server.metrics.LeaderElectionMetrics.LAST_LEADER_ELECTION_ELAPSED_TIME; import static org.apache.ratis.server.metrics.LeaderElectionMetrics.LEADER_ELECTION_COUNT_METRIC; diff --git a/ratis-test/src/test/java/org/apache/ratis/RaftLogTruncateTests.java b/ratis-test/src/test/java/org/apache/ratis/RaftLogTruncateTests.java index 327163de0f..9ea78e47cd 100644 --- a/ratis-test/src/test/java/org/apache/ratis/RaftLogTruncateTests.java +++ b/ratis-test/src/test/java/org/apache/ratis/RaftLogTruncateTests.java @@ -42,6 +42,7 @@ import org.apache.ratis.util.TimeDuration; import org.junit.jupiter.api.Assertions; import org.junit.jupiter.api.Test; +import org.junit.jupiter.api.Timeout; import org.slf4j.event.Level; import java.util.ArrayList; @@ -54,6 +55,7 @@ import static org.apache.ratis.RaftTestUtil.waitForLeader; +@Timeout(value = 200) public abstract class RaftLogTruncateTests extends BaseTest implements MiniRaftCluster.Factory.Get { public static final int NUM_SERVERS = 5; @@ -84,11 +86,6 @@ static SimpleMessage[] arraycopy(SimpleMessage[] src1, SimpleMessage[] src2) { RaftServerConfigKeys.Rpc.setFirstElectionTimeoutMax(p, TimeDuration.ONE_SECOND.multiply(2)); } - @Override - public int getGlobalTimeoutSeconds() { - return 200; - } - @Test public void testLogTruncate() throws Exception { runWithNewCluster(NUM_SERVERS, this::runTestLogTruncate); diff --git a/ratis-test/src/test/java/org/apache/ratis/datastream/DataStreamAsyncClusterTests.java b/ratis-test/src/test/java/org/apache/ratis/datastream/DataStreamAsyncClusterTests.java index a343502069..eb25a369e6 100644 --- a/ratis-test/src/test/java/org/apache/ratis/datastream/DataStreamAsyncClusterTests.java +++ b/ratis-test/src/test/java/org/apache/ratis/datastream/DataStreamAsyncClusterTests.java @@ -36,6 +36,7 @@ import org.apache.ratis.util.function.CheckedBiFunction; import org.junit.jupiter.api.Assertions; import org.junit.jupiter.api.Test; +import org.junit.jupiter.api.Timeout; import org.slf4j.event.Level; import java.io.IOException; @@ -47,15 +48,11 @@ import java.util.concurrent.Executors; import java.util.concurrent.TimeUnit; +@Timeout(value = 300) public abstract class DataStreamAsyncClusterTests extends DataStreamClusterTests { final Executor executor = Executors.newFixedThreadPool(16); - @Override - public int getGlobalTimeoutSeconds() { - return 300; - } - @Test public void testSingleStreamsMultipleServers() throws Exception { Slf4jUtils.setLogLevel(NettyClientStreamRpc.LOG, Level.TRACE); diff --git a/ratis-test/src/test/java/org/apache/ratis/grpc/TestRaftWithGrpc.java b/ratis-test/src/test/java/org/apache/ratis/grpc/TestRaftWithGrpc.java index c13544a782..42211cefc1 100644 --- a/ratis-test/src/test/java/org/apache/ratis/grpc/TestRaftWithGrpc.java +++ b/ratis-test/src/test/java/org/apache/ratis/grpc/TestRaftWithGrpc.java @@ -33,6 +33,7 @@ import org.apache.ratis.util.TimeDuration; import org.junit.jupiter.api.Assertions; import org.junit.jupiter.api.Disabled; +import org.junit.jupiter.api.Timeout; import org.junit.jupiter.params.ParameterizedTest; import org.junit.jupiter.params.provider.ValueSource; @@ -57,6 +58,7 @@ public void testWithLoad() { } @Flaky("RATIS-2253") + @Timeout(300) @ParameterizedTest @ValueSource(booleans = {true, false}) public void testWithLoad(boolean separateHeartbeat) throws Exception { diff --git a/ratis-test/src/test/java/org/apache/ratis/retry/TestMultipleLinearRandomRetry.java b/ratis-test/src/test/java/org/apache/ratis/retry/TestMultipleLinearRandomRetry.java index eb4b7967cd..77bcb70f74 100644 --- a/ratis-test/src/test/java/org/apache/ratis/retry/TestMultipleLinearRandomRetry.java +++ b/ratis-test/src/test/java/org/apache/ratis/retry/TestMultipleLinearRandomRetry.java @@ -21,13 +21,10 @@ import org.apache.ratis.util.TimeDuration; import org.junit.jupiter.api.Assertions; import org.junit.jupiter.api.Test; +import org.junit.jupiter.api.Timeout; +@Timeout(value = 1) public class TestMultipleLinearRandomRetry extends BaseTest { - @Override - public int getGlobalTimeoutSeconds() { - return 1; - } - @Test public void testParseCommaSeparated() { assertIllegalInput(""); diff --git a/ratis-test/src/test/java/org/apache/ratis/retry/TestRetryPolicy.java b/ratis-test/src/test/java/org/apache/ratis/retry/TestRetryPolicy.java index cee8ee3385..43b2fedd1a 100644 --- a/ratis-test/src/test/java/org/apache/ratis/retry/TestRetryPolicy.java +++ b/ratis-test/src/test/java/org/apache/ratis/retry/TestRetryPolicy.java @@ -36,6 +36,7 @@ import org.apache.ratis.util.Timestamp; import org.junit.jupiter.api.Assertions; import org.junit.jupiter.api.Test; +import org.junit.jupiter.api.Timeout; import java.io.IOException; import java.util.ArrayList; @@ -45,12 +46,8 @@ import java.util.concurrent.TimeUnit; /** Test {@link RetryPolicy}. */ +@Timeout(value = 1) public class TestRetryPolicy extends BaseTest { - @Override - public int getGlobalTimeoutSeconds() { - return 1; - } - @Test public void testRetryMultipleTimesWithFixedSleep() { final int n = 4; From fae2552e88d6a8cb82a1c5d736a5bf55d70fb1a0 Mon Sep 17 00:00:00 2001 From: "Doroszlai, Attila" <6454655+adoroszlai@users.noreply.github.com> Date: Thu, 27 Feb 2025 20:01:56 +0100 Subject: [PATCH 155/397] RATIS-2022. Replace some Guava usage with builtins (#1231) --- .../src/main/java/org/apache/ratis/conf/ConfUtils.java | 4 ++-- .../java/org/apache/ratis/util/TestRefCountingMap.java | 6 +++--- .../src/test/java/org/apache/ratis/RaftTestUtil.java | 7 ++++--- 3 files changed, 9 insertions(+), 8 deletions(-) diff --git a/ratis-common/src/main/java/org/apache/ratis/conf/ConfUtils.java b/ratis-common/src/main/java/org/apache/ratis/conf/ConfUtils.java index 3f7678a0bc..43706faabc 100644 --- a/ratis-common/src/main/java/org/apache/ratis/conf/ConfUtils.java +++ b/ratis-common/src/main/java/org/apache/ratis/conf/ConfUtils.java @@ -18,7 +18,6 @@ package org.apache.ratis.conf; import org.apache.ratis.security.TlsConf; -import org.apache.ratis.thirdparty.com.google.common.base.Objects; import org.apache.ratis.util.NetUtils; import org.apache.ratis.util.SizeInBytes; import org.apache.ratis.util.TimeDuration; @@ -33,6 +32,7 @@ import java.net.InetSocketAddress; import java.util.Arrays; import java.util.List; +import java.util.Objects; import java.util.concurrent.ConcurrentHashMap; import java.util.concurrent.ConcurrentMap; import java.util.function.BiConsumer; @@ -60,7 +60,7 @@ private static boolean isNew(String key, T value) { static void logGet(String key, T value, T defaultValue, Consumer logger) { if (logger != null && Utils.isNew(key, value)) { logger.accept(String.format("%s = %s (%s)", key, value, - Objects.equal(value, defaultValue)? "default": "custom")); + Objects.equals(value, defaultValue)? "default": "custom")); } } diff --git a/ratis-common/src/test/java/org/apache/ratis/util/TestRefCountingMap.java b/ratis-common/src/test/java/org/apache/ratis/util/TestRefCountingMap.java index 6c438e247f..217d6e3da8 100644 --- a/ratis-common/src/test/java/org/apache/ratis/util/TestRefCountingMap.java +++ b/ratis-common/src/test/java/org/apache/ratis/util/TestRefCountingMap.java @@ -22,10 +22,10 @@ import static org.junit.jupiter.api.Assertions.assertNull; import static org.junit.jupiter.api.Assertions.assertTrue; +import java.util.Arrays; import java.util.Collection; import java.util.Set; -import org.apache.ratis.thirdparty.com.google.common.collect.Lists; import org.junit.jupiter.api.BeforeEach; import org.junit.jupiter.api.Test; @@ -128,7 +128,7 @@ public void testKeySet() { Set keys = map.keySet(); assertEquals(3, keys.size()); - Lists.newArrayList("foo", "bar", "baz").forEach(v -> assertTrue(keys.contains(v))); + Arrays.asList("foo", "bar", "baz").forEach(v -> assertTrue(keys.contains(v))); } @Test @@ -141,7 +141,7 @@ public void testValues() { Collection values = map.values(); assertEquals(3, values.size()); - Lists.newArrayList("foovalue", "foovalue3", "foovalue4") + Arrays.asList("foovalue", "foovalue3", "foovalue4") .forEach(v -> assertTrue(values.contains(v))); } } diff --git a/ratis-server/src/test/java/org/apache/ratis/RaftTestUtil.java b/ratis-server/src/test/java/org/apache/ratis/RaftTestUtil.java index b3a72a93a9..3c1b3eda24 100644 --- a/ratis-server/src/test/java/org/apache/ratis/RaftTestUtil.java +++ b/ratis-server/src/test/java/org/apache/ratis/RaftTestUtil.java @@ -36,11 +36,11 @@ import org.apache.ratis.server.raftlog.LogProtoUtils; import org.apache.ratis.server.raftlog.RaftLog; import org.apache.ratis.server.raftlog.RaftLogBase; -import org.apache.ratis.thirdparty.com.google.common.base.Preconditions; import org.apache.ratis.thirdparty.com.google.protobuf.ByteString; import org.apache.ratis.util.AutoCloseableLock; import org.apache.ratis.util.CollectionUtils; import org.apache.ratis.util.JavaUtils; +import org.apache.ratis.util.Preconditions; import org.apache.ratis.util.ProtoUtils; import org.apache.ratis.util.TimeDuration; import org.junit.Assert; @@ -150,8 +150,9 @@ static RaftPeerId waitAndKillLeader(MiniRaftCluster cluster) throws InterruptedE static void waitFor(Supplier check, int checkEveryMillis, int waitForMillis) throws TimeoutException, InterruptedException { - Preconditions.checkNotNull(check); - Preconditions.checkArgument(waitForMillis >= checkEveryMillis); + Preconditions.assertNotNull(check, "check"); + Preconditions.assertTrue(waitForMillis >= checkEveryMillis, + () -> "waitFor: " + waitForMillis + " < checkEvery: " + checkEveryMillis); long st = System.currentTimeMillis(); boolean result = check.get(); From 509f72ee6e1bd10976b8f1cbacd4333814e5e44b Mon Sep 17 00:00:00 2001 From: slfan1989 <55643692+slfan1989@users.noreply.github.com> Date: Wed, 5 Mar 2025 02:51:04 +0800 Subject: [PATCH 156/397] RATIS-2124. Remove the use of org.junit.Rule. (#1232) --- .../test/java/org/apache/ratis/BaseTest.java | 10 +--- .../statemachine/RaftSnapshotBaseTest.java | 50 +++++++++---------- 2 files changed, 27 insertions(+), 33 deletions(-) diff --git a/ratis-common/src/test/java/org/apache/ratis/BaseTest.java b/ratis-common/src/test/java/org/apache/ratis/BaseTest.java index 3e8e9a0fe0..ff912879d7 100644 --- a/ratis-common/src/test/java/org/apache/ratis/BaseTest.java +++ b/ratis-common/src/test/java/org/apache/ratis/BaseTest.java @@ -27,14 +27,12 @@ import org.apache.ratis.util.function.CheckedRunnable; import org.junit.After; import org.junit.Before; -import org.junit.Rule; import org.junit.jupiter.api.AfterEach; import org.junit.jupiter.api.Assertions; import org.junit.jupiter.api.Assumptions; import org.junit.jupiter.api.BeforeEach; import org.junit.jupiter.api.TestInfo; import org.junit.jupiter.api.Timeout; -import org.junit.rules.TestName; import org.slf4j.Logger; import org.slf4j.LoggerFactory; import org.slf4j.event.Level; @@ -88,6 +86,7 @@ public void setup(TestInfo testInfo) { // @Before annotation is retained to support junit 4 tests. @Before + @BeforeEach public void checkAssumptions() { final Throwable first = firstException.get(); Assumptions.assumeTrue(first == null, () -> "Already failed with " + first); @@ -108,10 +107,6 @@ public void assertNoFailures() { ExitUtils.assertNotTerminated(); } - // Retained to support junit 4 tests. - @Rule - public final TestName testName = new TestName(); - private static final Supplier ROOT_TEST_DIR = JavaUtils.memoize( () -> JavaUtils.callAsUnchecked(() -> { final File dir = new File(System.getProperty("test.build.data", "target/test/data"), @@ -135,8 +130,7 @@ public File getClassTestDir() { public File getTestDir() { // This will work for both junit 4 and 5. - final String name = testCaseName != null ? testCaseName : testName.getMethodName(); - return new File(getClassTestDir(), name); + return new File(getClassTestDir(), testCaseName); } @SafeVarargs diff --git a/ratis-server/src/test/java/org/apache/ratis/statemachine/RaftSnapshotBaseTest.java b/ratis-server/src/test/java/org/apache/ratis/statemachine/RaftSnapshotBaseTest.java index fe1a97ddca..507dd63ad5 100644 --- a/ratis-server/src/test/java/org/apache/ratis/statemachine/RaftSnapshotBaseTest.java +++ b/ratis-server/src/test/java/org/apache/ratis/statemachine/RaftSnapshotBaseTest.java @@ -49,10 +49,10 @@ import org.apache.ratis.util.JavaUtils; import org.apache.ratis.util.LifeCycle; import org.apache.ratis.util.Slf4jUtils; -import org.junit.After; -import org.junit.Assert; -import org.junit.Before; -import org.junit.Test; +import org.junit.jupiter.api.Assertions; +import org.junit.jupiter.api.AfterEach; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.Test; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -96,16 +96,16 @@ public static void assertLogContent(RaftServer.Division server, boolean isLeader final RaftLog log = server.getRaftLog(); final long lastIndex = log.getLastEntryTermIndex().getIndex(); final LogEntryProto e = log.get(lastIndex); - Assert.assertTrue(e.hasMetadataEntry()); + Assertions.assertTrue(e.hasMetadataEntry()); JavaUtils.attemptRepeatedly(() -> { - Assert.assertEquals(log.getLastCommittedIndex() - 1, e.getMetadataEntry().getCommitIndex()); + Assertions.assertEquals(log.getLastCommittedIndex() - 1, e.getMetadataEntry().getCommitIndex()); return null; }, 50, BaseTest.HUNDRED_MILLIS, "CheckMetadataEntry", LOG); SimpleStateMachine4Testing simpleStateMachine = SimpleStateMachine4Testing.get(server); if (isLeader) { - Assert.assertTrue("Not notified as a leader", simpleStateMachine.isNotifiedAsLeader()); + Assertions.assertTrue(simpleStateMachine.isNotifiedAsLeader(), "Not notified as a leader"); } final LogEntryProto[] entries = simpleStateMachine.getContent(); long message = 0; @@ -113,7 +113,7 @@ public static void assertLogContent(RaftServer.Division server, boolean isLeader LOG.info("{}) {} {}", i, message, entries[i].toString().replace("\n", ", ")); if (entries[i].hasStateMachineLogEntry()) { final SimpleMessage m = new SimpleMessage("m" + message++); - Assert.assertArrayEquals(m.getContent().toByteArray(), + Assertions.assertArrayEquals(m.getContent().toByteArray(), entries[i].getStateMachineLogEntry().getLogData().toByteArray()); } } @@ -123,7 +123,7 @@ public static void assertLogContent(RaftServer.Division server, boolean isLeader public abstract MiniRaftCluster.Factory getFactory(); - @Before + @BeforeEach public void setup() throws IOException { final RaftProperties prop = new RaftProperties(); prop.setClass(MiniRaftCluster.STATEMACHINE_CLASS_KEY, @@ -135,7 +135,7 @@ public void setup() throws IOException { cluster.start(); } - @After + @AfterEach public void tearDown() { if (cluster != null) { cluster.shutdown(); @@ -155,7 +155,7 @@ public void testRestartPeer() throws Exception { try(final RaftClient client = cluster.createClient(leaderId)) { for (; i < SNAPSHOT_TRIGGER_THRESHOLD * 2 - 1; i++) { RaftClientReply reply = client.io().send(new SimpleMessage("m" + i)); - Assert.assertTrue(reply.isSuccess()); + Assertions.assertTrue(reply.isSuccess()); } } @@ -164,7 +164,7 @@ public void testRestartPeer() throws Exception { // wait for the snapshot to be done final List snapshotFiles = getSnapshotFiles(cluster, nextIndex - SNAPSHOT_TRIGGER_THRESHOLD, nextIndex); JavaUtils.attemptRepeatedly(() -> { - Assert.assertTrue(snapshotFiles.stream().anyMatch(RaftSnapshotBaseTest::exists)); + Assertions.assertTrue(snapshotFiles.stream().anyMatch(RaftSnapshotBaseTest::exists)); return null; }, 10, ONE_SECOND, "snapshotFile.exist", LOG); @@ -202,7 +202,7 @@ public void testBasicInstallSnapshot() throws Exception { try(final RaftClient client = cluster.createClient(leaderId)) { for (; i < SNAPSHOT_TRIGGER_THRESHOLD * 2 - 1; i++) { RaftClientReply reply = client.io().send(new SimpleMessage("m" + i)); - Assert.assertTrue(reply.isSuccess()); + Assertions.assertTrue(reply.isSuccess()); } } @@ -211,7 +211,7 @@ public void testBasicInstallSnapshot() throws Exception { LOG.info("nextIndex = {}", nextIndex); final List snapshotFiles = getSnapshotFiles(cluster, nextIndex - SNAPSHOT_TRIGGER_THRESHOLD, nextIndex); JavaUtils.attemptRepeatedly(() -> { - Assert.assertTrue(snapshotFiles.stream().anyMatch(RaftSnapshotBaseTest::exists)); + Assertions.assertTrue(snapshotFiles.stream().anyMatch(RaftSnapshotBaseTest::exists)); return null; }, 10, ONE_SECOND, "snapshotFile.exist", LOG); verifyTakeSnapshotMetric(cluster.getLeader()); @@ -233,7 +233,7 @@ public void testBasicInstallSnapshot() throws Exception { // generate some more traffic try(final RaftClient client = cluster.createClient(cluster.getLeader().getId())) { - Assert.assertTrue(client.io().send(new SimpleMessage("m" + i)).isSuccess()); + Assertions.assertTrue(client.io().send(new SimpleMessage("m" + i)).isSuccess()); } // add two more peers @@ -247,7 +247,7 @@ public void testBasicInstallSnapshot() throws Exception { for (String newPeer : newPeers) { final RaftServer.Division s = cluster.getDivision(RaftPeerId.valueOf(newPeer)); SimpleStateMachine4Testing simpleStateMachine = SimpleStateMachine4Testing.get(s); - Assert.assertSame(LifeCycle.State.RUNNING, simpleStateMachine.getLifeCycleState()); + Assertions.assertSame(LifeCycle.State.RUNNING, simpleStateMachine.getLifeCycleState()); } // Verify installSnapshot counter on leader before restart. @@ -262,7 +262,7 @@ public void testBasicInstallSnapshot() throws Exception { assertLeaderContent(cluster); // verify that snapshot was taken when stopping the server - Assert.assertTrue(count < timer.getCount()); + Assertions.assertTrue(count < timer.getCount()); } finally { cluster.shutdown(); } @@ -283,7 +283,7 @@ public void testInstallSnapshotDuringBootstrap() throws Exception { try(final RaftClient client = cluster.createClient(leaderId)) { for (; i < SNAPSHOT_TRIGGER_THRESHOLD * 2 - 1; i++) { RaftClientReply reply = client.io().send(new SimpleMessage("m" + i)); - Assert.assertTrue(reply.isSuccess()); + Assertions.assertTrue(reply.isSuccess()); } } @@ -292,7 +292,7 @@ public void testInstallSnapshotDuringBootstrap() throws Exception { LOG.info("nextIndex = {}", nextIndex); final List snapshotFiles = getSnapshotFiles(cluster, nextIndex - SNAPSHOT_TRIGGER_THRESHOLD, nextIndex); JavaUtils.attemptRepeatedly(() -> { - Assert.assertTrue(snapshotFiles.stream().anyMatch(RaftSnapshotBaseTest::exists)); + Assertions.assertTrue(snapshotFiles.stream().anyMatch(RaftSnapshotBaseTest::exists)); return null; }, 10, ONE_SECOND, "snapshotFile.exist", LOG); verifyTakeSnapshotMetric(cluster.getLeader()); @@ -310,7 +310,7 @@ public void testInstallSnapshotDuringBootstrap() throws Exception { for (String newPeer : newPeers) { final RaftServer.Division s = cluster.getDivision(RaftPeerId.valueOf(newPeer)); SimpleStateMachine4Testing simpleStateMachine = SimpleStateMachine4Testing.get(s); - Assert.assertSame(LifeCycle.State.RUNNING, simpleStateMachine.getLifeCycleState()); + Assertions.assertSame(LifeCycle.State.RUNNING, simpleStateMachine.getLifeCycleState()); } // Verify installSnapshot counter on leader @@ -324,13 +324,13 @@ public void testInstallSnapshotDuringBootstrap() throws Exception { protected void verifyInstallSnapshotMetric(RaftServer.Division leader) { final LongCounter installSnapshotCounter = ((RaftServerMetricsImpl)leader.getRaftServerMetrics()) .getNumInstallSnapshot(); - Assert.assertNotNull(installSnapshotCounter); - Assert.assertTrue(installSnapshotCounter.getCount() >= 1); + Assertions.assertNotNull(installSnapshotCounter); + Assertions.assertTrue(installSnapshotCounter.getCount() >= 1); } private static void verifyTakeSnapshotMetric(RaftServer.Division leader) { Timer timer = getTakeSnapshotTimer(leader); - Assert.assertTrue(timer.getCount() > 0); + Assertions.assertTrue(timer.getCount() > 0); } private static Timer getTakeSnapshotTimer(RaftServer.Division leader) { @@ -338,9 +338,9 @@ private static Timer getTakeSnapshotTimer(RaftServer.Division leader) { RATIS_APPLICATION_NAME_METRICS, RATIS_STATEMACHINE_METRICS, RATIS_STATEMACHINE_METRICS_DESC); Optional opt = MetricRegistries.global().get(info); - Assert.assertTrue(opt.isPresent()); + Assertions.assertTrue(opt.isPresent()); RatisMetricRegistry metricRegistry = opt.get(); - Assert.assertNotNull(metricRegistry); + Assertions.assertNotNull(metricRegistry); return ((DefaultTimekeeperImpl)metricRegistry.timer(STATEMACHINE_TAKE_SNAPSHOT_TIMER)).getTimer(); } } From 81a35d7c50850706c496230c2685392e2226ea52 Mon Sep 17 00:00:00 2001 From: Tsz-Wo Nicholas Sze Date: Tue, 4 Mar 2025 22:43:21 -0800 Subject: [PATCH 157/397] RATIS-2256. Update Maven to 3.9.9 (#1234) --- .mvn/wrapper/maven-wrapper.properties | 5 +- mvnw | 435 ++++++++++++-------------- mvnw.cmd | 304 ++++++++---------- 3 files changed, 320 insertions(+), 424 deletions(-) diff --git a/.mvn/wrapper/maven-wrapper.properties b/.mvn/wrapper/maven-wrapper.properties index 08ea486aa5..d58dfb70ba 100644 --- a/.mvn/wrapper/maven-wrapper.properties +++ b/.mvn/wrapper/maven-wrapper.properties @@ -14,5 +14,6 @@ # KIND, either express or implied. See the License for the # specific language governing permissions and limitations # under the License. -distributionUrl=https://repo.maven.apache.org/maven2/org/apache/maven/apache-maven/3.9.0/apache-maven-3.9.0-bin.zip -wrapperUrl=https://repo.maven.apache.org/maven2/org/apache/maven/wrapper/maven-wrapper/3.2.0/maven-wrapper-3.2.0.jar +wrapperVersion=3.3.2 +distributionType=only-script +distributionUrl=https://repo.maven.apache.org/maven2/org/apache/maven/apache-maven/3.9.9/apache-maven-3.9.9-bin.zip diff --git a/mvnw b/mvnw index 8d937f4c14..19529ddf8c 100755 --- a/mvnw +++ b/mvnw @@ -19,290 +19,241 @@ # ---------------------------------------------------------------------------- # ---------------------------------------------------------------------------- -# Apache Maven Wrapper startup batch script, version 3.2.0 -# -# Required ENV vars: -# ------------------ -# JAVA_HOME - location of a JDK home dir +# Apache Maven Wrapper startup batch script, version 3.3.2 # # Optional ENV vars # ----------------- -# MAVEN_OPTS - parameters passed to the Java VM when running Maven -# e.g. to debug Maven itself, use -# set MAVEN_OPTS=-Xdebug -Xrunjdwp:transport=dt_socket,server=y,suspend=y,address=8000 -# MAVEN_SKIP_RC - flag to disable loading of mavenrc files +# JAVA_HOME - location of a JDK home dir, required when download maven via java source +# MVNW_REPOURL - repo url base for downloading maven distribution +# MVNW_USERNAME/MVNW_PASSWORD - user and password for downloading maven +# MVNW_VERBOSE - true: enable verbose log; debug: trace the mvnw script; others: silence the output # ---------------------------------------------------------------------------- -if [ -z "$MAVEN_SKIP_RC" ] ; then - - if [ -f /usr/local/etc/mavenrc ] ; then - . /usr/local/etc/mavenrc - fi - - if [ -f /etc/mavenrc ] ; then - . /etc/mavenrc - fi - - if [ -f "$HOME/.mavenrc" ] ; then - . "$HOME/.mavenrc" - fi - -fi +set -euf +[ "${MVNW_VERBOSE-}" != debug ] || set -x -# OS specific support. $var _must_ be set to either true or false. -cygwin=false; -darwin=false; -mingw=false +# OS specific support. +native_path() { printf %s\\n "$1"; } case "$(uname)" in - CYGWIN*) cygwin=true ;; - MINGW*) mingw=true;; - Darwin*) darwin=true - # Use /usr/libexec/java_home if available, otherwise fall back to /Library/Java/Home - # See https://developer.apple.com/library/mac/qa/qa1170/_index.html - if [ -z "$JAVA_HOME" ]; then - if [ -x "/usr/libexec/java_home" ]; then - JAVA_HOME="$(/usr/libexec/java_home)"; export JAVA_HOME - else - JAVA_HOME="/Library/Java/Home"; export JAVA_HOME - fi - fi - ;; +CYGWIN* | MINGW*) + [ -z "${JAVA_HOME-}" ] || JAVA_HOME="$(cygpath --unix "$JAVA_HOME")" + native_path() { cygpath --path --windows "$1"; } + ;; esac -if [ -z "$JAVA_HOME" ] ; then - if [ -r /etc/gentoo-release ] ; then - JAVA_HOME=$(java-config --jre-home) - fi -fi - -# For Cygwin, ensure paths are in UNIX format before anything is touched -if $cygwin ; then - [ -n "$JAVA_HOME" ] && - JAVA_HOME=$(cygpath --unix "$JAVA_HOME") - [ -n "$CLASSPATH" ] && - CLASSPATH=$(cygpath --path --unix "$CLASSPATH") -fi - -# For Mingw, ensure paths are in UNIX format before anything is touched -if $mingw ; then - [ -n "$JAVA_HOME" ] && [ -d "$JAVA_HOME" ] && - JAVA_HOME="$(cd "$JAVA_HOME" || (echo "cannot cd into $JAVA_HOME."; exit 1); pwd)" -fi - -if [ -z "$JAVA_HOME" ]; then - javaExecutable="$(which javac)" - if [ -n "$javaExecutable" ] && ! [ "$(expr "\"$javaExecutable\"" : '\([^ ]*\)')" = "no" ]; then - # readlink(1) is not available as standard on Solaris 10. - readLink=$(which readlink) - if [ ! "$(expr "$readLink" : '\([^ ]*\)')" = "no" ]; then - if $darwin ; then - javaHome="$(dirname "\"$javaExecutable\"")" - javaExecutable="$(cd "\"$javaHome\"" && pwd -P)/javac" - else - javaExecutable="$(readlink -f "\"$javaExecutable\"")" - fi - javaHome="$(dirname "\"$javaExecutable\"")" - javaHome=$(expr "$javaHome" : '\(.*\)/bin') - JAVA_HOME="$javaHome" - export JAVA_HOME - fi - fi -fi - -if [ -z "$JAVACMD" ] ; then - if [ -n "$JAVA_HOME" ] ; then - if [ -x "$JAVA_HOME/jre/sh/java" ] ; then +# set JAVACMD and JAVACCMD +set_java_home() { + # For Cygwin and MinGW, ensure paths are in Unix format before anything is touched + if [ -n "${JAVA_HOME-}" ]; then + if [ -x "$JAVA_HOME/jre/sh/java" ]; then # IBM's JDK on AIX uses strange locations for the executables JAVACMD="$JAVA_HOME/jre/sh/java" + JAVACCMD="$JAVA_HOME/jre/sh/javac" else JAVACMD="$JAVA_HOME/bin/java" + JAVACCMD="$JAVA_HOME/bin/javac" + + if [ ! -x "$JAVACMD" ] || [ ! -x "$JAVACCMD" ]; then + echo "The JAVA_HOME environment variable is not defined correctly, so mvnw cannot run." >&2 + echo "JAVA_HOME is set to \"$JAVA_HOME\", but \"\$JAVA_HOME/bin/java\" or \"\$JAVA_HOME/bin/javac\" does not exist." >&2 + return 1 + fi fi else - JAVACMD="$(\unset -f command 2>/dev/null; \command -v java)" - fi -fi - -if [ ! -x "$JAVACMD" ] ; then - echo "Error: JAVA_HOME is not defined correctly." >&2 - echo " We cannot execute $JAVACMD" >&2 - exit 1 -fi + JAVACMD="$( + 'set' +e + 'unset' -f command 2>/dev/null + 'command' -v java + )" || : + JAVACCMD="$( + 'set' +e + 'unset' -f command 2>/dev/null + 'command' -v javac + )" || : -if [ -z "$JAVA_HOME" ] ; then - echo "Warning: JAVA_HOME environment variable is not set." -fi - -# traverses directory structure from process work directory to filesystem root -# first directory with .mvn subdirectory is considered project base directory -find_maven_basedir() { - if [ -z "$1" ] - then - echo "Path not specified to find_maven_basedir" - return 1 + if [ ! -x "${JAVACMD-}" ] || [ ! -x "${JAVACCMD-}" ]; then + echo "The java/javac command does not exist in PATH nor is JAVA_HOME set, so mvnw cannot run." >&2 + return 1 + fi fi +} - basedir="$1" - wdir="$1" - while [ "$wdir" != '/' ] ; do - if [ -d "$wdir"/.mvn ] ; then - basedir=$wdir - break - fi - # workaround for JBEAP-8937 (on Solaris 10/Sparc) - if [ -d "${wdir}" ]; then - wdir=$(cd "$wdir/.." || exit 1; pwd) - fi - # end of workaround +# hash string like Java String::hashCode +hash_string() { + str="${1:-}" h=0 + while [ -n "$str" ]; do + char="${str%"${str#?}"}" + h=$(((h * 31 + $(LC_CTYPE=C printf %d "'$char")) % 4294967296)) + str="${str#?}" done - printf '%s' "$(cd "$basedir" || exit 1; pwd)" + printf %x\\n $h } -# concatenates all lines of a file -concat_lines() { - if [ -f "$1" ]; then - # Remove \r in case we run on Windows within Git Bash - # and check out the repository with auto CRLF management - # enabled. Otherwise, we may read lines that are delimited with - # \r\n and produce $'-Xarg\r' rather than -Xarg due to word - # splitting rules. - tr -s '\r\n' ' ' < "$1" - fi +verbose() { :; } +[ "${MVNW_VERBOSE-}" != true ] || verbose() { printf %s\\n "${1-}"; } + +die() { + printf %s\\n "$1" >&2 + exit 1 } -log() { - if [ "$MVNW_VERBOSE" = true ]; then - printf '%s\n' "$1" - fi +trim() { + # MWRAPPER-139: + # Trims trailing and leading whitespace, carriage returns, tabs, and linefeeds. + # Needed for removing poorly interpreted newline sequences when running in more + # exotic environments such as mingw bash on Windows. + printf "%s" "${1}" | tr -d '[:space:]' +} + +# parse distributionUrl and optional distributionSha256Sum, requires .mvn/wrapper/maven-wrapper.properties +while IFS="=" read -r key value; do + case "${key-}" in + distributionUrl) distributionUrl=$(trim "${value-}") ;; + distributionSha256Sum) distributionSha256Sum=$(trim "${value-}") ;; + esac +done <"${0%/*}/.mvn/wrapper/maven-wrapper.properties" +[ -n "${distributionUrl-}" ] || die "cannot read distributionUrl property in ${0%/*}/.mvn/wrapper/maven-wrapper.properties" + +case "${distributionUrl##*/}" in +maven-mvnd-*bin.*) + MVN_CMD=mvnd.sh _MVNW_REPO_PATTERN=/maven/mvnd/ + case "${PROCESSOR_ARCHITECTURE-}${PROCESSOR_ARCHITEW6432-}:$(uname -a)" in + *AMD64:CYGWIN* | *AMD64:MINGW*) distributionPlatform=windows-amd64 ;; + :Darwin*x86_64) distributionPlatform=darwin-amd64 ;; + :Darwin*arm64) distributionPlatform=darwin-aarch64 ;; + :Linux*x86_64*) distributionPlatform=linux-amd64 ;; + *) + echo "Cannot detect native platform for mvnd on $(uname)-$(uname -m), use pure java version" >&2 + distributionPlatform=linux-amd64 + ;; + esac + distributionUrl="${distributionUrl%-bin.*}-$distributionPlatform.zip" + ;; +maven-mvnd-*) MVN_CMD=mvnd.sh _MVNW_REPO_PATTERN=/maven/mvnd/ ;; +*) MVN_CMD="mvn${0##*/mvnw}" _MVNW_REPO_PATTERN=/org/apache/maven/ ;; +esac + +# apply MVNW_REPOURL and calculate MAVEN_HOME +# maven home pattern: ~/.m2/wrapper/dists/{apache-maven-,maven-mvnd--}/ +[ -z "${MVNW_REPOURL-}" ] || distributionUrl="$MVNW_REPOURL$_MVNW_REPO_PATTERN${distributionUrl#*"$_MVNW_REPO_PATTERN"}" +distributionUrlName="${distributionUrl##*/}" +distributionUrlNameMain="${distributionUrlName%.*}" +distributionUrlNameMain="${distributionUrlNameMain%-bin}" +MAVEN_USER_HOME="${MAVEN_USER_HOME:-${HOME}/.m2}" +MAVEN_HOME="${MAVEN_USER_HOME}/wrapper/dists/${distributionUrlNameMain-}/$(hash_string "$distributionUrl")" + +exec_maven() { + unset MVNW_VERBOSE MVNW_USERNAME MVNW_PASSWORD MVNW_REPOURL || : + exec "$MAVEN_HOME/bin/$MVN_CMD" "$@" || die "cannot exec $MAVEN_HOME/bin/$MVN_CMD" } -BASE_DIR=$(find_maven_basedir "$(dirname "$0")") -if [ -z "$BASE_DIR" ]; then - exit 1; +if [ -d "$MAVEN_HOME" ]; then + verbose "found existing MAVEN_HOME at $MAVEN_HOME" + exec_maven "$@" fi -MAVEN_PROJECTBASEDIR=${MAVEN_BASEDIR:-"$BASE_DIR"}; export MAVEN_PROJECTBASEDIR -log "$MAVEN_PROJECTBASEDIR" +case "${distributionUrl-}" in +*?-bin.zip | *?maven-mvnd-?*-?*.zip) ;; +*) die "distributionUrl is not valid, must match *-bin.zip or maven-mvnd-*.zip, but found '${distributionUrl-}'" ;; +esac -########################################################################################## -# Extension to allow automatically downloading the maven-wrapper.jar from Maven-central -# This allows using the maven wrapper in projects that prohibit checking in binary data. -########################################################################################## -wrapperJarPath="$MAVEN_PROJECTBASEDIR/.mvn/wrapper/maven-wrapper.jar" -if [ -r "$wrapperJarPath" ]; then - log "Found $wrapperJarPath" +# prepare tmp dir +if TMP_DOWNLOAD_DIR="$(mktemp -d)" && [ -d "$TMP_DOWNLOAD_DIR" ]; then + clean() { rm -rf -- "$TMP_DOWNLOAD_DIR"; } + trap clean HUP INT TERM EXIT else - log "Couldn't find $wrapperJarPath, downloading it ..." + die "cannot create temp dir" +fi - if [ -n "$MVNW_REPOURL" ]; then - wrapperUrl="$MVNW_REPOURL/org/apache/maven/wrapper/maven-wrapper/3.2.0/maven-wrapper-3.2.0.jar" - else - wrapperUrl="https://repo.maven.apache.org/maven2/org/apache/maven/wrapper/maven-wrapper/3.2.0/maven-wrapper-3.2.0.jar" - fi - while IFS="=" read -r key value; do - # Remove '\r' from value to allow usage on windows as IFS does not consider '\r' as a separator ( considers space, tab, new line ('\n'), and custom '=' ) - safeValue=$(echo "$value" | tr -d '\r') - case "$key" in (wrapperUrl) wrapperUrl="$safeValue"; break ;; - esac - done < "$MAVEN_PROJECTBASEDIR/.mvn/wrapper/maven-wrapper.properties" - log "Downloading from: $wrapperUrl" +mkdir -p -- "${MAVEN_HOME%/*}" - if $cygwin; then - wrapperJarPath=$(cygpath --path --windows "$wrapperJarPath") - fi +# Download and Install Apache Maven +verbose "Couldn't find MAVEN_HOME, downloading and installing it ..." +verbose "Downloading from: $distributionUrl" +verbose "Downloading to: $TMP_DOWNLOAD_DIR/$distributionUrlName" - if command -v wget > /dev/null; then - log "Found wget ... using wget" - [ "$MVNW_VERBOSE" = true ] && QUIET="" || QUIET="--quiet" - if [ -z "$MVNW_USERNAME" ] || [ -z "$MVNW_PASSWORD" ]; then - wget $QUIET "$wrapperUrl" -O "$wrapperJarPath" || rm -f "$wrapperJarPath" - else - wget $QUIET --http-user="$MVNW_USERNAME" --http-password="$MVNW_PASSWORD" "$wrapperUrl" -O "$wrapperJarPath" || rm -f "$wrapperJarPath" - fi - elif command -v curl > /dev/null; then - log "Found curl ... using curl" - [ "$MVNW_VERBOSE" = true ] && QUIET="" || QUIET="--silent" - if [ -z "$MVNW_USERNAME" ] || [ -z "$MVNW_PASSWORD" ]; then - curl $QUIET -o "$wrapperJarPath" "$wrapperUrl" -f -L || rm -f "$wrapperJarPath" - else - curl $QUIET --user "$MVNW_USERNAME:$MVNW_PASSWORD" -o "$wrapperJarPath" "$wrapperUrl" -f -L || rm -f "$wrapperJarPath" - fi - else - log "Falling back to using Java to download" - javaSource="$MAVEN_PROJECTBASEDIR/.mvn/wrapper/MavenWrapperDownloader.java" - javaClass="$MAVEN_PROJECTBASEDIR/.mvn/wrapper/MavenWrapperDownloader.class" - # For Cygwin, switch paths to Windows format before running javac - if $cygwin; then - javaSource=$(cygpath --path --windows "$javaSource") - javaClass=$(cygpath --path --windows "$javaClass") - fi - if [ -e "$javaSource" ]; then - if [ ! -e "$javaClass" ]; then - log " - Compiling MavenWrapperDownloader.java ..." - ("$JAVA_HOME/bin/javac" "$javaSource") - fi - if [ -e "$javaClass" ]; then - log " - Running MavenWrapperDownloader.java ..." - ("$JAVA_HOME/bin/java" -cp .mvn/wrapper MavenWrapperDownloader "$wrapperUrl" "$wrapperJarPath") || rm -f "$wrapperJarPath" - fi - fi - fi +# select .zip or .tar.gz +if ! command -v unzip >/dev/null; then + distributionUrl="${distributionUrl%.zip}.tar.gz" + distributionUrlName="${distributionUrl##*/}" fi -########################################################################################## -# End of extension -########################################################################################## -# If specified, validate the SHA-256 sum of the Maven wrapper jar file -wrapperSha256Sum="" -while IFS="=" read -r key value; do - case "$key" in (wrapperSha256Sum) wrapperSha256Sum=$value; break ;; - esac -done < "$MAVEN_PROJECTBASEDIR/.mvn/wrapper/maven-wrapper.properties" -if [ -n "$wrapperSha256Sum" ]; then - wrapperSha256Result=false - if command -v sha256sum > /dev/null; then - if echo "$wrapperSha256Sum $wrapperJarPath" | sha256sum -c > /dev/null 2>&1; then - wrapperSha256Result=true +# verbose opt +__MVNW_QUIET_WGET=--quiet __MVNW_QUIET_CURL=--silent __MVNW_QUIET_UNZIP=-q __MVNW_QUIET_TAR='' +[ "${MVNW_VERBOSE-}" != true ] || __MVNW_QUIET_WGET='' __MVNW_QUIET_CURL='' __MVNW_QUIET_UNZIP='' __MVNW_QUIET_TAR=v + +# normalize http auth +case "${MVNW_PASSWORD:+has-password}" in +'') MVNW_USERNAME='' MVNW_PASSWORD='' ;; +has-password) [ -n "${MVNW_USERNAME-}" ] || MVNW_USERNAME='' MVNW_PASSWORD='' ;; +esac + +if [ -z "${MVNW_USERNAME-}" ] && command -v wget >/dev/null; then + verbose "Found wget ... using wget" + wget ${__MVNW_QUIET_WGET:+"$__MVNW_QUIET_WGET"} "$distributionUrl" -O "$TMP_DOWNLOAD_DIR/$distributionUrlName" || die "wget: Failed to fetch $distributionUrl" +elif [ -z "${MVNW_USERNAME-}" ] && command -v curl >/dev/null; then + verbose "Found curl ... using curl" + curl ${__MVNW_QUIET_CURL:+"$__MVNW_QUIET_CURL"} -f -L -o "$TMP_DOWNLOAD_DIR/$distributionUrlName" "$distributionUrl" || die "curl: Failed to fetch $distributionUrl" +elif set_java_home; then + verbose "Falling back to use Java to download" + javaSource="$TMP_DOWNLOAD_DIR/Downloader.java" + targetZip="$TMP_DOWNLOAD_DIR/$distributionUrlName" + cat >"$javaSource" <<-END + public class Downloader extends java.net.Authenticator + { + protected java.net.PasswordAuthentication getPasswordAuthentication() + { + return new java.net.PasswordAuthentication( System.getenv( "MVNW_USERNAME" ), System.getenv( "MVNW_PASSWORD" ).toCharArray() ); + } + public static void main( String[] args ) throws Exception + { + setDefault( new Downloader() ); + java.nio.file.Files.copy( java.net.URI.create( args[0] ).toURL().openStream(), java.nio.file.Paths.get( args[1] ).toAbsolutePath().normalize() ); + } + } + END + # For Cygwin/MinGW, switch paths to Windows format before running javac and java + verbose " - Compiling Downloader.java ..." + "$(native_path "$JAVACCMD")" "$(native_path "$javaSource")" || die "Failed to compile Downloader.java" + verbose " - Running Downloader.java ..." + "$(native_path "$JAVACMD")" -cp "$(native_path "$TMP_DOWNLOAD_DIR")" Downloader "$distributionUrl" "$(native_path "$targetZip")" +fi + +# If specified, validate the SHA-256 sum of the Maven distribution zip file +if [ -n "${distributionSha256Sum-}" ]; then + distributionSha256Result=false + if [ "$MVN_CMD" = mvnd.sh ]; then + echo "Checksum validation is not supported for maven-mvnd." >&2 + echo "Please disable validation by removing 'distributionSha256Sum' from your maven-wrapper.properties." >&2 + exit 1 + elif command -v sha256sum >/dev/null; then + if echo "$distributionSha256Sum $TMP_DOWNLOAD_DIR/$distributionUrlName" | sha256sum -c >/dev/null 2>&1; then + distributionSha256Result=true fi - elif command -v shasum > /dev/null; then - if echo "$wrapperSha256Sum $wrapperJarPath" | shasum -a 256 -c > /dev/null 2>&1; then - wrapperSha256Result=true + elif command -v shasum >/dev/null; then + if echo "$distributionSha256Sum $TMP_DOWNLOAD_DIR/$distributionUrlName" | shasum -a 256 -c >/dev/null 2>&1; then + distributionSha256Result=true fi else - echo "Checksum validation was requested but neither 'sha256sum' or 'shasum' are available." - echo "Please install either command, or disable validation by removing 'wrapperSha256Sum' from your maven-wrapper.properties." + echo "Checksum validation was requested but neither 'sha256sum' or 'shasum' are available." >&2 + echo "Please install either command, or disable validation by removing 'distributionSha256Sum' from your maven-wrapper.properties." >&2 exit 1 fi - if [ $wrapperSha256Result = false ]; then - echo "Error: Failed to validate Maven wrapper SHA-256, your Maven wrapper might be compromised." >&2 - echo "Investigate or delete $wrapperJarPath to attempt a clean download." >&2 - echo "If you updated your Maven version, you need to update the specified wrapperSha256Sum property." >&2 + if [ $distributionSha256Result = false ]; then + echo "Error: Failed to validate Maven distribution SHA-256, your Maven distribution might be compromised." >&2 + echo "If you updated your Maven version, you need to update the specified distributionSha256Sum property." >&2 exit 1 fi fi -MAVEN_OPTS="$(concat_lines "$MAVEN_PROJECTBASEDIR/.mvn/jvm.config") $MAVEN_OPTS" - -# For Cygwin, switch paths to Windows format before running java -if $cygwin; then - [ -n "$JAVA_HOME" ] && - JAVA_HOME=$(cygpath --path --windows "$JAVA_HOME") - [ -n "$CLASSPATH" ] && - CLASSPATH=$(cygpath --path --windows "$CLASSPATH") - [ -n "$MAVEN_PROJECTBASEDIR" ] && - MAVEN_PROJECTBASEDIR=$(cygpath --path --windows "$MAVEN_PROJECTBASEDIR") +# unzip and move +if command -v unzip >/dev/null; then + unzip ${__MVNW_QUIET_UNZIP:+"$__MVNW_QUIET_UNZIP"} "$TMP_DOWNLOAD_DIR/$distributionUrlName" -d "$TMP_DOWNLOAD_DIR" || die "failed to unzip" +else + tar xzf${__MVNW_QUIET_TAR:+"$__MVNW_QUIET_TAR"} "$TMP_DOWNLOAD_DIR/$distributionUrlName" -C "$TMP_DOWNLOAD_DIR" || die "failed to untar" fi +printf %s\\n "$distributionUrl" >"$TMP_DOWNLOAD_DIR/$distributionUrlNameMain/mvnw.url" +mv -- "$TMP_DOWNLOAD_DIR/$distributionUrlNameMain" "$MAVEN_HOME" || [ -d "$MAVEN_HOME" ] || die "fail to move MAVEN_HOME" -# Provide a "standardized" way to retrieve the CLI args that will -# work with both Windows and non-Windows executions. -MAVEN_CMD_LINE_ARGS="$MAVEN_CONFIG $*" -export MAVEN_CMD_LINE_ARGS - -WRAPPER_LAUNCHER=org.apache.maven.wrapper.MavenWrapperMain - -# shellcheck disable=SC2086 # safe args -exec "$JAVACMD" \ - $MAVEN_OPTS \ - $MAVEN_DEBUG_OPTS \ - -classpath "$MAVEN_PROJECTBASEDIR/.mvn/wrapper/maven-wrapper.jar" \ - "-Dmaven.multiModuleProjectDirectory=${MAVEN_PROJECTBASEDIR}" \ - ${WRAPPER_LAUNCHER} $MAVEN_CONFIG "$@" +clean || : +exec_maven "$@" diff --git a/mvnw.cmd b/mvnw.cmd index f80fbad3e7..b150b91ed5 100644 --- a/mvnw.cmd +++ b/mvnw.cmd @@ -1,3 +1,4 @@ +<# : batch portion @REM ---------------------------------------------------------------------------- @REM Licensed to the Apache Software Foundation (ASF) under one @REM or more contributor license agreements. See the NOTICE file @@ -18,188 +19,131 @@ @REM ---------------------------------------------------------------------------- @REM ---------------------------------------------------------------------------- -@REM Apache Maven Wrapper startup batch script, version 3.2.0 -@REM -@REM Required ENV vars: -@REM JAVA_HOME - location of a JDK home dir +@REM Apache Maven Wrapper startup batch script, version 3.3.2 @REM @REM Optional ENV vars -@REM MAVEN_BATCH_ECHO - set to 'on' to enable the echoing of the batch commands -@REM MAVEN_BATCH_PAUSE - set to 'on' to wait for a keystroke before ending -@REM MAVEN_OPTS - parameters passed to the Java VM when running Maven -@REM e.g. to debug Maven itself, use -@REM set MAVEN_OPTS=-Xdebug -Xrunjdwp:transport=dt_socket,server=y,suspend=y,address=8000 -@REM MAVEN_SKIP_RC - flag to disable loading of mavenrc files +@REM MVNW_REPOURL - repo url base for downloading maven distribution +@REM MVNW_USERNAME/MVNW_PASSWORD - user and password for downloading maven +@REM MVNW_VERBOSE - true: enable verbose log; others: silence the output @REM ---------------------------------------------------------------------------- -@REM Begin all REM lines with '@' in case MAVEN_BATCH_ECHO is 'on' -@echo off -@REM set title of command window -title %0 -@REM enable echoing by setting MAVEN_BATCH_ECHO to 'on' -@if "%MAVEN_BATCH_ECHO%" == "on" echo %MAVEN_BATCH_ECHO% - -@REM set %HOME% to equivalent of $HOME -if "%HOME%" == "" (set "HOME=%HOMEDRIVE%%HOMEPATH%") - -@REM Execute a user defined script before this one -if not "%MAVEN_SKIP_RC%" == "" goto skipRcPre -@REM check for pre script, once with legacy .bat ending and once with .cmd ending -if exist "%USERPROFILE%\mavenrc_pre.bat" call "%USERPROFILE%\mavenrc_pre.bat" %* -if exist "%USERPROFILE%\mavenrc_pre.cmd" call "%USERPROFILE%\mavenrc_pre.cmd" %* -:skipRcPre - -@setlocal - -set ERROR_CODE=0 - -@REM To isolate internal variables from possible post scripts, we use another setlocal -@setlocal - -@REM ==== START VALIDATION ==== -if not "%JAVA_HOME%" == "" goto OkJHome - -echo. -echo Error: JAVA_HOME not found in your environment. >&2 -echo Please set the JAVA_HOME variable in your environment to match the >&2 -echo location of your Java installation. >&2 -echo. -goto error - -:OkJHome -if exist "%JAVA_HOME%\bin\java.exe" goto init - -echo. -echo Error: JAVA_HOME is set to an invalid directory. >&2 -echo JAVA_HOME = "%JAVA_HOME%" >&2 -echo Please set the JAVA_HOME variable in your environment to match the >&2 -echo location of your Java installation. >&2 -echo. -goto error - -@REM ==== END VALIDATION ==== - -:init - -@REM Find the project base dir, i.e. the directory that contains the folder ".mvn". -@REM Fallback to current working directory if not found. - -set MAVEN_PROJECTBASEDIR=%MAVEN_BASEDIR% -IF NOT "%MAVEN_PROJECTBASEDIR%"=="" goto endDetectBaseDir - -set EXEC_DIR=%CD% -set WDIR=%EXEC_DIR% -:findBaseDir -IF EXIST "%WDIR%"\.mvn goto baseDirFound -cd .. -IF "%WDIR%"=="%CD%" goto baseDirNotFound -set WDIR=%CD% -goto findBaseDir - -:baseDirFound -set MAVEN_PROJECTBASEDIR=%WDIR% -cd "%EXEC_DIR%" -goto endDetectBaseDir - -:baseDirNotFound -set MAVEN_PROJECTBASEDIR=%EXEC_DIR% -cd "%EXEC_DIR%" - -:endDetectBaseDir - -IF NOT EXIST "%MAVEN_PROJECTBASEDIR%\.mvn\jvm.config" goto endReadAdditionalConfig - -@setlocal EnableExtensions EnableDelayedExpansion -for /F "usebackq delims=" %%a in ("%MAVEN_PROJECTBASEDIR%\.mvn\jvm.config") do set JVM_CONFIG_MAVEN_PROPS=!JVM_CONFIG_MAVEN_PROPS! %%a -@endlocal & set JVM_CONFIG_MAVEN_PROPS=%JVM_CONFIG_MAVEN_PROPS% - -:endReadAdditionalConfig - -SET MAVEN_JAVA_EXE="%JAVA_HOME%\bin\java.exe" -set WRAPPER_JAR="%MAVEN_PROJECTBASEDIR%\.mvn\wrapper\maven-wrapper.jar" -set WRAPPER_LAUNCHER=org.apache.maven.wrapper.MavenWrapperMain - -set WRAPPER_URL="https://repo.maven.apache.org/maven2/org/apache/maven/wrapper/maven-wrapper/3.2.0/maven-wrapper-3.2.0.jar" - -FOR /F "usebackq tokens=1,2 delims==" %%A IN ("%MAVEN_PROJECTBASEDIR%\.mvn\wrapper\maven-wrapper.properties") DO ( - IF "%%A"=="wrapperUrl" SET WRAPPER_URL=%%B -) - -@REM Extension to allow automatically downloading the maven-wrapper.jar from Maven-central -@REM This allows using the maven wrapper in projects that prohibit checking in binary data. -if exist %WRAPPER_JAR% ( - if "%MVNW_VERBOSE%" == "true" ( - echo Found %WRAPPER_JAR% - ) -) else ( - if not "%MVNW_REPOURL%" == "" ( - SET WRAPPER_URL="%MVNW_REPOURL%/org/apache/maven/wrapper/maven-wrapper/3.2.0/maven-wrapper-3.2.0.jar" - ) - if "%MVNW_VERBOSE%" == "true" ( - echo Couldn't find %WRAPPER_JAR%, downloading it ... - echo Downloading from: %WRAPPER_URL% - ) - - powershell -Command "&{"^ - "$webclient = new-object System.Net.WebClient;"^ - "if (-not ([string]::IsNullOrEmpty('%MVNW_USERNAME%') -and [string]::IsNullOrEmpty('%MVNW_PASSWORD%'))) {"^ - "$webclient.Credentials = new-object System.Net.NetworkCredential('%MVNW_USERNAME%', '%MVNW_PASSWORD%');"^ - "}"^ - "[Net.ServicePointManager]::SecurityProtocol = [Net.SecurityProtocolType]::Tls12; $webclient.DownloadFile('%WRAPPER_URL%', '%WRAPPER_JAR%')"^ - "}" - if "%MVNW_VERBOSE%" == "true" ( - echo Finished downloading %WRAPPER_JAR% - ) -) -@REM End of extension - -@REM If specified, validate the SHA-256 sum of the Maven wrapper jar file -SET WRAPPER_SHA_256_SUM="" -FOR /F "usebackq tokens=1,2 delims==" %%A IN ("%MAVEN_PROJECTBASEDIR%\.mvn\wrapper\maven-wrapper.properties") DO ( - IF "%%A"=="wrapperSha256Sum" SET WRAPPER_SHA_256_SUM=%%B +@IF "%__MVNW_ARG0_NAME__%"=="" (SET __MVNW_ARG0_NAME__=%~nx0) +@SET __MVNW_CMD__= +@SET __MVNW_ERROR__= +@SET __MVNW_PSMODULEP_SAVE=%PSModulePath% +@SET PSModulePath= +@FOR /F "usebackq tokens=1* delims==" %%A IN (`powershell -noprofile "& {$scriptDir='%~dp0'; $script='%__MVNW_ARG0_NAME__%'; icm -ScriptBlock ([Scriptblock]::Create((Get-Content -Raw '%~f0'))) -NoNewScope}"`) DO @( + IF "%%A"=="MVN_CMD" (set __MVNW_CMD__=%%B) ELSE IF "%%B"=="" (echo %%A) ELSE (echo %%A=%%B) ) -IF NOT %WRAPPER_SHA_256_SUM%=="" ( - powershell -Command "&{"^ - "$hash = (Get-FileHash \"%WRAPPER_JAR%\" -Algorithm SHA256).Hash.ToLower();"^ - "If('%WRAPPER_SHA_256_SUM%' -ne $hash){"^ - " Write-Output 'Error: Failed to validate Maven wrapper SHA-256, your Maven wrapper might be compromised.';"^ - " Write-Output 'Investigate or delete %WRAPPER_JAR% to attempt a clean download.';"^ - " Write-Output 'If you updated your Maven version, you need to update the specified wrapperSha256Sum property.';"^ - " exit 1;"^ - "}"^ - "}" - if ERRORLEVEL 1 goto error -) - -@REM Provide a "standardized" way to retrieve the CLI args that will -@REM work with both Windows and non-Windows executions. -set MAVEN_CMD_LINE_ARGS=%* - -%MAVEN_JAVA_EXE% ^ - %JVM_CONFIG_MAVEN_PROPS% ^ - %MAVEN_OPTS% ^ - %MAVEN_DEBUG_OPTS% ^ - -classpath %WRAPPER_JAR% ^ - "-Dmaven.multiModuleProjectDirectory=%MAVEN_PROJECTBASEDIR%" ^ - %WRAPPER_LAUNCHER% %MAVEN_CONFIG% %* -if ERRORLEVEL 1 goto error -goto end - -:error -set ERROR_CODE=1 - -:end -@endlocal & set ERROR_CODE=%ERROR_CODE% - -if not "%MAVEN_SKIP_RC%"=="" goto skipRcPost -@REM check for post script, once with legacy .bat ending and once with .cmd ending -if exist "%USERPROFILE%\mavenrc_post.bat" call "%USERPROFILE%\mavenrc_post.bat" -if exist "%USERPROFILE%\mavenrc_post.cmd" call "%USERPROFILE%\mavenrc_post.cmd" -:skipRcPost - -@REM pause the script if MAVEN_BATCH_PAUSE is set to 'on' -if "%MAVEN_BATCH_PAUSE%"=="on" pause - -if "%MAVEN_TERMINATE_CMD%"=="on" exit %ERROR_CODE% - -cmd /C exit /B %ERROR_CODE% +@SET PSModulePath=%__MVNW_PSMODULEP_SAVE% +@SET __MVNW_PSMODULEP_SAVE= +@SET __MVNW_ARG0_NAME__= +@SET MVNW_USERNAME= +@SET MVNW_PASSWORD= +@IF NOT "%__MVNW_CMD__%"=="" (%__MVNW_CMD__% %*) +@echo Cannot start maven from wrapper >&2 && exit /b 1 +@GOTO :EOF +: end batch / begin powershell #> + +$ErrorActionPreference = "Stop" +if ($env:MVNW_VERBOSE -eq "true") { + $VerbosePreference = "Continue" +} + +# calculate distributionUrl, requires .mvn/wrapper/maven-wrapper.properties +$distributionUrl = (Get-Content -Raw "$scriptDir/.mvn/wrapper/maven-wrapper.properties" | ConvertFrom-StringData).distributionUrl +if (!$distributionUrl) { + Write-Error "cannot read distributionUrl property in $scriptDir/.mvn/wrapper/maven-wrapper.properties" +} + +switch -wildcard -casesensitive ( $($distributionUrl -replace '^.*/','') ) { + "maven-mvnd-*" { + $USE_MVND = $true + $distributionUrl = $distributionUrl -replace '-bin\.[^.]*$',"-windows-amd64.zip" + $MVN_CMD = "mvnd.cmd" + break + } + default { + $USE_MVND = $false + $MVN_CMD = $script -replace '^mvnw','mvn' + break + } +} + +# apply MVNW_REPOURL and calculate MAVEN_HOME +# maven home pattern: ~/.m2/wrapper/dists/{apache-maven-,maven-mvnd--}/ +if ($env:MVNW_REPOURL) { + $MVNW_REPO_PATTERN = if ($USE_MVND) { "/org/apache/maven/" } else { "/maven/mvnd/" } + $distributionUrl = "$env:MVNW_REPOURL$MVNW_REPO_PATTERN$($distributionUrl -replace '^.*'+$MVNW_REPO_PATTERN,'')" +} +$distributionUrlName = $distributionUrl -replace '^.*/','' +$distributionUrlNameMain = $distributionUrlName -replace '\.[^.]*$','' -replace '-bin$','' +$MAVEN_HOME_PARENT = "$HOME/.m2/wrapper/dists/$distributionUrlNameMain" +if ($env:MAVEN_USER_HOME) { + $MAVEN_HOME_PARENT = "$env:MAVEN_USER_HOME/wrapper/dists/$distributionUrlNameMain" +} +$MAVEN_HOME_NAME = ([System.Security.Cryptography.MD5]::Create().ComputeHash([byte[]][char[]]$distributionUrl) | ForEach-Object {$_.ToString("x2")}) -join '' +$MAVEN_HOME = "$MAVEN_HOME_PARENT/$MAVEN_HOME_NAME" + +if (Test-Path -Path "$MAVEN_HOME" -PathType Container) { + Write-Verbose "found existing MAVEN_HOME at $MAVEN_HOME" + Write-Output "MVN_CMD=$MAVEN_HOME/bin/$MVN_CMD" + exit $? +} + +if (! $distributionUrlNameMain -or ($distributionUrlName -eq $distributionUrlNameMain)) { + Write-Error "distributionUrl is not valid, must end with *-bin.zip, but found $distributionUrl" +} + +# prepare tmp dir +$TMP_DOWNLOAD_DIR_HOLDER = New-TemporaryFile +$TMP_DOWNLOAD_DIR = New-Item -Itemtype Directory -Path "$TMP_DOWNLOAD_DIR_HOLDER.dir" +$TMP_DOWNLOAD_DIR_HOLDER.Delete() | Out-Null +trap { + if ($TMP_DOWNLOAD_DIR.Exists) { + try { Remove-Item $TMP_DOWNLOAD_DIR -Recurse -Force | Out-Null } + catch { Write-Warning "Cannot remove $TMP_DOWNLOAD_DIR" } + } +} + +New-Item -Itemtype Directory -Path "$MAVEN_HOME_PARENT" -Force | Out-Null + +# Download and Install Apache Maven +Write-Verbose "Couldn't find MAVEN_HOME, downloading and installing it ..." +Write-Verbose "Downloading from: $distributionUrl" +Write-Verbose "Downloading to: $TMP_DOWNLOAD_DIR/$distributionUrlName" + +$webclient = New-Object System.Net.WebClient +if ($env:MVNW_USERNAME -and $env:MVNW_PASSWORD) { + $webclient.Credentials = New-Object System.Net.NetworkCredential($env:MVNW_USERNAME, $env:MVNW_PASSWORD) +} +[Net.ServicePointManager]::SecurityProtocol = [Net.SecurityProtocolType]::Tls12 +$webclient.DownloadFile($distributionUrl, "$TMP_DOWNLOAD_DIR/$distributionUrlName") | Out-Null + +# If specified, validate the SHA-256 sum of the Maven distribution zip file +$distributionSha256Sum = (Get-Content -Raw "$scriptDir/.mvn/wrapper/maven-wrapper.properties" | ConvertFrom-StringData).distributionSha256Sum +if ($distributionSha256Sum) { + if ($USE_MVND) { + Write-Error "Checksum validation is not supported for maven-mvnd. `nPlease disable validation by removing 'distributionSha256Sum' from your maven-wrapper.properties." + } + Import-Module $PSHOME\Modules\Microsoft.PowerShell.Utility -Function Get-FileHash + if ((Get-FileHash "$TMP_DOWNLOAD_DIR/$distributionUrlName" -Algorithm SHA256).Hash.ToLower() -ne $distributionSha256Sum) { + Write-Error "Error: Failed to validate Maven distribution SHA-256, your Maven distribution might be compromised. If you updated your Maven version, you need to update the specified distributionSha256Sum property." + } +} + +# unzip and move +Expand-Archive "$TMP_DOWNLOAD_DIR/$distributionUrlName" -DestinationPath "$TMP_DOWNLOAD_DIR" | Out-Null +Rename-Item -Path "$TMP_DOWNLOAD_DIR/$distributionUrlNameMain" -NewName $MAVEN_HOME_NAME | Out-Null +try { + Move-Item -Path "$TMP_DOWNLOAD_DIR/$MAVEN_HOME_NAME" -Destination $MAVEN_HOME_PARENT | Out-Null +} catch { + if (! (Test-Path -Path "$MAVEN_HOME" -PathType Container)) { + Write-Error "fail to move MAVEN_HOME" + } +} finally { + try { Remove-Item $TMP_DOWNLOAD_DIR -Recurse -Force | Out-Null } + catch { Write-Warning "Cannot remove $TMP_DOWNLOAD_DIR" } +} + +Write-Output "MVN_CMD=$MAVEN_HOME/bin/$MVN_CMD" From e2a4156111eb87c861d497151cc9688a08f8b572 Mon Sep 17 00:00:00 2001 From: Tsz-Wo Nicholas Sze Date: Wed, 5 Mar 2025 08:38:37 -0800 Subject: [PATCH 158/397] RATIS-2255. Migrate ratis-grpc to junit 5. (#1233) --- ratis-grpc/pom.xml | 4 ++-- ratis-metrics-default/pom.xml | 5 ----- ratis-metrics-dropwizard3/pom.xml | 5 ----- 3 files changed, 2 insertions(+), 12 deletions(-) diff --git a/ratis-grpc/pom.xml b/ratis-grpc/pom.xml index e0736de195..554a1a763a 100644 --- a/ratis-grpc/pom.xml +++ b/ratis-grpc/pom.xml @@ -75,8 +75,8 @@ - junit - junit + org.junit.jupiter + junit-jupiter-api test diff --git a/ratis-metrics-default/pom.xml b/ratis-metrics-default/pom.xml index 7e45f42949..1f9e368622 100644 --- a/ratis-metrics-default/pom.xml +++ b/ratis-metrics-default/pom.xml @@ -49,11 +49,6 @@ slf4j-api - - junit - junit - test - org.junit.jupiter junit-jupiter-engine diff --git a/ratis-metrics-dropwizard3/pom.xml b/ratis-metrics-dropwizard3/pom.xml index d9bf14ddff..45707d8d58 100644 --- a/ratis-metrics-dropwizard3/pom.xml +++ b/ratis-metrics-dropwizard3/pom.xml @@ -53,11 +53,6 @@ slf4j-api - - junit - junit - test - org.junit.jupiter junit-jupiter-engine From 2acf08604d00830dead9f88b7d6468b2bd05310c Mon Sep 17 00:00:00 2001 From: "Doroszlai, Attila" Date: Mon, 17 Mar 2025 15:17:59 +0100 Subject: [PATCH 159/397] RATIS-2261. Mark TestRaftSnapshotWithGrpc as flaky --- .../java/org/apache/ratis/grpc/TestRaftSnapshotWithGrpc.java | 2 ++ 1 file changed, 2 insertions(+) diff --git a/ratis-test/src/test/java/org/apache/ratis/grpc/TestRaftSnapshotWithGrpc.java b/ratis-test/src/test/java/org/apache/ratis/grpc/TestRaftSnapshotWithGrpc.java index 82318c43ee..e6c2f6613b 100644 --- a/ratis-test/src/test/java/org/apache/ratis/grpc/TestRaftSnapshotWithGrpc.java +++ b/ratis-test/src/test/java/org/apache/ratis/grpc/TestRaftSnapshotWithGrpc.java @@ -26,8 +26,10 @@ import org.apache.ratis.metrics.RatisMetricRegistry; import org.apache.ratis.server.RaftServer; import org.apache.ratis.statemachine.RaftSnapshotBaseTest; +import org.apache.ratis.test.tag.Flaky; import org.junit.jupiter.api.Assertions; +@Flaky("RATIS-2261") public class TestRaftSnapshotWithGrpc extends RaftSnapshotBaseTest { @Override public MiniRaftCluster.Factory getFactory() { From 57444778f209d0493224b078f5faaf6a2708864b Mon Sep 17 00:00:00 2001 From: "Doroszlai, Attila" Date: Mon, 17 Mar 2025 15:48:44 +0100 Subject: [PATCH 160/397] RATIS-2262. Mark RaftBasicTests#testStateMachineMetrics as flaky --- ratis-server/src/test/java/org/apache/ratis/RaftBasicTests.java | 2 ++ 1 file changed, 2 insertions(+) diff --git a/ratis-server/src/test/java/org/apache/ratis/RaftBasicTests.java b/ratis-server/src/test/java/org/apache/ratis/RaftBasicTests.java index b16905d9a0..adef385674 100644 --- a/ratis-server/src/test/java/org/apache/ratis/RaftBasicTests.java +++ b/ratis-server/src/test/java/org/apache/ratis/RaftBasicTests.java @@ -17,6 +17,7 @@ */ package org.apache.ratis; +import org.apache.ratis.test.tag.Flaky; import org.apache.ratis.thirdparty.com.codahale.metrics.Gauge; import org.apache.ratis.RaftTestUtil.SimpleMessage; import org.apache.ratis.client.RaftClient; @@ -454,6 +455,7 @@ public static void testRequestTimeout(boolean async, MiniRaftCluster cluster, Lo } } + @Flaky("RATIS-2262") @Test public void testStateMachineMetrics() throws Exception { runWithNewCluster(NUM_SERVERS, cluster -> runTestStateMachineMetrics(false, cluster)); From 5cdc54ed90765bddf31cadd326bfe95de1c6a99e Mon Sep 17 00:00:00 2001 From: "Doroszlai, Attila" <6454655+adoroszlai@users.noreply.github.com> Date: Mon, 17 Mar 2025 18:47:59 +0100 Subject: [PATCH 161/397] RATIS-2260. AtomicFileOutputStream should truncate temp file (#1237) --- .../java/org/apache/ratis/util/AtomicFileOutputStream.java | 7 +++++-- 1 file changed, 5 insertions(+), 2 deletions(-) diff --git a/ratis-common/src/main/java/org/apache/ratis/util/AtomicFileOutputStream.java b/ratis-common/src/main/java/org/apache/ratis/util/AtomicFileOutputStream.java index 530eb383c7..b54427066f 100644 --- a/ratis-common/src/main/java/org/apache/ratis/util/AtomicFileOutputStream.java +++ b/ratis-common/src/main/java/org/apache/ratis/util/AtomicFileOutputStream.java @@ -17,6 +17,10 @@ */ package org.apache.ratis.util; +import static java.nio.file.StandardOpenOption.CREATE; +import static java.nio.file.StandardOpenOption.TRUNCATE_EXISTING; +import static java.nio.file.StandardOpenOption.WRITE; + import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -24,7 +28,6 @@ import java.io.FilterOutputStream; import java.io.IOException; import java.nio.file.StandardCopyOption; -import java.nio.file.StandardOpenOption; import java.util.concurrent.atomic.AtomicBoolean; /** @@ -60,7 +63,7 @@ public AtomicFileOutputStream(File outFile) throws IOException { } public AtomicFileOutputStream(File outFile, File tmpFile) throws IOException { - super(FileUtils.newOutputStreamForceAtClose(tmpFile, StandardOpenOption.CREATE, StandardOpenOption.WRITE)); + super(FileUtils.newOutputStreamForceAtClose(tmpFile, CREATE, TRUNCATE_EXISTING, WRITE)); this.outFile = outFile.getAbsoluteFile(); this.tmpFile = tmpFile.getAbsoluteFile(); } From 5b2b1df8ad164fc7197dba214c03d31239d7cf9a Mon Sep 17 00:00:00 2001 From: "Doroszlai, Attila" <6454655+adoroszlai@users.noreply.github.com> Date: Mon, 17 Mar 2025 19:19:22 +0100 Subject: [PATCH 162/397] RATIS-2263. Bump sonar-maven-plugin to 5.0.0.4389 (#1238) --- dev-support/checks/sonar.sh | 2 +- pom.xml | 6 ++++++ 2 files changed, 7 insertions(+), 1 deletion(-) diff --git a/dev-support/checks/sonar.sh b/dev-support/checks/sonar.sh index 9646ab9d8f..55edbdaaea 100755 --- a/dev-support/checks/sonar.sh +++ b/dev-support/checks/sonar.sh @@ -24,6 +24,6 @@ if [ ! "$SONAR_TOKEN" ]; then fi ${MVN} -B verify -DskipShade -DskipTests \ - org.sonarsource.scanner.maven:sonar-maven-plugin:3.6.0.1398:sonar \ + sonar:sonar \ -Dsonar.coverage.jacoco.xmlReportPaths="$(pwd)/target/coverage/all.xml" \ -Dsonar.host.url=https://sonarcloud.io -Dsonar.organization=apache -Dsonar.projectKey=apache-ratis diff --git a/pom.xml b/pom.xml index 0a8b1c7319..df44796055 100644 --- a/pom.xml +++ b/pom.xml @@ -177,6 +177,7 @@ 3.0.0-M4 3.5.3 3.4.0 + 5.0.0.4389 @@ -732,6 +733,11 @@ cyclonedx-maven-plugin ${cyclonedx.version} + + org.sonarsource.scanner.maven + sonar-maven-plugin + ${sonar-maven-plugin.version} + From eed0760a3a9fe48876b0d21c4c10ebed3a526af4 Mon Sep 17 00:00:00 2001 From: Tsz-Wo Nicholas Sze Date: Wed, 19 Mar 2025 02:30:47 -0700 Subject: [PATCH 163/397] RATIS-2258. Caching TermIndex objects (#1239) --- .../apache/ratis/util/BiWeakValueCache.java | 144 ++++++++++++++++++ .../ratis/server/protocol/TermIndex.java | 92 ++++++----- .../java/org/apache/ratis/RaftTestUtil.java | 15 ++ .../server/protocol/ProtocolTestUtils.java | 26 ++++ .../org/apache/ratis/util/TestTermIndex.java | 100 ++++++++++++ 5 files changed, 341 insertions(+), 36 deletions(-) create mode 100644 ratis-common/src/main/java/org/apache/ratis/util/BiWeakValueCache.java create mode 100644 ratis-test/src/test/java/org/apache/ratis/server/protocol/ProtocolTestUtils.java create mode 100644 ratis-test/src/test/java/org/apache/ratis/util/TestTermIndex.java diff --git a/ratis-common/src/main/java/org/apache/ratis/util/BiWeakValueCache.java b/ratis-common/src/main/java/org/apache/ratis/util/BiWeakValueCache.java new file mode 100644 index 0000000000..c1aa6bcd5a --- /dev/null +++ b/ratis-common/src/main/java/org/apache/ratis/util/BiWeakValueCache.java @@ -0,0 +1,144 @@ +/* + * 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.ratis.util; + +import org.apache.ratis.thirdparty.com.google.common.collect.MapMaker; + +import java.util.Map; +import java.util.Objects; +import java.util.concurrent.ConcurrentHashMap; +import java.util.concurrent.ConcurrentMap; +import java.util.concurrent.atomic.AtomicInteger; +import java.util.function.BiFunction; +import java.util.function.Consumer; + +/** + * Weak Value Cache: ({@link OUTER}, {@link INNER}) -> {@link T}. + *

+ * Note that the cached values are weakly referenced. + * A cached value could be garage-collected (i.e. evicted from the cache) + * when there are no external (strong) references. + * + * @param the type of the outer keys. + * @param the type of the inner keys. + * @param the type to be cached. + */ +public final class BiWeakValueCache { + private static ConcurrentMap newMap() { + return new MapMaker().weakValues().makeMap(); + } + + private final String outerName; + private final String innerName; + private final String name; + + /** For constructing {@link T} values from ({@link OUTER}, {@link INNER}) keys. */ + private final BiFunction constructor; + /** Count the number of {@link T} values constructed. */ + private final AtomicInteger valueCount = new AtomicInteger(0); + + /** + * Actual map {@link OUTER} -> ({@link INNER} -> {@link T}) + * for the logical view ({@link OUTER}, {@link INNER}) -> {@link T}. + */ + private final ConcurrentMap> map = new ConcurrentHashMap<>(); + + /** + * Create a cache for mapping ({@link OUTER}, {@link INNER}) keys to {@link T} values. + * + * @param outerName the name of the outer long. + * @param innerName the name of the inner long. + * @param constructor for constructing {@link T} values. + */ + public BiWeakValueCache(String outerName, String innerName, BiFunction constructor) { + this.outerName = outerName; + this.innerName = innerName; + this.name = "(" + outerName + ", " + innerName + ")-cache"; + this.constructor = constructor; + } + + private T construct(OUTER outer, INNER inner) { + final T constructed = constructor.apply(outer, inner); + Objects.requireNonNull(constructed, "constructed == null"); + valueCount.incrementAndGet(); + return constructed; + } + + /** + * If the key ({@link OUTER}, {@link INNER}) is in the cache, return the cached values. + * Otherwise, create a new value and then return it. + */ + public T getOrCreate(OUTER outer, INNER inner) { + Objects.requireNonNull(outer, () -> outerName + " (outer) == null"); + Objects.requireNonNull(inner, () -> innerName + " (inner) == null"); + final ConcurrentMap innerMap = map.computeIfAbsent(outer, k -> newMap()); + final T computed = innerMap.computeIfAbsent(inner, i -> construct(outer, i)); + if ((valueCount.get() & 0xFFF) == 0) { + cleanupEmptyInnerMaps(); // cleanup empty maps once in a while + } + return computed; + } + + /** @return the value count for the given outer key. */ + int count(OUTER outer) { + final ConcurrentMap innerMap = map.get(outer); + if (innerMap == null) { + return 0; + } + + // size() may return incorrect result; see Guava MapMaker javadoc + int n = 0; + for (INNER ignored : innerMap.keySet()) { + n++; + } + return n; + } + + void cleanupEmptyInnerMaps() { + // isEmpty() may return incorrect result; see Guava MapMaker javadoc + map.values().removeIf(e -> !e.entrySet().iterator().hasNext()); + } + + @Override + public String toString() { + return name; + } + + /** The cache content for debugging. */ + int dump(Consumer out) { + out.accept(name + ":\n"); + int emptyCount = 0; + for (Map.Entry> entry : map.entrySet()) { + final OUTER outer = entry.getKey(); + final ConcurrentMap innerMap = entry.getValue(); + final int count = count(outer); + if (count == 0) { + emptyCount++; + } + + out.accept(" " + outerName + ":" + outer); + out.accept(", " + innerName + ":" + innerMap.keySet()); + out.accept(", count=" + count); + out.accept(", size=" + innerMap.size()); + out.accept("\n"); + } + out.accept(" emptyCount=" + emptyCount); + out.accept("\n"); + return emptyCount; + } +} diff --git a/ratis-server-api/src/main/java/org/apache/ratis/server/protocol/TermIndex.java b/ratis-server-api/src/main/java/org/apache/ratis/server/protocol/TermIndex.java index dac1a51d23..6115bccadf 100644 --- a/ratis-server-api/src/main/java/org/apache/ratis/server/protocol/TermIndex.java +++ b/ratis-server-api/src/main/java/org/apache/ratis/server/protocol/TermIndex.java @@ -20,6 +20,7 @@ import org.apache.ratis.proto.RaftProtos.LogEntryProto; import org.apache.ratis.proto.RaftProtos.TermIndexProto; import org.apache.ratis.server.raftlog.RaftLog; +import org.apache.ratis.util.BiWeakValueCache; import java.util.Comparator; import java.util.Optional; @@ -73,43 +74,62 @@ static TermIndex valueOf(LogEntryProto proto) { /** @return a {@link TermIndex} object. */ static TermIndex valueOf(long term, long index) { - return new TermIndex() { - @Override - public long getTerm() { - return term; - } - - @Override - public long getIndex() { - return index; - } - - @Override - public boolean equals(Object obj) { - if (obj == this) { - return true; - } else if (!(obj instanceof TermIndex)) { - return false; + return Impl.getCache().getOrCreate(term, index); + } + + /** + * An implementation for private use. + * Note that this is not a public API, although this is public class. + */ + final class Impl { + private Impl() { } + + private static final BiWeakValueCache CACHE + = new BiWeakValueCache<>("term", "index", Impl::newTermIndex); + + static BiWeakValueCache getCache() { + return CACHE; + } + + private static TermIndex newTermIndex(long term, long index) { + return new TermIndex() { + @Override + public long getTerm() { + return term; + } + + @Override + public long getIndex() { + return index; + } + + @Override + public boolean equals(Object obj) { + if (obj == this) { + return true; + } else if (!(obj instanceof TermIndex)) { + return false; + } + + final TermIndex that = (TermIndex) obj; + return this.getTerm() == that.getTerm() + && this.getIndex() == that.getIndex(); + } + + @Override + public int hashCode() { + return Long.hashCode(term) ^ Long.hashCode(index); + } + + private String longToString(long n) { + return n >= 0L ? String.valueOf(n) : "~"; } - final TermIndex that = (TermIndex) obj; - return this.getTerm() == that.getTerm() - && this.getIndex() == that.getIndex(); - } - - @Override - public int hashCode() { - return Long.hashCode(term) ^ Long.hashCode(index); - } - - private String longToString(long n) { - return n >= 0L? String.valueOf(n) : "~"; - } - - @Override - public String toString() { - return String.format("(t:%s, i:%s)", longToString(term), longToString(index)); - } - }; + @Override + public String toString() { + return String.format("(t:%s, i:%s)", longToString(term), longToString(index)); + } + }; + } } } \ No newline at end of file diff --git a/ratis-server/src/test/java/org/apache/ratis/RaftTestUtil.java b/ratis-server/src/test/java/org/apache/ratis/RaftTestUtil.java index 3c1b3eda24..206b6dd949 100644 --- a/ratis-server/src/test/java/org/apache/ratis/RaftTestUtil.java +++ b/ratis-server/src/test/java/org/apache/ratis/RaftTestUtil.java @@ -50,6 +50,7 @@ import org.slf4j.LoggerFactory; import java.io.IOException; +import java.lang.ref.WeakReference; import java.lang.reflect.Field; import java.util.ArrayList; import java.util.Arrays; @@ -590,4 +591,18 @@ static void assertSuccessReply(RaftClientReply reply) { Assert.assertNotNull("reply == null", reply); Assert.assertTrue("reply is not success: " + reply, reply.isSuccess()); } + + static void gc() throws InterruptedException { + // use WeakReference to detect gc + Object obj = new Object(); + final WeakReference weakRef = new WeakReference<>(obj); + obj = null; + + // loop until gc has completed. + for (int i = 0; weakRef.get() != null; i++) { + LOG.info("gc {}", i); + System.gc(); + Thread.sleep(100); + } + } } diff --git a/ratis-test/src/test/java/org/apache/ratis/server/protocol/ProtocolTestUtils.java b/ratis-test/src/test/java/org/apache/ratis/server/protocol/ProtocolTestUtils.java new file mode 100644 index 0000000000..dee3f224c4 --- /dev/null +++ b/ratis-test/src/test/java/org/apache/ratis/server/protocol/ProtocolTestUtils.java @@ -0,0 +1,26 @@ +/* + * 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.ratis.server.protocol; + +import org.apache.ratis.util.BiWeakValueCache; + +public interface ProtocolTestUtils { + static BiWeakValueCache getTermIndexCache() { + return TermIndex.Impl.getCache(); + } +} \ No newline at end of file diff --git a/ratis-test/src/test/java/org/apache/ratis/util/TestTermIndex.java b/ratis-test/src/test/java/org/apache/ratis/util/TestTermIndex.java new file mode 100644 index 0000000000..678d7afe68 --- /dev/null +++ b/ratis-test/src/test/java/org/apache/ratis/util/TestTermIndex.java @@ -0,0 +1,100 @@ +/* + * 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.ratis.util; + +import org.apache.ratis.BaseTest; +import org.apache.ratis.RaftTestUtil; +import org.apache.ratis.server.protocol.ProtocolTestUtils; +import org.apache.ratis.server.protocol.TermIndex; +import org.junit.jupiter.api.Test; + +import static org.junit.jupiter.api.Assertions.*; + +/** Testing {@link BiWeakValueCache}. */ +public class TestTermIndex extends BaseTest { + static BiWeakValueCache CACHE = ProtocolTestUtils.getTermIndexCache(); + + static void dumpCache(Integer expectedEmptyCount) { + final int computed = CACHE.dump(System.out::print); + if (expectedEmptyCount != null) { + assertEquals(expectedEmptyCount, computed); + } + System.out.flush(); + } + + static void assertCacheSize(int expectedSize, long term) { + final int computed = CACHE.count(term); + if (computed != expectedSize) { + dumpCache(null); + } + assertEquals(expectedSize, computed); + } + + void assertCacheSizeWithGC(int expectedSize, long term) throws Exception{ + JavaUtils.attempt(() -> { + RaftTestUtil.gc(); + assertCacheSize(expectedSize, term); + }, 5, HUNDRED_MILLIS, "assertCacheSizeWithGC", LOG); + } + + static void initTermIndex(TermIndex[][] ti, int term, int index) { + ti[term][index] = TermIndex.valueOf(term, index); + } + + @Test + public void testCaching() throws Exception { + final int n = 9; + final TermIndex[][] ti = new TermIndex[n][n]; + final long[] terms = new long[n]; + final long[] indices = new long[n]; + for(int j = 0; j < n; j++) { + terms[j] = j; + indices[j] = j; + } + + assertCacheSize(0, terms[1]); + initTermIndex(ti, 1, 1); + assertSame(ti[1][1], TermIndex.valueOf(terms[1], indices[1])); + assertCacheSize(1, terms[1]); + + initTermIndex(ti, 1, 2); + assertSame(ti[1][1], TermIndex.valueOf(terms[1], indices[1])); + assertSame(ti[1][2], TermIndex.valueOf(terms[1], indices[2])); + assertCacheSize(2, terms[1]); + dumpCache(0); + + initTermIndex(ti, 2, 2); + assertSame(ti[1][1], TermIndex.valueOf(terms[1], indices[1])); + assertSame(ti[1][2], TermIndex.valueOf(terms[1], indices[2])); + assertSame(ti[2][2], TermIndex.valueOf(terms[2], indices[2])); + assertCacheSize(2, terms[1]); + assertCacheSize(1, terms[2]); + dumpCache(0); + + ti[1][1] = null; // release ti[1][1]; + assertCacheSizeWithGC(1, terms[1]); + dumpCache(0); + + ti[1][2] = null; // release ti[1][2]; + assertCacheSizeWithGC(0, terms[1]); + dumpCache(1); + + CACHE.cleanupEmptyInnerMaps(); + dumpCache(0); + } +} From 0657d41d0871fb2e9b012ef2549f2cac8aa7bf94 Mon Sep 17 00:00:00 2001 From: Tsz-Wo Nicholas Sze Date: Sat, 22 Mar 2025 16:47:27 -0700 Subject: [PATCH 164/397] RATIS-2266. Use WeakValueCache instead of Guava cache in RaftId. (#1240) --- .../org/apache/ratis/protocol/ClientId.java | 7 +- .../apache/ratis/protocol/RaftGroupId.java | 7 +- .../org/apache/ratis/protocol/RaftId.java | 22 +-- .../apache/ratis/util/BiWeakValueCache.java | 8 +- .../org/apache/ratis/util/WeakValueCache.java | 89 ++++++++++++ .../org/apache/ratis/protocol/TestRaftId.java | 8 ++ .../apache/ratis/util/TestRaftIdCache.java | 129 ++++++++++++++++++ 7 files changed, 254 insertions(+), 16 deletions(-) create mode 100644 ratis-common/src/main/java/org/apache/ratis/util/WeakValueCache.java create mode 100644 ratis-test/src/test/java/org/apache/ratis/util/TestRaftIdCache.java diff --git a/ratis-common/src/main/java/org/apache/ratis/protocol/ClientId.java b/ratis-common/src/main/java/org/apache/ratis/protocol/ClientId.java index 4de615730c..09b77e6e81 100644 --- a/ratis-common/src/main/java/org/apache/ratis/protocol/ClientId.java +++ b/ratis-common/src/main/java/org/apache/ratis/protocol/ClientId.java @@ -18,6 +18,7 @@ package org.apache.ratis.protocol; import org.apache.ratis.thirdparty.com.google.protobuf.ByteString; +import org.apache.ratis.util.WeakValueCache; import java.util.UUID; @@ -26,13 +27,17 @@ * to correctly identify retry requests from the same client. */ public final class ClientId extends RaftId { - private static final Factory FACTORY = new Factory() { + private static final Factory FACTORY = new Factory(ClientId.class) { @Override ClientId newInstance(UUID uuid) { return new ClientId(uuid); } }; + static WeakValueCache getCache() { + return FACTORY.getCache(); + } + public static ClientId emptyClientId() { return FACTORY.emptyId(); } diff --git a/ratis-common/src/main/java/org/apache/ratis/protocol/RaftGroupId.java b/ratis-common/src/main/java/org/apache/ratis/protocol/RaftGroupId.java index 9caedf7574..af40746918 100644 --- a/ratis-common/src/main/java/org/apache/ratis/protocol/RaftGroupId.java +++ b/ratis-common/src/main/java/org/apache/ratis/protocol/RaftGroupId.java @@ -18,6 +18,7 @@ package org.apache.ratis.protocol; import org.apache.ratis.thirdparty.com.google.protobuf.ByteString; +import org.apache.ratis.util.WeakValueCache; import java.util.UUID; @@ -27,13 +28,17 @@ * This is a value-based class. */ public final class RaftGroupId extends RaftId { - private static final Factory FACTORY = new Factory() { + private static final Factory FACTORY = new Factory(RaftGroupId.class) { @Override RaftGroupId newInstance(UUID uuid) { return new RaftGroupId(uuid); } }; + static WeakValueCache getCache() { + return FACTORY.getCache(); + } + public static RaftGroupId emptyGroupId() { return FACTORY.emptyId(); } diff --git a/ratis-common/src/main/java/org/apache/ratis/protocol/RaftId.java b/ratis-common/src/main/java/org/apache/ratis/protocol/RaftId.java index 9c2a83ffa3..d8a3f73ab3 100644 --- a/ratis-common/src/main/java/org/apache/ratis/protocol/RaftId.java +++ b/ratis-common/src/main/java/org/apache/ratis/protocol/RaftId.java @@ -17,17 +17,15 @@ */ package org.apache.ratis.protocol; -import org.apache.ratis.thirdparty.com.google.common.cache.Cache; -import org.apache.ratis.thirdparty.com.google.common.cache.CacheBuilder; import org.apache.ratis.thirdparty.com.google.protobuf.ByteString; import org.apache.ratis.thirdparty.com.google.protobuf.UnsafeByteOperations; import org.apache.ratis.util.JavaUtils; import org.apache.ratis.util.Preconditions; +import org.apache.ratis.util.WeakValueCache; import java.nio.ByteBuffer; import java.util.Objects; import java.util.UUID; -import java.util.concurrent.ExecutionException; import java.util.function.Supplier; /** Unique identifier implemented using {@link UUID}. */ @@ -53,18 +51,20 @@ static ByteString toByteString(UUID uuid) { } abstract static class Factory { - private final Cache cache = CacheBuilder.newBuilder() - .weakValues() - .build(); + private final WeakValueCache cache; + + Factory(Class clazz) { + this.cache = new WeakValueCache<>(clazz.getSimpleName() + "_UUID", this::newInstance); + } abstract ID newInstance(UUID uuid); + WeakValueCache getCache() { + return cache; + } + final ID valueOf(UUID uuid) { - try { - return cache.get(uuid, () -> newInstance(uuid)); - } catch (ExecutionException e) { - throw new IllegalStateException("Failed to valueOf(" + uuid + ")", e); - } + return cache.getOrCreate(uuid); } final ID valueOf(ByteString bytes) { diff --git a/ratis-common/src/main/java/org/apache/ratis/util/BiWeakValueCache.java b/ratis-common/src/main/java/org/apache/ratis/util/BiWeakValueCache.java index c1aa6bcd5a..d7eaf5744a 100644 --- a/ratis-common/src/main/java/org/apache/ratis/util/BiWeakValueCache.java +++ b/ratis-common/src/main/java/org/apache/ratis/util/BiWeakValueCache.java @@ -33,13 +33,15 @@ * Note that the cached values are weakly referenced. * A cached value could be garage-collected (i.e. evicted from the cache) * when there are no external (strong) references. + *

+ * For key types with a component, use {@link WeakValueCache}. * * @param the type of the outer keys. * @param the type of the inner keys. * @param the type to be cached. */ public final class BiWeakValueCache { - private static ConcurrentMap newMap() { + static ConcurrentMap newMap() { return new MapMaker().weakValues().makeMap(); } @@ -61,8 +63,8 @@ private static ConcurrentMap newMap() { /** * Create a cache for mapping ({@link OUTER}, {@link INNER}) keys to {@link T} values. * - * @param outerName the name of the outer long. - * @param innerName the name of the inner long. + * @param outerName the name of the outer keys. + * @param innerName the name of the inner keys. * @param constructor for constructing {@link T} values. */ public BiWeakValueCache(String outerName, String innerName, BiFunction constructor) { diff --git a/ratis-common/src/main/java/org/apache/ratis/util/WeakValueCache.java b/ratis-common/src/main/java/org/apache/ratis/util/WeakValueCache.java new file mode 100644 index 0000000000..5c6fcd3683 --- /dev/null +++ b/ratis-common/src/main/java/org/apache/ratis/util/WeakValueCache.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.ratis.util; + +import java.util.ArrayList; +import java.util.List; +import java.util.Objects; +import java.util.concurrent.ConcurrentMap; +import java.util.concurrent.atomic.AtomicInteger; +import java.util.function.Function; + +import static org.apache.ratis.util.BiWeakValueCache.newMap; + +/** + * Weak Value Cache: {@link K} -> {@link V}. + *

+ * Note that the cached values are weakly referenced. + * A cached value could be garage-collected (i.e. evicted from the cache) + * when there are no external (strong) references. + *

+ * For key types with two components, use {@link BiWeakValueCache}. + * + * @param the type of the keys. + * @param the type to be cached values. + */ +public final class WeakValueCache { + private final String keyName; + private final String name; + + /** For constructing a value from a key. */ + private final Function constructor; + /** Count the number of values constructed. */ + private final AtomicInteger constructionCount = new AtomicInteger(0); + + /** Map: {@link K} -> {@link V}. */ + private final ConcurrentMap map = newMap(); + + /** + * Create a cache for mapping {@link K} keys to {@link V} values. + * + * @param keyName the name of the key. + * @param constructor for constructing {@link V} values. + */ + public WeakValueCache(String keyName, Function constructor) { + this.keyName = keyName; + this.name = keyName + "-cache"; + this.constructor = constructor; + } + + private V construct(K key) { + final V constructed = constructor.apply(key); + Objects.requireNonNull(constructed, "constructed == null"); + constructionCount.incrementAndGet(); + return constructed; + } + + /** + * If the given key is in the cache, return its cached values. + * Otherwise, create a new value, put it in the cache and then return it. + */ + public V getOrCreate(K key) { + Objects.requireNonNull(key, () -> keyName + " (key) == null"); + return map.computeIfAbsent(key, this::construct); + } + + List getValues() { + return new ArrayList<>(map.values()); + } + + @Override + public String toString() { + return name; + } +} diff --git a/ratis-test/src/test/java/org/apache/ratis/protocol/TestRaftId.java b/ratis-test/src/test/java/org/apache/ratis/protocol/TestRaftId.java index b0e31ce72e..907235e117 100644 --- a/ratis-test/src/test/java/org/apache/ratis/protocol/TestRaftId.java +++ b/ratis-test/src/test/java/org/apache/ratis/protocol/TestRaftId.java @@ -19,6 +19,7 @@ import org.apache.ratis.BaseTest; import org.apache.ratis.thirdparty.com.google.protobuf.ByteString; +import org.apache.ratis.util.WeakValueCache; import org.junit.jupiter.api.Assertions; import org.junit.jupiter.api.Test; import org.junit.jupiter.api.Timeout; @@ -27,6 +28,13 @@ @Timeout(value = 1) public class TestRaftId extends BaseTest { + public static WeakValueCache getClientIdCache() { + return ClientId.getCache(); + } + + public static WeakValueCache getRaftGroupIdCache() { + return RaftGroupId.getCache(); + } @Test public void testRaftId() { diff --git a/ratis-test/src/test/java/org/apache/ratis/util/TestRaftIdCache.java b/ratis-test/src/test/java/org/apache/ratis/util/TestRaftIdCache.java new file mode 100644 index 0000000000..16d5cd6526 --- /dev/null +++ b/ratis-test/src/test/java/org/apache/ratis/util/TestRaftIdCache.java @@ -0,0 +1,129 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.ratis.util; + +import org.apache.ratis.BaseTest; +import org.apache.ratis.RaftTestUtil; +import org.apache.ratis.protocol.ClientId; +import org.apache.ratis.protocol.TestRaftId; +import org.junit.jupiter.api.Test; + +import java.util.ArrayList; +import java.util.Comparator; +import java.util.LinkedList; +import java.util.List; +import java.util.UUID; +import java.util.concurrent.ThreadLocalRandom; + +import static org.junit.jupiter.api.Assertions.assertEquals; +import static org.junit.jupiter.api.Assertions.assertSame; + +/** Testing {@link WeakValueCache}. */ +public class TestRaftIdCache extends BaseTest { + static WeakValueCache CACHE = TestRaftId.getClientIdCache(); + + static String dumpCache() { + final List values = CACHE.getValues(); + values.sort(Comparator.comparing(ClientId::getUuid)); + String header = CACHE + ": " + values.size(); + System.out.println(header); + System.out.println(" " + values); + return header; + } + + static void assertCache(IDs expectedIDs) { + final List computed = CACHE.getValues(); + computed.sort(Comparator.comparing(ClientId::getUuid)); + + final List expected = expectedIDs.getIds(); + expected.sort(Comparator.comparing(ClientId::getUuid)); + + assertEquals(expected, computed, TestRaftIdCache::dumpCache); + } + + void assertCacheSizeWithGC(IDs expectedIDs) throws Exception{ + JavaUtils.attempt(() -> { + RaftTestUtil.gc(); + assertCache(expectedIDs); + }, 5, HUNDRED_MILLIS, "assertCacheSizeWithGC", LOG); + } + + class IDs { + private final List ids = new LinkedList<>(); + + List getIds() { + return new ArrayList<>(ids); + } + + int size() { + return ids.size(); + } + + ClientId allocate() { + final ClientId id = ClientId.randomId(); + LOG.info("allocate {}", id); + ids.add(id); + return id; + } + + void release() { + final int r = ThreadLocalRandom.current().nextInt(size()); + final ClientId removed = ids.remove(r); + LOG.info("release {}", removed); + } + } + + @Test + public void testCaching() throws Exception { + final int n = 100; + final IDs ids = new IDs(); + assertEquals(0, ids.size()); + assertCache(ids); + + for(int i = 0; i < n; i++) { + final ClientId id = ids.allocate(); + assertSame(id, ClientId.valueOf(id.getUuid())); + assertCache(ids); + } + + for(int i = 0; i < n/2; i++) { + ids.release(); + if (ThreadLocalRandom.current().nextInt(10) == 0) { + assertCacheSizeWithGC(ids); + } + } + assertCacheSizeWithGC(ids); + + for(int i = 0; i < n/2; i++) { + final ClientId id = ids.allocate(); + assertSame(id, ClientId.valueOf(id.getUuid())); + assertCache(ids); + } + + + for(int i = 0; i < n; i++) { + ids.release(); + if (ThreadLocalRandom.current().nextInt(10) == 0) { + assertCacheSizeWithGC(ids); + } + } + assertCacheSizeWithGC(ids); + + assertEquals(0, ids.size()); + } +} From bf48b0aca8ceeeff4995dd855d127ed57a089f15 Mon Sep 17 00:00:00 2001 From: hfutatzhanghb Date: Mon, 24 Mar 2025 23:36:16 +0800 Subject: [PATCH 165/397] RATIS-2268. A minor documatention fix for method getSuccessors. (#1242) --- .../org/apache/ratis/netty/server/DataStreamManagement.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/ratis-netty/src/main/java/org/apache/ratis/netty/server/DataStreamManagement.java b/ratis-netty/src/main/java/org/apache/ratis/netty/server/DataStreamManagement.java index 7110631858..127ed09835 100644 --- a/ratis-netty/src/main/java/org/apache/ratis/netty/server/DataStreamManagement.java +++ b/ratis-netty/src/main/java/org/apache/ratis/netty/server/DataStreamManagement.java @@ -202,7 +202,7 @@ private Set getSuccessors(RaftPeerId peerId) { } if (isPrimary()) { - // Default start topology + // Default star topology // get the other peers from the current configuration return conf.getCurrentPeers().stream() .filter(p -> !p.getId().equals(division.getId())) From e0f99c6b2ad8d19044585be60cfd1a5182129e7a Mon Sep 17 00:00:00 2001 From: hfutatzhanghb Date: Tue, 25 Mar 2025 23:29:54 +0800 Subject: [PATCH 166/397] RATIS-2269. Remove all import java.util.* statements. (#1243) --- .../java/org/apache/ratis/protocol/RaftGroup.java | 8 +++++++- .../java/org/apache/ratis/util/CollectionUtils.java | 9 ++++++++- .../main/java/org/apache/ratis/util/FileUtils.java | 13 ++++++++++++- .../main/java/org/apache/ratis/util/NetUtils.java | 8 +++++++- .../java/org/apache/ratis/util/ReflectionUtils.java | 10 ++++++++-- .../ratis/examples/ParameterizedBaseTest.java | 7 ++++++- .../examples/filestore/FileStoreAsyncBaseTest.java | 6 +++++- .../apache/ratis/grpc/server/GrpcLogAppender.java | 8 +++++++- .../ratis/server/impl/ConfigurationManager.java | 5 ++++- .../apache/ratis/server/impl/RaftServerProxy.java | 7 ++++++- .../raftlog/segmented/SegmentedRaftLogCache.java | 10 +++++++++- .../raftlog/segmented/SegmentedRaftLogWorker.java | 4 +++- .../server/impl/RaftReconfigurationBaseTest.java | 5 ++++- .../server/impl/StateMachineShutdownTests.java | 10 ++++++++-- .../server/simulation/SimulatedRequestReply.java | 6 +++++- .../apache/ratis/grpc/TestRaftServerWithGrpc.java | 6 +++++- .../java/org/apache/ratis/util/TestLifeCycle.java | 6 +++++- 17 files changed, 109 insertions(+), 19 deletions(-) diff --git a/ratis-common/src/main/java/org/apache/ratis/protocol/RaftGroup.java b/ratis-common/src/main/java/org/apache/ratis/protocol/RaftGroup.java index 0612a16f9d..5cf970afc3 100644 --- a/ratis-common/src/main/java/org/apache/ratis/protocol/RaftGroup.java +++ b/ratis-common/src/main/java/org/apache/ratis/protocol/RaftGroup.java @@ -19,7 +19,13 @@ import org.apache.ratis.util.Preconditions; -import java.util.*; +import java.util.Arrays; +import java.util.Collection; +import java.util.Collections; +import java.util.HashMap; +import java.util.Map; +import java.util.Objects; + /** * Description of a raft group, which has a unique {@link RaftGroupId} and a collection of {@link RaftPeer}. diff --git a/ratis-common/src/main/java/org/apache/ratis/util/CollectionUtils.java b/ratis-common/src/main/java/org/apache/ratis/util/CollectionUtils.java index 11f484608a..2615c2659c 100644 --- a/ratis-common/src/main/java/org/apache/ratis/util/CollectionUtils.java +++ b/ratis-common/src/main/java/org/apache/ratis/util/CollectionUtils.java @@ -17,7 +17,14 @@ */ package org.apache.ratis.util; -import java.util.*; +import java.util.ArrayList; +import java.util.Arrays; +import java.util.Collection; +import java.util.Comparator; +import java.util.Iterator; +import java.util.List; +import java.util.Map; +import java.util.Objects; import java.util.concurrent.ConcurrentMap; import java.util.concurrent.ThreadLocalRandom; import java.util.function.BiFunction; diff --git a/ratis-common/src/main/java/org/apache/ratis/util/FileUtils.java b/ratis-common/src/main/java/org/apache/ratis/util/FileUtils.java index 63bf3a4e5a..4b9d9e3b28 100644 --- a/ratis-common/src/main/java/org/apache/ratis/util/FileUtils.java +++ b/ratis-common/src/main/java/org/apache/ratis/util/FileUtils.java @@ -27,7 +27,18 @@ import java.io.OutputStream; import java.nio.ByteBuffer; import java.nio.channels.FileChannel; -import java.nio.file.*; +import java.nio.file.AtomicMoveNotSupportedException; +import java.nio.file.CopyOption; +import java.nio.file.FileAlreadyExistsException; +import java.nio.file.FileVisitResult; +import java.nio.file.Files; +import java.nio.file.LinkOption; +import java.nio.file.OpenOption; +import java.nio.file.Path; +import java.nio.file.Paths; +import java.nio.file.SimpleFileVisitor; +import java.nio.file.StandardCopyOption; +import java.nio.file.StandardOpenOption; import java.nio.file.attribute.BasicFileAttributes; import java.util.Arrays; import java.util.List; diff --git a/ratis-common/src/main/java/org/apache/ratis/util/NetUtils.java b/ratis-common/src/main/java/org/apache/ratis/util/NetUtils.java index baf6762aaa..c5f22c2893 100644 --- a/ratis-common/src/main/java/org/apache/ratis/util/NetUtils.java +++ b/ratis-common/src/main/java/org/apache/ratis/util/NetUtils.java @@ -23,7 +23,13 @@ import org.slf4j.LoggerFactory; import java.io.IOException; -import java.net.*; +import java.net.Inet6Address; +import java.net.InetAddress; +import java.net.InetSocketAddress; +import java.net.ServerSocket; +import java.net.URI; +import java.net.URISyntaxException; +import java.net.UnknownHostException; import java.util.ArrayList; import java.util.List; import java.util.Map; diff --git a/ratis-common/src/main/java/org/apache/ratis/util/ReflectionUtils.java b/ratis-common/src/main/java/org/apache/ratis/util/ReflectionUtils.java index 6f05c9c530..a8c217b6a1 100644 --- a/ratis-common/src/main/java/org/apache/ratis/util/ReflectionUtils.java +++ b/ratis-common/src/main/java/org/apache/ratis/util/ReflectionUtils.java @@ -20,11 +20,17 @@ package org.apache.ratis.util; - import java.lang.ref.WeakReference; import java.lang.reflect.Constructor; import java.lang.reflect.InvocationTargetException; -import java.util.*; +import java.util.ArrayList; +import java.util.Arrays; +import java.util.Collections; +import java.util.List; +import java.util.Map; +import java.util.Objects; +import java.util.Optional; +import java.util.WeakHashMap; import java.util.concurrent.ConcurrentHashMap; /** diff --git a/ratis-examples/src/test/java/org/apache/ratis/examples/ParameterizedBaseTest.java b/ratis-examples/src/test/java/org/apache/ratis/examples/ParameterizedBaseTest.java index fc3c4c8be8..12445d1d2c 100644 --- a/ratis-examples/src/test/java/org/apache/ratis/examples/ParameterizedBaseTest.java +++ b/ratis-examples/src/test/java/org/apache/ratis/examples/ParameterizedBaseTest.java @@ -35,7 +35,12 @@ import org.slf4j.LoggerFactory; import java.io.IOException; -import java.util.*; +import java.util.ArrayList; +import java.util.Arrays; +import java.util.Collection; +import java.util.Collections; +import java.util.Iterator; +import java.util.List; import java.util.concurrent.TimeUnit; import java.util.concurrent.atomic.AtomicReference; diff --git a/ratis-examples/src/test/java/org/apache/ratis/examples/filestore/FileStoreAsyncBaseTest.java b/ratis-examples/src/test/java/org/apache/ratis/examples/filestore/FileStoreAsyncBaseTest.java index eb51e643b0..0b5071c448 100644 --- a/ratis-examples/src/test/java/org/apache/ratis/examples/filestore/FileStoreAsyncBaseTest.java +++ b/ratis-examples/src/test/java/org/apache/ratis/examples/filestore/FileStoreAsyncBaseTest.java @@ -32,7 +32,11 @@ import java.io.File; import java.util.ArrayList; import java.util.List; -import java.util.concurrent.*; +import java.util.concurrent.Callable; +import java.util.concurrent.CompletableFuture; +import java.util.concurrent.Executor; +import java.util.concurrent.ExecutorService; +import java.util.concurrent.Executors; public abstract class FileStoreAsyncBaseTest extends BaseTest diff --git a/ratis-grpc/src/main/java/org/apache/ratis/grpc/server/GrpcLogAppender.java b/ratis-grpc/src/main/java/org/apache/ratis/grpc/server/GrpcLogAppender.java index 3bff1fb35b..70f6d9a1ae 100644 --- a/ratis-grpc/src/main/java/org/apache/ratis/grpc/server/GrpcLogAppender.java +++ b/ratis-grpc/src/main/java/org/apache/ratis/grpc/server/GrpcLogAppender.java @@ -51,7 +51,13 @@ import java.io.IOException; import java.io.InterruptedIOException; -import java.util.*; +import java.util.Comparator; +import java.util.LinkedList; +import java.util.Map; +import java.util.Objects; +import java.util.Optional; +import java.util.Queue; +import java.util.UUID; import java.util.concurrent.CompletableFuture; import java.util.concurrent.ConcurrentHashMap; import java.util.concurrent.ExecutionException; diff --git a/ratis-server/src/main/java/org/apache/ratis/server/impl/ConfigurationManager.java b/ratis-server/src/main/java/org/apache/ratis/server/impl/ConfigurationManager.java index 6d3f68d5ce..2ba8107837 100644 --- a/ratis-server/src/main/java/org/apache/ratis/server/impl/ConfigurationManager.java +++ b/ratis-server/src/main/java/org/apache/ratis/server/impl/ConfigurationManager.java @@ -25,7 +25,10 @@ import org.apache.ratis.util.Preconditions; import org.apache.ratis.util.StringUtils; -import java.util.*; +import java.util.NavigableMap; +import java.util.SortedMap; +import java.util.TreeMap; + /** * Maintain the mappings between log index and corresponding raft configuration. diff --git a/ratis-server/src/main/java/org/apache/ratis/server/impl/RaftServerProxy.java b/ratis-server/src/main/java/org/apache/ratis/server/impl/RaftServerProxy.java index 6f09998a1a..2d265cf95d 100644 --- a/ratis-server/src/main/java/org/apache/ratis/server/impl/RaftServerProxy.java +++ b/ratis-server/src/main/java/org/apache/ratis/server/impl/RaftServerProxy.java @@ -66,7 +66,12 @@ import java.util.Objects; import java.util.Optional; import java.util.UUID; -import java.util.concurrent.*; +import java.util.concurrent.CompletableFuture; +import java.util.concurrent.CompletionException; +import java.util.concurrent.ConcurrentHashMap; +import java.util.concurrent.ConcurrentMap; +import java.util.concurrent.ExecutorService; +import java.util.concurrent.TimeUnit; import java.util.function.Predicate; import java.util.function.Supplier; import java.util.stream.Collectors; diff --git a/ratis-server/src/main/java/org/apache/ratis/server/raftlog/segmented/SegmentedRaftLogCache.java b/ratis-server/src/main/java/org/apache/ratis/server/raftlog/segmented/SegmentedRaftLogCache.java index e2ddb1508c..64e4496b88 100644 --- a/ratis-server/src/main/java/org/apache/ratis/server/raftlog/segmented/SegmentedRaftLogCache.java +++ b/ratis-server/src/main/java/org/apache/ratis/server/raftlog/segmented/SegmentedRaftLogCache.java @@ -38,7 +38,15 @@ import java.io.File; import java.io.IOException; -import java.util.*; +import java.util.ArrayList; +import java.util.Arrays; +import java.util.Collections; +import java.util.Comparator; +import java.util.Iterator; +import java.util.List; +import java.util.NoSuchElementException; +import java.util.Objects; +import java.util.Optional; import java.util.concurrent.CopyOnWriteArrayList; import java.util.function.Consumer; diff --git a/ratis-server/src/main/java/org/apache/ratis/server/raftlog/segmented/SegmentedRaftLogWorker.java b/ratis-server/src/main/java/org/apache/ratis/server/raftlog/segmented/SegmentedRaftLogWorker.java index c59c37be86..cac35f55b3 100644 --- a/ratis-server/src/main/java/org/apache/ratis/server/raftlog/segmented/SegmentedRaftLogWorker.java +++ b/ratis-server/src/main/java/org/apache/ratis/server/raftlog/segmented/SegmentedRaftLogWorker.java @@ -51,7 +51,9 @@ import java.util.Objects; import java.util.Optional; import java.util.Queue; -import java.util.concurrent.*; +import java.util.concurrent.CompletableFuture; +import java.util.concurrent.ExecutorService; +import java.util.concurrent.TimeUnit; import java.util.function.Consumer; import java.util.function.Supplier; diff --git a/ratis-server/src/test/java/org/apache/ratis/server/impl/RaftReconfigurationBaseTest.java b/ratis-server/src/test/java/org/apache/ratis/server/impl/RaftReconfigurationBaseTest.java index c8c7fd026f..843c271b1c 100644 --- a/ratis-server/src/test/java/org/apache/ratis/server/impl/RaftReconfigurationBaseTest.java +++ b/ratis-server/src/test/java/org/apache/ratis/server/impl/RaftReconfigurationBaseTest.java @@ -51,7 +51,10 @@ import org.slf4j.event.Level; import java.io.IOException; -import java.util.*; +import java.util.ArrayList; +import java.util.Arrays; +import java.util.Collections; +import java.util.List; import java.util.concurrent.CompletableFuture; import java.util.concurrent.CountDownLatch; import java.util.concurrent.TimeUnit; diff --git a/ratis-server/src/test/java/org/apache/ratis/server/impl/StateMachineShutdownTests.java b/ratis-server/src/test/java/org/apache/ratis/server/impl/StateMachineShutdownTests.java index c70464a18c..3333fdb852 100644 --- a/ratis-server/src/test/java/org/apache/ratis/server/impl/StateMachineShutdownTests.java +++ b/ratis-server/src/test/java/org/apache/ratis/server/impl/StateMachineShutdownTests.java @@ -34,8 +34,14 @@ import org.slf4j.Logger; import org.slf4j.LoggerFactory; -import java.util.*; -import java.util.concurrent.*; +import java.util.Comparator; +import java.util.HashSet; +import java.util.Map; +import java.util.Set; +import java.util.concurrent.CompletableFuture; +import java.util.concurrent.ConcurrentHashMap; +import java.util.concurrent.ExecutorService; +import java.util.concurrent.Executors; import java.util.concurrent.atomic.AtomicLong; public abstract class StateMachineShutdownTests diff --git a/ratis-server/src/test/java/org/apache/ratis/server/simulation/SimulatedRequestReply.java b/ratis-server/src/test/java/org/apache/ratis/server/simulation/SimulatedRequestReply.java index 0874e7466c..1b7da951c1 100644 --- a/ratis-server/src/test/java/org/apache/ratis/server/simulation/SimulatedRequestReply.java +++ b/ratis-server/src/test/java/org/apache/ratis/server/simulation/SimulatedRequestReply.java @@ -27,7 +27,11 @@ import java.io.IOException; import java.util.Map; -import java.util.concurrent.*; +import java.util.concurrent.BlockingQueue; +import java.util.concurrent.ConcurrentHashMap; +import java.util.concurrent.LinkedBlockingQueue; +import java.util.concurrent.ThreadLocalRandom; +import java.util.concurrent.TimeUnit; import java.util.concurrent.atomic.AtomicBoolean; import java.util.concurrent.atomic.AtomicInteger; diff --git a/ratis-test/src/test/java/org/apache/ratis/grpc/TestRaftServerWithGrpc.java b/ratis-test/src/test/java/org/apache/ratis/grpc/TestRaftServerWithGrpc.java index e6e39e8ee6..322eb52287 100644 --- a/ratis-test/src/test/java/org/apache/ratis/grpc/TestRaftServerWithGrpc.java +++ b/ratis-test/src/test/java/org/apache/ratis/grpc/TestRaftServerWithGrpc.java @@ -70,7 +70,11 @@ import javax.net.ssl.TrustManager; import java.io.IOException; import java.nio.channels.OverlappingFileLockException; -import java.util.*; +import java.util.ArrayList; +import java.util.Arrays; +import java.util.Collection; +import java.util.Collections; +import java.util.List; import java.util.concurrent.CompletableFuture; import java.util.concurrent.CompletionException; import java.util.concurrent.ExecutionException; diff --git a/ratis-test/src/test/java/org/apache/ratis/util/TestLifeCycle.java b/ratis-test/src/test/java/org/apache/ratis/util/TestLifeCycle.java index 36ef250bf8..201b510571 100644 --- a/ratis-test/src/test/java/org/apache/ratis/util/TestLifeCycle.java +++ b/ratis-test/src/test/java/org/apache/ratis/util/TestLifeCycle.java @@ -21,12 +21,16 @@ import org.junit.jupiter.api.Test; import org.junit.jupiter.api.Timeout; +import java.util.Arrays; +import java.util.EnumMap; +import java.util.List; +import java.util.Map; + import static org.apache.ratis.util.LifeCycle.State.*; import static org.junit.jupiter.api.Assertions.assertEquals; import static org.junit.jupiter.api.Assertions.assertFalse; import static org.junit.jupiter.api.Assertions.assertTrue; -import java.util.*; public class TestLifeCycle { /** From 5bc25abeb130df11993bd2f7c341e0b7c020f351 Mon Sep 17 00:00:00 2001 From: "Doroszlai, Attila" <6454655+adoroszlai@users.noreply.github.com> Date: Mon, 31 Mar 2025 10:07:49 +0200 Subject: [PATCH 167/397] RATIS-2209. Exercise make_rc.sh in CI (#1244) --- .github/workflows/ci.yaml | 7 +++++ dev-support/checks/release.sh | 49 +++++++++++++++++++++++++++++++++++ dev-support/make_rc.sh | 7 +++-- 3 files changed, 61 insertions(+), 2 deletions(-) create mode 100755 dev-support/checks/release.sh diff --git a/.github/workflows/ci.yaml b/.github/workflows/ci.yaml index cf230d3847..072bb3505a 100644 --- a/.github/workflows/ci.yaml +++ b/.github/workflows/ci.yaml @@ -50,6 +50,13 @@ jobs: timeout-minutes: 30 secrets: inherit + release: + uses: ./.github/workflows/check.yaml + with: + script: release + timeout-minutes: 30 + secrets: inherit + repro: needs: - build diff --git a/dev-support/checks/release.sh b/dev-support/checks/release.sh new file mode 100755 index 0000000000..1297b36b0c --- /dev/null +++ b/dev-support/checks/release.sh @@ -0,0 +1,49 @@ +#!/usr/bin/env bash +# 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. + +set -e -u -o pipefail + +# This script tests the local part of the release process. It does not publish anything. + +DIR="$( cd "$( dirname "${BASH_SOURCE[0]}" )" >/dev/null 2>&1 && pwd )" +cd "$DIR/../.." || exit 1 + +: "${RATISVERSION:="0.0.1"}" +: "${RC:="-ci-test"}" +: "${STAGING_REPO_DIR:="/tmp/ratis.staging-repo"}" +: "${SVNDISTDIR:="/tmp/ratis.svn"}" +: "${USERID:="ratis-ci-not-for-release"}" + +MVN_REPO_DIR="${HOME}/.m2/repository" + +mkdir -p "${SVNDISTDIR}" + +if [[ -z "${CODESIGNINGKEY:-}" ]]; then + gpg --batch --passphrase '' --pinentry-mode loopback --quick-generate-key "${USERID}" rsa4096 default 1d + CODESIGNINGKEY=$(gpg --list-keys --with-colons "${USERID}" | grep '^pub:' | cut -f5 -d:) +fi + +git config user.email || git config user.email 'test@example.com' +git config user.name || git config user.name 'Test User' + +export CODESIGNINGKEY MVN_REPO_DIR RATISVERSION RC SVNDISTDIR + +export MAVEN_ARGS="--batch-mode" + +dev-support/make_rc.sh 1-prepare-src +dev-support/make_rc.sh 2-verify-bin +dev-support/make_rc.sh 3-publish-mvn -DaltDeploymentRepository="local::default::file://${STAGING_REPO_DIR}" +dev-support/make_rc.sh 4-assembly diff --git a/dev-support/make_rc.sh b/dev-support/make_rc.sh index 65dbf79a10..c6a93a6d9d 100755 --- a/dev-support/make_rc.sh +++ b/dev-support/make_rc.sh @@ -118,7 +118,7 @@ mvnFun() { mv "apache-ratis-${RATISVERSION}-src" "apache-ratis-${RATISVERSION}" cd "apache-ratis-${RATISVERSION}" - mvnFun clean verify -DskipTests=true -Prelease -Papache-release -Dgpg.keyname="${CODESIGNINGKEY}" + mvnFun clean verify -DskipTests=true -Prelease -Papache-release -Dgpg.keyname="${CODESIGNINGKEY}" "$@" } 3-publish-mvn() { @@ -137,7 +137,10 @@ mvnFun() { for i in *.tar.gz; do gpg --print-md SHA512 "${i}" > "${i}.sha512"; done for i in *.tar.gz; do gpg --print-mds "${i}" > "${i}.mds"; done cd "$SVNDISTDIR" - svn add "${RATISVERSION}" || svn add "${RATISVERSION}/${RC#-}" + # skip svn add in CI + if [[ -z "${CI:-}" ]]; then + svn add "${RATISVERSION}" || svn add "${RATISVERSION}/${RC#-}" + fi } 5-publish-git(){ From 129f997da8e274d76877f1bfb0c828f830ced321 Mon Sep 17 00:00:00 2001 From: GewuNewOne <89496957+RkGrit@users.noreply.github.com> Date: Sat, 5 Apr 2025 21:26:18 +0800 Subject: [PATCH 168/397] RATIS-2271. Leadership Loss Causes ClosedByInterruptException and NullPointerException in LogAppender Thread (#1245) --- .../ratis/server/raftlog/segmented/LogSegment.java | 10 ++++++++-- .../raftlog/segmented/SegmentedRaftLogInputStream.java | 7 ++++++- .../raftlog/segmented/SegmentedRaftLogReader.java | 10 +++++++++- 3 files changed, 23 insertions(+), 4 deletions(-) diff --git a/ratis-server/src/main/java/org/apache/ratis/server/raftlog/segmented/LogSegment.java b/ratis-server/src/main/java/org/apache/ratis/server/raftlog/segmented/LogSegment.java index 1b55b97275..1b0064e445 100644 --- a/ratis-server/src/main/java/org/apache/ratis/server/raftlog/segmented/LogSegment.java +++ b/ratis-server/src/main/java/org/apache/ratis/server/raftlog/segmented/LogSegment.java @@ -244,7 +244,11 @@ public LogEntryProto load(LogRecord key) throws IOException { } }); loadingTimes.incrementAndGet(); - return Objects.requireNonNull(toReturn.get()); + final LogEntryProto proto = toReturn.get(); + if (proto == null) { + throw new RaftLogIOException("Failed to load log entry " + key); + } + return proto; } } @@ -346,8 +350,10 @@ synchronized LogEntryProto loadCache(LogRecord record) throws RaftLogIOException } try { return cacheLoader.load(record); + } catch (RaftLogIOException e) { + throw e; } catch (Exception e) { - throw new RaftLogIOException(e); + throw new RaftLogIOException("Failed to loadCache for log entry " + record, e); } } diff --git a/ratis-server/src/main/java/org/apache/ratis/server/raftlog/segmented/SegmentedRaftLogInputStream.java b/ratis-server/src/main/java/org/apache/ratis/server/raftlog/segmented/SegmentedRaftLogInputStream.java index 050c472dd2..3cc8767fa4 100644 --- a/ratis-server/src/main/java/org/apache/ratis/server/raftlog/segmented/SegmentedRaftLogInputStream.java +++ b/ratis-server/src/main/java/org/apache/ratis/server/raftlog/segmented/SegmentedRaftLogInputStream.java @@ -21,6 +21,7 @@ import java.io.EOFException; import java.io.File; import java.io.IOException; +import java.nio.channels.ClosedByInterruptException; import java.util.Optional; import org.apache.ratis.proto.RaftProtos.LogEntryProto; @@ -104,7 +105,11 @@ public LogEntryProto nextEntry() throws IOException { try { init(); } catch (Exception e) { - LOG.error("caught exception initializing " + this, e); + if (e.getCause() instanceof ClosedByInterruptException) { + LOG.warn("Initialization is interrupted: {}", this, e); + } else { + LOG.error("Failed to initialize {}", this, e); + } throw IOUtils.asIOException(e); } } diff --git a/ratis-server/src/main/java/org/apache/ratis/server/raftlog/segmented/SegmentedRaftLogReader.java b/ratis-server/src/main/java/org/apache/ratis/server/raftlog/segmented/SegmentedRaftLogReader.java index 7d03105b95..57baffb2fd 100644 --- a/ratis-server/src/main/java/org/apache/ratis/server/raftlog/segmented/SegmentedRaftLogReader.java +++ b/ratis-server/src/main/java/org/apache/ratis/server/raftlog/segmented/SegmentedRaftLogReader.java @@ -42,6 +42,7 @@ import java.io.FilterInputStream; import java.io.IOException; import java.io.InputStream; +import java.nio.channels.ClosedByInterruptException; import java.util.Optional; import java.util.zip.Checksum; @@ -169,7 +170,14 @@ public long skip(long amt) throws IOException { */ boolean verifyHeader() throws IOException { final int headerLength = SegmentedRaftLogFormat.getHeaderLength(); - final int readLength = in.read(temp, 0, headerLength); + final int readLength; + try{ + readLength = in.read(temp, 0, headerLength); + } catch (ClosedByInterruptException e) { + Thread.currentThread().interrupt(); + throw new IOException("Interrupted while reading the header of " + file, e); + } + Preconditions.assertTrue(readLength <= headerLength); final int matchLength = SegmentedRaftLogFormat.matchHeader(temp, 0, readLength); Preconditions.assertTrue(matchLength <= readLength); From 4685e69a0916679baade91d6860cc33cac4b463d Mon Sep 17 00:00:00 2001 From: GewuNewOne <89496957+RkGrit@users.noreply.github.com> Date: Sun, 6 Apr 2025 01:03:40 +0800 Subject: [PATCH 169/397] RATIS-2274. Newly added peer may retain outdated configuration after membership change, causing election failure. (#1246) --- .../main/java/org/apache/ratis/server/impl/LeaderStateImpl.java | 1 + 1 file changed, 1 insertion(+) diff --git a/ratis-server/src/main/java/org/apache/ratis/server/impl/LeaderStateImpl.java b/ratis-server/src/main/java/org/apache/ratis/server/impl/LeaderStateImpl.java index 66d2c1a1c3..9e39bd0efd 100644 --- a/ratis-server/src/main/java/org/apache/ratis/server/impl/LeaderStateImpl.java +++ b/ratis-server/src/main/java/org/apache/ratis/server/impl/LeaderStateImpl.java @@ -797,6 +797,7 @@ private BootStrapProgress checkProgress(FollowerInfo follower, long committed) { follower.getLastRpcResponseTime().elapsedTimeMs()); return BootStrapProgress.NOPROGRESS; } else if (follower.getMatchIndex() + stagingCatchupGap > committed + && follower.getMatchIndex() >= server.getRaftConf().getLogEntryIndex() && follower.getLastRpcResponseTime().compareTo(progressTime) > 0 && follower.hasAttemptedToInstallSnapshot()) { return BootStrapProgress.CAUGHTUP; From b87ed58f1b4eacd01ff0e82d30a142c12f48855d Mon Sep 17 00:00:00 2001 From: GewuNewOne <89496957+RkGrit@users.noreply.github.com> Date: Sun, 20 Apr 2025 17:40:10 +0800 Subject: [PATCH 170/397] RATIS-2282. LogAppender Restart Due to Premature Log Entry Access During Concurrent Write Processing (#1249) --- .../org/apache/ratis/server/raftlog/segmented/LogSegment.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/ratis-server/src/main/java/org/apache/ratis/server/raftlog/segmented/LogSegment.java b/ratis-server/src/main/java/org/apache/ratis/server/raftlog/segmented/LogSegment.java index 1b0064e445..444d417ba5 100644 --- a/ratis-server/src/main/java/org/apache/ratis/server/raftlog/segmented/LogSegment.java +++ b/ratis-server/src/main/java/org/apache/ratis/server/raftlog/segmented/LogSegment.java @@ -328,10 +328,10 @@ private void append(boolean keepEntryInCache, LogEntryProto entry, Op op) { } final LogRecord record = new LogRecord(totalFileSize, entry); - records.add(record); if (keepEntryInCache) { putEntryCache(record.getTermIndex(), entry, op); } + records.add(record); totalFileSize += getEntrySize(entry, op); endIndex = entry.getIndex(); } From 40c8938d2f13bcaf4e57126b5f2132321b1bd0a9 Mon Sep 17 00:00:00 2001 From: GewuNewOne <89496957+RkGrit@users.noreply.github.com> Date: Wed, 23 Apr 2025 00:23:24 +0800 Subject: [PATCH 171/397] RATIS-2278. Follower Fails to Append Entries Due to Index Validation Race Condition in NavigableIndices (#1248) --- .../ratis/server/impl/RaftServerImpl.java | 5 ++++- .../ratis/server/impl/ServerImplUtils.java | 19 ++++++++++++------- 2 files changed, 16 insertions(+), 8 deletions(-) diff --git a/ratis-server/src/main/java/org/apache/ratis/server/impl/RaftServerImpl.java b/ratis-server/src/main/java/org/apache/ratis/server/impl/RaftServerImpl.java index 0186841efb..9b9e204d42 100644 --- a/ratis-server/src/main/java/org/apache/ratis/server/impl/RaftServerImpl.java +++ b/ratis-server/src/main/java/org/apache/ratis/server/impl/RaftServerImpl.java @@ -1634,7 +1634,10 @@ leaderId, getMemberId(), currentTerm, followerCommit, inconsistencyReplyNextInde } private CompletableFuture appendLog(List entries) { final List entriesTermIndices = ConsecutiveIndices.convert(entries); - appendLogTermIndices.append(entriesTermIndices); + if (!appendLogTermIndices.append(entriesTermIndices)) { + // index already exists, return the last future + return appendLogFuture.get(); + } return appendLogFuture.updateAndGet(f -> f.thenCompose( ignored -> JavaUtils.allOf(state.getLog().append(entries)))) diff --git a/ratis-server/src/main/java/org/apache/ratis/server/impl/ServerImplUtils.java b/ratis-server/src/main/java/org/apache/ratis/server/impl/ServerImplUtils.java index c5010a5346..ce4702d958 100644 --- a/ratis-server/src/main/java/org/apache/ratis/server/impl/ServerImplUtils.java +++ b/ratis-server/src/main/java/org/apache/ratis/server/impl/ServerImplUtils.java @@ -136,15 +136,20 @@ synchronized Long getTerm(long index) { return floorEntry.getValue().getTerm(index); } - synchronized void append(List entriesTermIndices) { - for(ConsecutiveIndices indices : entriesTermIndices) { - // validate startIndex - final Map.Entry lastEntry = map.lastEntry(); - if (lastEntry != null) { - Preconditions.assertSame(lastEntry.getValue().getNextIndex(), indices.startIndex, "startIndex"); + synchronized boolean append(List entriesTermIndices) { + for(int i = 0; i < entriesTermIndices.size(); i++) { + final ConsecutiveIndices indices = entriesTermIndices.get(i); + final ConsecutiveIndices previous = map.put(indices.startIndex, indices); + if (previous != null) { + // index already exists, revert this append + map.put(previous.startIndex, previous); + for(int j = 0; j < i; j++) { + map.remove(entriesTermIndices.get(j).startIndex); + } + return false; } - map.put(indices.startIndex, indices); } + return true; } synchronized void removeExisting(List entriesTermIndices) { From 0b3f8f5535c7f126772e3a7e816606e503686efd Mon Sep 17 00:00:00 2001 From: GewuNewOne <89496957+RkGrit@users.noreply.github.com> Date: Thu, 24 Apr 2025 01:49:51 +0800 Subject: [PATCH 172/397] RATIS-2283. GrpcLogAppender Thread Restart Leaves catchup=false, Blocking Reconfiguration Progress (#1250) --- .../org/apache/ratis/server/leader/LogAppender.java | 2 +- .../org/apache/ratis/server/impl/LeaderStateImpl.java | 11 +++++++++-- 2 files changed, 10 insertions(+), 3 deletions(-) diff --git a/ratis-server-api/src/main/java/org/apache/ratis/server/leader/LogAppender.java b/ratis-server-api/src/main/java/org/apache/ratis/server/leader/LogAppender.java index 36331e3abb..cff5425d32 100644 --- a/ratis-server-api/src/main/java/org/apache/ratis/server/leader/LogAppender.java +++ b/ratis-server-api/src/main/java/org/apache/ratis/server/leader/LogAppender.java @@ -143,7 +143,7 @@ default SnapshotInfo shouldInstallSnapshot() { // we should install snapshot if the follower needs to catch up and: // 1. there is no local log entry but there is snapshot // 2. or the follower's next index is smaller than the log start index - // 3. or the follower is bootstrapping and has not installed any snapshot yet + // 3. or the follower is bootstrapping (i.e. not yet caught up) and has not installed any snapshot yet final FollowerInfo follower = getFollower(); final boolean isFollowerBootstrapping = getLeaderState().isFollowerBootstrapping(follower); final SnapshotInfo snapshot = getServer().getStateMachine().getLatestSnapshot(); diff --git a/ratis-server/src/main/java/org/apache/ratis/server/impl/LeaderStateImpl.java b/ratis-server/src/main/java/org/apache/ratis/server/impl/LeaderStateImpl.java index 9e39bd0efd..a84732d1c7 100644 --- a/ratis-server/src/main/java/org/apache/ratis/server/impl/LeaderStateImpl.java +++ b/ratis-server/src/main/java/org/apache/ratis/server/impl/LeaderStateImpl.java @@ -818,7 +818,9 @@ public void onFollowerSuccessAppendEntries(FollowerInfo follower) { @Override public boolean isFollowerBootstrapping(FollowerInfo follower) { - return isBootStrappingPeer(follower.getId()); + // It is better to check caught up than staging state + // since a follower may have already caught up but still in the staging state. + return !isCaughtUp(follower); } private void checkStaging() { @@ -850,7 +852,12 @@ private void checkStaging() { } boolean isBootStrappingPeer(RaftPeerId peerId) { - return Optional.ofNullable(stagingState).map(s -> s.contains(peerId)).orElse(false); + final Optional info = getLogAppender(peerId); + if (info.isPresent()) { + return !isCaughtUp(info.get().getFollower()); + } + final ConfigurationStagingState staging = stagingState; + return staging != null && staging.contains(peerId); } void submitUpdateCommitEvent() { From 05a7b24579a3bb9f6d3a1f40e45642e6bc050acd Mon Sep 17 00:00:00 2001 From: "Doroszlai, Attila" <6454655+adoroszlai@users.noreply.github.com> Date: Fri, 2 May 2025 20:53:43 +0200 Subject: [PATCH 173/397] RATIS-2285. Bump ratis-thirdparty to 1.0.9 (#1252) --- pom.xml | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/pom.xml b/pom.xml index df44796055..1a85348a2b 100644 --- a/pom.xml +++ b/pom.xml @@ -208,11 +208,11 @@ 3.3.9 - 1.0.8 + 1.0.9 3.25.5 - 1.69.0 + 1.71.0 true From 3c58b82148bd4e84d74bbdd952b53b1e3853c3bf Mon Sep 17 00:00:00 2001 From: jianghuazhu <740087514@qq.com> Date: Sat, 3 May 2025 06:45:59 +0800 Subject: [PATCH 174/397] RATIS-2286. Generate the necessary unified names for the submodules under RaftServerImpl (#1251) --- .../org/apache/ratis/server/impl/FollowerState.java | 4 ++-- .../org/apache/ratis/server/impl/LeaderElection.java | 3 +-- .../org/apache/ratis/server/impl/LeaderStateImpl.java | 7 ++++--- .../apache/ratis/server/impl/StateMachineUpdater.java | 3 ++- .../apache/ratis/server/util/ServerStringUtils.java | 11 +++++++++++ 5 files changed, 20 insertions(+), 8 deletions(-) diff --git a/ratis-server/src/main/java/org/apache/ratis/server/impl/FollowerState.java b/ratis-server/src/main/java/org/apache/ratis/server/impl/FollowerState.java index 1be160f182..b01abcddc0 100644 --- a/ratis-server/src/main/java/org/apache/ratis/server/impl/FollowerState.java +++ b/ratis-server/src/main/java/org/apache/ratis/server/impl/FollowerState.java @@ -19,8 +19,8 @@ import org.apache.ratis.server.DivisionInfo; import org.apache.ratis.server.leader.LeaderState; +import org.apache.ratis.server.util.ServerStringUtils; import org.apache.ratis.util.Daemon; -import org.apache.ratis.util.JavaUtils; import org.apache.ratis.util.TimeDuration; import org.apache.ratis.util.Timestamp; import org.slf4j.Logger; @@ -68,7 +68,7 @@ int update(AtomicInteger outstanding) { FollowerState(RaftServerImpl server, Object reason) { super(newBuilder() - .setName(server.getMemberId() + "-" + JavaUtils.getClassSimpleName(FollowerState.class)) + .setName(ServerStringUtils.generateUnifiedName(server.getMemberId(), FollowerState.class)) .setThreadGroup(server.getThreadGroup())); this.server = server; this.reason = reason; diff --git a/ratis-server/src/main/java/org/apache/ratis/server/impl/LeaderElection.java b/ratis-server/src/main/java/org/apache/ratis/server/impl/LeaderElection.java index 39b401dda1..439405871d 100644 --- a/ratis-server/src/main/java/org/apache/ratis/server/impl/LeaderElection.java +++ b/ratis-server/src/main/java/org/apache/ratis/server/impl/LeaderElection.java @@ -29,7 +29,6 @@ import org.apache.ratis.server.util.ServerStringUtils; import org.apache.ratis.thirdparty.com.google.common.annotations.VisibleForTesting; import org.apache.ratis.util.Daemon; -import org.apache.ratis.util.JavaUtils; import org.apache.ratis.util.LifeCycle; import org.apache.ratis.util.LogUtils; import org.apache.ratis.util.Preconditions; @@ -191,7 +190,7 @@ public String toString() { private final ConfAndTerm round0; LeaderElection(RaftServerImpl server, boolean force) { - this.name = server.getMemberId() + "-" + JavaUtils.getClassSimpleName(getClass()) + COUNT.incrementAndGet(); + this.name = ServerStringUtils.generateUnifiedName(server.getMemberId(), getClass()) + COUNT.incrementAndGet(); this.lifeCycle = new LifeCycle(this); this.daemon = Daemon.newBuilder().setName(name).setRunnable(this) .setThreadGroup(server.getThreadGroup()).build(); diff --git a/ratis-server/src/main/java/org/apache/ratis/server/impl/LeaderStateImpl.java b/ratis-server/src/main/java/org/apache/ratis/server/impl/LeaderStateImpl.java index a84732d1c7..cd1d9ebea5 100644 --- a/ratis-server/src/main/java/org/apache/ratis/server/impl/LeaderStateImpl.java +++ b/ratis-server/src/main/java/org/apache/ratis/server/impl/LeaderStateImpl.java @@ -49,6 +49,7 @@ import org.apache.ratis.server.raftlog.LogEntryHeader; import org.apache.ratis.server.raftlog.LogProtoUtils; import org.apache.ratis.server.raftlog.RaftLog; +import org.apache.ratis.server.util.ServerStringUtils; import org.apache.ratis.statemachine.TransactionContext; import org.apache.ratis.util.CodeInjectionForTesting; import org.apache.ratis.util.CollectionUtils; @@ -146,7 +147,7 @@ public String toString() { } private class EventQueue { - private final String name = server.getMemberId() + "-" + JavaUtils.getClassSimpleName(getClass()); + private final String name = ServerStringUtils.generateUnifiedName(server.getMemberId(), getClass()); private final BlockingQueue queue = new ArrayBlockingQueue<>(4096); void submit(StateUpdateEvent event) { @@ -360,7 +361,7 @@ boolean isApplied() { private final LeaderLease lease; LeaderStateImpl(RaftServerImpl server) { - this.name = server.getMemberId() + "-" + JavaUtils.getClassSimpleName(getClass()); + this.name = ServerStringUtils.generateUnifiedName(server.getMemberId(), getClass()); this.server = server; final RaftProperties properties = server.getRaftServer().getProperties(); @@ -1234,7 +1235,7 @@ static Map newMap(Collection peers, String str) } private class ConfigurationStagingState { - private final String name = server.getMemberId() + "-" + JavaUtils.getClassSimpleName(getClass()); + private final String name = ServerStringUtils.generateUnifiedName(server.getMemberId(), getClass()); private final Map newPeers; private final Map newListeners; private final PeerConfiguration newConf; diff --git a/ratis-server/src/main/java/org/apache/ratis/server/impl/StateMachineUpdater.java b/ratis-server/src/main/java/org/apache/ratis/server/impl/StateMachineUpdater.java index 3dfe5e0aa9..bd7f26a8a2 100644 --- a/ratis-server/src/main/java/org/apache/ratis/server/impl/StateMachineUpdater.java +++ b/ratis-server/src/main/java/org/apache/ratis/server/impl/StateMachineUpdater.java @@ -29,6 +29,7 @@ import org.apache.ratis.proto.RaftProtos.LogEntryProto; import org.apache.ratis.server.raftlog.RaftLogIOException; import org.apache.ratis.server.raftlog.RaftLogIndex; +import org.apache.ratis.server.util.ServerStringUtils; import org.apache.ratis.statemachine.SnapshotInfo; import org.apache.ratis.statemachine.StateMachine; import org.apache.ratis.statemachine.SnapshotRetentionPolicy; @@ -95,7 +96,7 @@ enum State { StateMachineUpdater(StateMachine stateMachine, RaftServerImpl server, ServerState serverState, long lastAppliedIndex, RaftProperties properties, Consumer appliedIndexConsumer) { - this.name = serverState.getMemberId() + "-" + JavaUtils.getClassSimpleName(getClass()); + this.name = ServerStringUtils.generateUnifiedName(serverState.getMemberId(), getClass()); this.appliedIndexConsumer = appliedIndexConsumer; this.infoIndexChange = s -> LOG.info("{}: {}", name, s); this.debugIndexChange = s -> LOG.debug("{}: {}", name, s); diff --git a/ratis-server/src/main/java/org/apache/ratis/server/util/ServerStringUtils.java b/ratis-server/src/main/java/org/apache/ratis/server/util/ServerStringUtils.java index 6e0fce4d66..6601eddce0 100644 --- a/ratis-server/src/main/java/org/apache/ratis/server/util/ServerStringUtils.java +++ b/ratis-server/src/main/java/org/apache/ratis/server/util/ServerStringUtils.java @@ -24,8 +24,10 @@ import org.apache.ratis.proto.RaftProtos.LogEntryProto; import org.apache.ratis.proto.RaftProtos.RequestVoteReplyProto; import org.apache.ratis.proto.RaftProtos.StateMachineLogEntryProto; +import org.apache.ratis.protocol.RaftGroupMemberId; import org.apache.ratis.server.protocol.TermIndex; import org.apache.ratis.server.raftlog.LogProtoUtils; +import org.apache.ratis.util.JavaUtils; import org.apache.ratis.util.ProtoUtils; import java.util.List; @@ -118,4 +120,13 @@ public static String toRequestVoteReplyString(RequestVoteReplyProto proto) { } return ProtoUtils.toString(proto.getServerReply()) + "-t" + proto.getTerm(); } + + /** + * Used to generate the necessary unified name in the submodules under + * {@link org.apache.ratis.server.impl.RaftServerImpl}, which consists + * of {@link org.apache.ratis.server.impl.ServerState#memberId} and the specific class. + */ + public static String generateUnifiedName(RaftGroupMemberId memberId, Class clazz) { + return memberId + "-" + JavaUtils.getClassSimpleName(clazz); + } } From 8addb3cee4ac9f20aa6dc99576240163cdc75611 Mon Sep 17 00:00:00 2001 From: jianghuazhu <740087514@qq.com> Date: Wed, 7 May 2025 23:17:12 +0800 Subject: [PATCH 175/397] RATIS-2287. Improve RaftServerImpl#toString() to make it more readable (#1253) --- .../main/java/org/apache/ratis/server/impl/RaftServerImpl.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/ratis-server/src/main/java/org/apache/ratis/server/impl/RaftServerImpl.java b/ratis-server/src/main/java/org/apache/ratis/server/impl/RaftServerImpl.java index 9b9e204d42..e7898b9aa5 100644 --- a/ratis-server/src/main/java/org/apache/ratis/server/impl/RaftServerImpl.java +++ b/ratis-server/src/main/java/org/apache/ratis/server/impl/RaftServerImpl.java @@ -715,7 +715,7 @@ synchronized void changeToCandidate(boolean forceStartLeaderElection) { @Override public String toString() { - return role + " " + state + " " + lifeCycle.getCurrentState(); + return role + " (" + lifeCycle.getCurrentState() + "): " + state; } RaftClientReply.Builder newReplyBuilder(RaftClientRequest request) { From 8a8ec9e0f22f5d970cc0e2a3ae32c397626bffab Mon Sep 17 00:00:00 2001 From: "Doroszlai, Attila" <6454655+adoroszlai@users.noreply.github.com> Date: Fri, 9 May 2025 06:50:54 +0200 Subject: [PATCH 176/397] RATIS-2289. Coverage is 0% due to no execution data files provided (#1255) --- .github/workflows/check.yaml | 3 +-- .github/workflows/ci.yaml | 3 ++- 2 files changed, 3 insertions(+), 3 deletions(-) diff --git a/.github/workflows/check.yaml b/.github/workflows/check.yaml index be5718f7ef..1655d60148 100644 --- a/.github/workflows/check.yaml +++ b/.github/workflows/check.yaml @@ -79,7 +79,7 @@ on: env: MAVEN_ARGS: --batch-mode --show-version MAVEN_OPTS: -Dhttp.keepAlive=false -Dmaven.wagon.http.pool=false -Dmaven.wagon.http.retryHandler.class=standard -Dmaven.wagon.http.retryHandler.count=3 - WITH_COVERAGE: true + WITH_COVERAGE: ${{ github.event_name == 'push' }} jobs: check: @@ -157,7 +157,6 @@ jobs: dev-support/checks/${{ inputs.script }}.sh ${{ inputs.script-args }} env: DEVELOCITY_ACCESS_KEY: ${{ secrets.DEVELOCITY_ACCESS_KEY }} - WITH_COVERAGE: ${{ inputs.with-coverage }} - name: Summary of failures if: ${{ failure() }} diff --git a/.github/workflows/ci.yaml b/.github/workflows/ci.yaml index 072bb3505a..20704f5ea9 100644 --- a/.github/workflows/ci.yaml +++ b/.github/workflows/ci.yaml @@ -106,7 +106,7 @@ jobs: - unit runs-on: ubuntu-24.04 timeout-minutes: 30 - if: (github.repository == 'apache/ratis' || github.repository == 'apache/incubator-ratis') && github.event_name != 'pull_request' + if: github.event_name != 'pull_request' steps: - name: Checkout project uses: actions/checkout@v4 @@ -137,6 +137,7 @@ jobs: - name: Calculate combined coverage run: ./dev-support/checks/coverage.sh - name: Upload coverage to Sonar + if: github.repository == 'apache/ratis' run: ./dev-support/checks/sonar.sh env: SONAR_TOKEN: ${{ secrets.SONARCLOUD_TOKEN }} From 13b39d5b4bed0af13158c55bfcce4b5f27f2748d Mon Sep 17 00:00:00 2001 From: jianghuazhu <740087514@qq.com> Date: Mon, 12 May 2025 23:33:29 +0800 Subject: [PATCH 177/397] RATIS-2180. Use Objects.requireNonNull instead of Preconditions.assertNotNull (#1256) --- .../org/apache/ratis/protocol/RaftClientMessage.java | 9 +++++---- .../src/main/java/org/apache/ratis/protocol/RaftId.java | 2 +- .../org/apache/ratis/retry/ExponentialBackoffRetry.java | 4 ++-- .../src/main/java/org/apache/ratis/util/JavaUtils.java | 6 +++--- .../org/apache/ratis/grpc/server/GrpcLogAppender.java | 2 +- .../org/apache/ratis/server/impl/RaftServerImpl.java | 3 ++- .../org/apache/ratis/server/impl/ServerImplUtils.java | 5 +++-- .../server/raftlog/segmented/SegmentedRaftLogCache.java | 2 +- .../src/test/java/org/apache/ratis/RaftTestUtil.java | 2 +- .../java/org/apache/ratis/server/ServerBuilderTest.java | 4 ++-- .../raftlog/segmented/TestSegmentedRaftLogCache.java | 2 +- 11 files changed, 22 insertions(+), 19 deletions(-) diff --git a/ratis-common/src/main/java/org/apache/ratis/protocol/RaftClientMessage.java b/ratis-common/src/main/java/org/apache/ratis/protocol/RaftClientMessage.java index 8d3104a73d..92ae77ce21 100644 --- a/ratis-common/src/main/java/org/apache/ratis/protocol/RaftClientMessage.java +++ b/ratis-common/src/main/java/org/apache/ratis/protocol/RaftClientMessage.java @@ -18,7 +18,8 @@ package org.apache.ratis.protocol; import org.apache.ratis.util.JavaUtils; -import org.apache.ratis.util.Preconditions; + +import java.util.Objects; public abstract class RaftClientMessage implements RaftRpcMessage { private final ClientId clientId; @@ -27,9 +28,9 @@ public abstract class RaftClientMessage implements RaftRpcMessage { private final long callId; RaftClientMessage(ClientId clientId, RaftPeerId serverId, RaftGroupId groupId, long callId) { - this.clientId = Preconditions.assertNotNull(clientId, "clientId"); - this.serverId = Preconditions.assertNotNull(serverId, "serverId"); - this.groupId = Preconditions.assertNotNull(groupId, "groupId"); + this.clientId = Objects.requireNonNull(clientId, "clientId == null"); + this.serverId = Objects.requireNonNull(serverId, "serverId == null"); + this.groupId = Objects.requireNonNull(groupId, "groupId == null"); this.callId = callId; } diff --git a/ratis-common/src/main/java/org/apache/ratis/protocol/RaftId.java b/ratis-common/src/main/java/org/apache/ratis/protocol/RaftId.java index d8a3f73ab3..d089c7d3cb 100644 --- a/ratis-common/src/main/java/org/apache/ratis/protocol/RaftId.java +++ b/ratis-common/src/main/java/org/apache/ratis/protocol/RaftId.java @@ -85,7 +85,7 @@ ID randomId() { private final Supplier uuidString; RaftId(UUID uuid) { - this.uuid = Preconditions.assertNotNull(uuid, "uuid"); + this.uuid = Objects.requireNonNull(uuid, "uuid == null"); this.uuidBytes = JavaUtils.memoize(() -> toByteString(uuid)); this.uuidString = JavaUtils.memoize(() -> createUuidString(uuid)); Preconditions.assertTrue(ZERO_UUID == uuid || !uuid.equals(ZERO_UUID), diff --git a/ratis-common/src/main/java/org/apache/ratis/retry/ExponentialBackoffRetry.java b/ratis-common/src/main/java/org/apache/ratis/retry/ExponentialBackoffRetry.java index bb2f50e43a..d506c85c80 100644 --- a/ratis-common/src/main/java/org/apache/ratis/retry/ExponentialBackoffRetry.java +++ b/ratis-common/src/main/java/org/apache/ratis/retry/ExponentialBackoffRetry.java @@ -17,9 +17,9 @@ */ package org.apache.ratis.retry; -import org.apache.ratis.util.Preconditions; import org.apache.ratis.util.TimeDuration; +import java.util.Objects; import java.util.concurrent.ThreadLocalRandom; /** @@ -56,7 +56,7 @@ public Builder setMaxSleepTime(TimeDuration maxSleepTime) { } public ExponentialBackoffRetry build() { - Preconditions.assertNotNull(baseSleepTime, "baseSleepTime"); + Objects.requireNonNull(baseSleepTime, "baseSleepTime == null"); return new ExponentialBackoffRetry(baseSleepTime, maxSleepTime, maxAttempts); } diff --git a/ratis-common/src/main/java/org/apache/ratis/util/JavaUtils.java b/ratis-common/src/main/java/org/apache/ratis/util/JavaUtils.java index 7d1d75309a..958e88cee5 100644 --- a/ratis-common/src/main/java/org/apache/ratis/util/JavaUtils.java +++ b/ratis-common/src/main/java/org/apache/ratis/util/JavaUtils.java @@ -148,7 +148,7 @@ static T doPrivileged(Supplier action, Supplier name) { * otherwise, return system property value. */ static String getSystemProperty(final String key) { - Preconditions.assertNotNull(key, "key"); + Objects.requireNonNull(key, "key == null"); Preconditions.assertTrue(!key.isEmpty(), "key is empty."); return doPrivileged(() -> System.getProperty(key), () -> "get system property " + key); } @@ -166,9 +166,9 @@ static String getEnv(String variable) { * When there is a {@link SecurityException}, this becomes a NOOP. */ static void setSystemProperty(String key, String value) { - Preconditions.assertNotNull(key, "key"); + Objects.requireNonNull(key, "key == null"); Preconditions.assertTrue(!key.isEmpty(), "key is empty."); - Preconditions.assertNotNull(value, "value"); + Objects.requireNonNull(value, "value == null"); doPrivileged(() -> System.setProperty(key, value), () -> "set system property " + key + " to " + value); } diff --git a/ratis-grpc/src/main/java/org/apache/ratis/grpc/server/GrpcLogAppender.java b/ratis-grpc/src/main/java/org/apache/ratis/grpc/server/GrpcLogAppender.java index 70f6d9a1ae..1622c5df9f 100644 --- a/ratis-grpc/src/main/java/org/apache/ratis/grpc/server/GrpcLogAppender.java +++ b/ratis-grpc/src/main/java/org/apache/ratis/grpc/server/GrpcLogAppender.java @@ -177,7 +177,7 @@ synchronized int process(Event event) { public GrpcLogAppender(RaftServer.Division server, LeaderState leaderState, FollowerInfo f) { super(server, leaderState, f); - Preconditions.assertNotNull(getServerRpc(), "getServerRpc()"); + Objects.requireNonNull(getServerRpc(), "getServerRpc() == null"); final RaftProperties properties = server.getRaftServer().getProperties(); this.maxPendingRequestsNum = GrpcConfigKeys.Server.leaderOutstandingAppendsMax(properties); diff --git a/ratis-server/src/main/java/org/apache/ratis/server/impl/RaftServerImpl.java b/ratis-server/src/main/java/org/apache/ratis/server/impl/RaftServerImpl.java index e7898b9aa5..b068265ce8 100644 --- a/ratis-server/src/main/java/org/apache/ratis/server/impl/RaftServerImpl.java +++ b/ratis-server/src/main/java/org/apache/ratis/server/impl/RaftServerImpl.java @@ -1241,7 +1241,7 @@ CompletableFuture takeSnapshotAsync(SnapshotManagementRequest r LOG.info("{}: takeSnapshotAsync {}", getMemberId(), request); assertLifeCycleState(LifeCycle.States.RUNNING); assertGroup(getMemberId(), request); - Preconditions.assertNotNull(request.getCreate(), "create"); + Objects.requireNonNull(request.getCreate(), "create == null"); final long creationGap = request.getCreate().getCreationGap(); long minGapValue = creationGap > 0? creationGap : RaftServerConfigKeys.Snapshot.creationGap(proxy.getProperties()); @@ -1847,6 +1847,7 @@ CompletableFuture applyLogToStateMachine(LogEntryProto next) throws Raf break; case STATEMACHINELOGENTRY: TransactionContext trx = getTransactionContext(next, true); + Objects.requireNonNull(trx, "trx == null"); final ClientInvocationId invocationId = ClientInvocationId.valueOf(next.getStateMachineLogEntry()); writeIndexCache.add(invocationId.getClientId(), ((TransactionContextImpl) trx).getLogIndexFuture()); diff --git a/ratis-server/src/main/java/org/apache/ratis/server/impl/ServerImplUtils.java b/ratis-server/src/main/java/org/apache/ratis/server/impl/ServerImplUtils.java index ce4702d958..864b402a23 100644 --- a/ratis-server/src/main/java/org/apache/ratis/server/impl/ServerImplUtils.java +++ b/ratis-server/src/main/java/org/apache/ratis/server/impl/ServerImplUtils.java @@ -46,6 +46,7 @@ import java.util.Map; import java.util.NavigableMap; import java.util.TreeMap; +import java.util.Objects; import java.util.concurrent.TimeUnit; /** Server utilities for internal use. */ @@ -170,8 +171,8 @@ public static RaftServerProxy newRaftServer( ThreadGroup threadGroup, RaftProperties properties, Parameters parameters) throws IOException { RaftServer.LOG.debug("newRaftServer: {}, {}", id, group); if (group != null && !group.getPeers().isEmpty()) { - Preconditions.assertNotNull(id, "RaftPeerId %s is not in RaftGroup %s", id, group); - Preconditions.assertNotNull(group.getPeer(id), "RaftPeerId %s is not in RaftGroup %s", id, group); + Objects.requireNonNull(id, () -> "RaftPeerId " + id + " is not in RaftGroup " + group); + Objects.requireNonNull(group.getPeer(id), () -> "RaftPeerId " + id + " is not in RaftGroup " + group); } final RaftServerProxy proxy = newRaftServer(id, stateMachineRegistry, threadGroup, properties, parameters); proxy.initGroups(group, option); diff --git a/ratis-server/src/main/java/org/apache/ratis/server/raftlog/segmented/SegmentedRaftLogCache.java b/ratis-server/src/main/java/org/apache/ratis/server/raftlog/segmented/SegmentedRaftLogCache.java index 64e4496b88..a1f0cdd8ab 100644 --- a/ratis-server/src/main/java/org/apache/ratis/server/raftlog/segmented/SegmentedRaftLogCache.java +++ b/ratis-server/src/main/java/org/apache/ratis/server/raftlog/segmented/SegmentedRaftLogCache.java @@ -620,7 +620,7 @@ TermIndex getLastTermIndex() { void appendEntry(LogEntryProto entry, LogSegment.Op op) { // SegmentedRaftLog does the segment creation/rolling work. Here we just // simply append the entry into the open segment. - Preconditions.assertNotNull(openSegment, "openSegment"); + Objects.requireNonNull(openSegment, "openSegment == null"); openSegment.appendToOpenSegment(entry, op); } diff --git a/ratis-server/src/test/java/org/apache/ratis/RaftTestUtil.java b/ratis-server/src/test/java/org/apache/ratis/RaftTestUtil.java index 206b6dd949..58dedaf8c6 100644 --- a/ratis-server/src/test/java/org/apache/ratis/RaftTestUtil.java +++ b/ratis-server/src/test/java/org/apache/ratis/RaftTestUtil.java @@ -151,7 +151,7 @@ static RaftPeerId waitAndKillLeader(MiniRaftCluster cluster) throws InterruptedE static void waitFor(Supplier check, int checkEveryMillis, int waitForMillis) throws TimeoutException, InterruptedException { - Preconditions.assertNotNull(check, "check"); + Objects.requireNonNull(check, "check == null"); Preconditions.assertTrue(waitForMillis >= checkEveryMillis, () -> "waitFor: " + waitForMillis + " < checkEvery: " + checkEveryMillis); diff --git a/ratis-test/src/test/java/org/apache/ratis/server/ServerBuilderTest.java b/ratis-test/src/test/java/org/apache/ratis/server/ServerBuilderTest.java index 9360b9b847..58d553367c 100644 --- a/ratis-test/src/test/java/org/apache/ratis/server/ServerBuilderTest.java +++ b/ratis-test/src/test/java/org/apache/ratis/server/ServerBuilderTest.java @@ -60,7 +60,7 @@ public void testPeerIdNotInRaftGroup() { .build(); Assertions.fail("did not get expected exception"); } catch (IOException e) { - Preconditions.assertInstanceOf(e.getCause(), IllegalStateException.class); + Preconditions.assertInstanceOf(e.getCause(), NullPointerException.class); } } @@ -76,7 +76,7 @@ public void testNullPeerIdWithRaftGroup() { .build(); Assertions.fail("did not get expected exception"); } catch (IOException e) { - Preconditions.assertInstanceOf(e.getCause(), IllegalStateException.class); + Preconditions.assertInstanceOf(e.getCause(), NullPointerException.class); } } diff --git a/ratis-test/src/test/java/org/apache/ratis/server/raftlog/segmented/TestSegmentedRaftLogCache.java b/ratis-test/src/test/java/org/apache/ratis/server/raftlog/segmented/TestSegmentedRaftLogCache.java index 8015f18274..7c2dbac912 100644 --- a/ratis-test/src/test/java/org/apache/ratis/server/raftlog/segmented/TestSegmentedRaftLogCache.java +++ b/ratis-test/src/test/java/org/apache/ratis/server/raftlog/segmented/TestSegmentedRaftLogCache.java @@ -157,7 +157,7 @@ public void testAppendEntry() throws Exception { LogEntryProto entry = LogProtoUtils.toLogEntryProto(m.getLogEntryContent(), 0, 0); cache.appendEntry(entry, LogSegment.Op.WRITE_CACHE_WITHOUT_STATE_MACHINE_CACHE); Assertions.fail("the open segment is null"); - } catch (IllegalStateException ignored) { + } catch (IllegalStateException | NullPointerException ignored) { } LogSegment openSegment = prepareLogSegment(100, 100, true); From 9609da11bd894384d2fb018f6337f212a87fe288 Mon Sep 17 00:00:00 2001 From: slfan1989 <55643692+slfan1989@users.noreply.github.com> Date: Mon, 12 May 2025 23:50:32 +0800 Subject: [PATCH 178/397] RATIS-2291. Fix failing TestInstallSnapshotNotificationWithGrpc#testAddNewFollowersNoSnapshot. (#1257) --- .../ratis/InstallSnapshotNotificationTests.java | 4 ++-- .../apache/ratis/server/impl/MiniRaftCluster.java | 14 +++++++------- 2 files changed, 9 insertions(+), 9 deletions(-) diff --git a/ratis-server/src/test/java/org/apache/ratis/InstallSnapshotNotificationTests.java b/ratis-server/src/test/java/org/apache/ratis/InstallSnapshotNotificationTests.java index 6aaa8acc39..ae5d792246 100644 --- a/ratis-server/src/test/java/org/apache/ratis/InstallSnapshotNotificationTests.java +++ b/ratis-server/src/test/java/org/apache/ratis/InstallSnapshotNotificationTests.java @@ -220,7 +220,7 @@ private void testAddNewFollowers(CLUSTER cluster, int numRequests) throws Except // delete the log segments from the leader LOG.info("Delete logs {}", logs); for (LogSegmentPath path : logs) { - FileUtils.deleteFully(path.getPath()); // the log may be already puged + FileUtils.deleteFully(path.getPath()); // the log may be already purged } // restart the peer @@ -252,7 +252,7 @@ private void testAddNewFollowers(CLUSTER cluster, int numRequests) throws Except // Check the installed snapshot index on each Follower matches with the // leader snapshot. for (RaftServer.Division follower : cluster.getFollowers()) { - final long expected = shouldInstallSnapshot ? leaderSnapshotInfo.getIndex() : RaftLog.INVALID_LOG_INDEX; + final long expected = leaderSnapshotInfo.getIndex(); Assert.assertEquals(expected, RaftServerTestUtil.getLatestInstalledSnapshotIndex(follower)); RaftSnapshotBaseTest.assertLogContent(follower, false); } diff --git a/ratis-server/src/test/java/org/apache/ratis/server/impl/MiniRaftCluster.java b/ratis-server/src/test/java/org/apache/ratis/server/impl/MiniRaftCluster.java index 1dc9e8f526..f4785ce906 100644 --- a/ratis-server/src/test/java/org/apache/ratis/server/impl/MiniRaftCluster.java +++ b/ratis-server/src/test/java/org/apache/ratis/server/impl/MiniRaftCluster.java @@ -133,7 +133,7 @@ default void runWithNewCluster(int numServers, int numListeners, CheckedConsumer default void runWithNewCluster(int numServers, int numListeners, boolean startCluster, CheckedConsumer testCase) throws Exception { final StackTraceElement caller = JavaUtils.getCallerStackTraceElement(); - LOG.info("Running " + caller.getMethodName()); + LOG.info("Running {}", caller.getMethodName()); final CLUSTER cluster = newCluster(numServers, numListeners); try { if (startCluster) { @@ -142,7 +142,7 @@ default void runWithNewCluster(int numServers, int numListeners, boolean startCl testCase.accept(cluster); } catch(Exception t) { LOG.info(cluster.printServers()); - LOG.error("Failed " + caller, t); + LOG.error("Failed {}", caller, t); throw t; } finally { cluster.shutdown(); @@ -156,7 +156,7 @@ default void runWithSameCluster(int numServers, CheckedConsumer testCase) throws Exception { final StackTraceElement caller = JavaUtils.getCallerStackTraceElement(); - LOG.info("Running " + caller.getMethodName()); + LOG.info("Running {}", caller.getMethodName()); CLUSTER cluster = null; try { cluster = getFactory().reuseCluster(numServers, numListeners, getProperties()); @@ -165,7 +165,7 @@ default void runWithSameCluster(int numServers, int numListeners, CheckedConsume if (cluster != null) { LOG.info(cluster.printServers()); } - LOG.error("Failed " + caller, t); + LOG.error("Failed {}", caller, t); throw t; } } @@ -317,7 +317,7 @@ public RaftProperties getProperties() { } public MiniRaftCluster initServers() { - LOG.info("servers = " + servers); + LOG.info("servers = {}", servers); if (servers.isEmpty()) { putNewServers(CollectionUtils.as(group.getPeers(), RaftPeer::getId), true, group); } @@ -348,7 +348,7 @@ public void start() throws IOException { startServers(servers.values()); this.timer.updateAndGet(t -> t != null? t - : JavaUtils.runRepeatedly(() -> LOG.info("TIMED-PRINT: " + printServers()), 10, 10, TimeUnit.SECONDS)); + : JavaUtils.runRepeatedly(() -> LOG.info("TIMED-PRINT: {}.", printServers()), 10, 10, TimeUnit.SECONDS)); } /** @@ -535,7 +535,7 @@ public PeerChanges removePeers(int number, boolean removeLeader, } public void killServer(RaftPeerId id) { - LOG.info("killServer " + id); + LOG.info("killServer {}", id); servers.get(id).close(); } From df6f2176a0264aeda41123a7ce784ec6c4fb145b Mon Sep 17 00:00:00 2001 From: jianghuazhu <740087514@qq.com> Date: Thu, 15 May 2025 02:30:01 +0800 Subject: [PATCH 179/397] RATIS-2290. Simply the EventQueue in leader (#1258) --- .../ratis/server/impl/LeaderStateImpl.java | 37 ++++++++----------- 1 file changed, 16 insertions(+), 21 deletions(-) diff --git a/ratis-server/src/main/java/org/apache/ratis/server/impl/LeaderStateImpl.java b/ratis-server/src/main/java/org/apache/ratis/server/impl/LeaderStateImpl.java index cd1d9ebea5..8358f063d2 100644 --- a/ratis-server/src/main/java/org/apache/ratis/server/impl/LeaderStateImpl.java +++ b/ratis-server/src/main/java/org/apache/ratis/server/impl/LeaderStateImpl.java @@ -111,12 +111,10 @@ private enum Type { } private final Type type; - private final long newTerm; private final Runnable handler; - StateUpdateEvent(Type type, long newTerm, Runnable handler) { + StateUpdateEvent(Type type, Runnable handler) { this.type = type; - this.newTerm = newTerm; this.handler = handler; } @@ -132,25 +130,30 @@ public boolean equals(Object obj) { return false; } final StateUpdateEvent that = (StateUpdateEvent)obj; - return this.type == that.type && this.newTerm == that.newTerm; + return this.type == that.type; } @Override public int hashCode() { - return Objects.hash(type, newTerm); + return type.hashCode(); } @Override public String toString() { - return type + (newTerm >= 0? ":" + newTerm: ""); + return type.name(); } } private class EventQueue { private final String name = ServerStringUtils.generateUnifiedName(server.getMemberId(), getClass()); - private final BlockingQueue queue = new ArrayBlockingQueue<>(4096); + private final BlockingQueue queue = new ArrayBlockingQueue<>( + StateUpdateEvent.Type.values().length);; - void submit(StateUpdateEvent event) { + // submit can be invoked by different threads -- need to be synchronized + synchronized void submit(StateUpdateEvent event) { + if (queue.contains(event)) { // avoid duplicated events + return; + } try { queue.put(event); } catch (InterruptedException e) { @@ -159,10 +162,10 @@ void submit(StateUpdateEvent event) { } } + // poll is invoked only by the EventProcessor thread -- synchronized is not needed StateUpdateEvent poll() { - final StateUpdateEvent e; try { - e = queue.poll(server.getMaxTimeoutMs(), TimeUnit.MILLISECONDS); + return queue.poll(server.getMaxTimeoutMs(), TimeUnit.MILLISECONDS); } catch (InterruptedException ie) { Thread.currentThread().interrupt(); String s = this + ": poll() is interrupted"; @@ -173,14 +176,6 @@ StateUpdateEvent poll() { throw new IllegalStateException(s + " UNEXPECTEDLY", ie); } } - - if (e != null) { - // remove duplicated events from the head. - while(e.equals(queue.peek())) { - queue.poll(); - } - } - return e; } @Override @@ -322,9 +317,9 @@ boolean isApplied() { } private final StateUpdateEvent updateCommitEvent = - new StateUpdateEvent(StateUpdateEvent.Type.UPDATE_COMMIT, -1, this::updateCommit); + new StateUpdateEvent(StateUpdateEvent.Type.UPDATE_COMMIT, this::updateCommit); private final StateUpdateEvent checkStagingEvent = - new StateUpdateEvent(StateUpdateEvent.Type.CHECK_STAGING, -1, this::checkStaging); + new StateUpdateEvent(StateUpdateEvent.Type.CHECK_STAGING, this::checkStaging); private final String name; private final RaftServerImpl server; @@ -702,7 +697,7 @@ void submitStepDownEvent(StepDownReason reason) { } void submitStepDownEvent(long term, StepDownReason reason) { - eventQueue.submit(new StateUpdateEvent(StateUpdateEvent.Type.STEP_DOWN, term, () -> stepDown(term, reason))); + eventQueue.submit(new StateUpdateEvent(StateUpdateEvent.Type.STEP_DOWN, () -> stepDown(term, reason))); } private void stepDown(long term, StepDownReason reason) { From d0b1c5827163d8d4fb103f4f726f37da98d94704 Mon Sep 17 00:00:00 2001 From: Tsz-Wo Nicholas Sze Date: Fri, 16 May 2025 04:43:24 -0700 Subject: [PATCH 180/397] RATIS-1995. Prevent data loss when a storage is accidentally re-formatted. (#1261) --- ratis-proto/src/main/proto/Raft.proto | 1 + .../ratis/server/protocol/TermIndex.java | 15 ++ .../ratis/server/impl/LeaderElection.java | 191 ++++++++++++++--- .../ratis/server/impl/RaftServerImpl.java | 2 +- .../ratis/server/impl/ServerProtoUtils.java | 4 +- .../ratis/server/util/ServerStringUtils.java | 3 +- .../TestLeaderElectionServerInterface.java | 193 ++++++++++++++++++ 7 files changed, 382 insertions(+), 27 deletions(-) create mode 100644 ratis-test/src/test/java/org/apache/ratis/server/impl/TestLeaderElectionServerInterface.java diff --git a/ratis-proto/src/main/proto/Raft.proto b/ratis-proto/src/main/proto/Raft.proto index 7cf2fd87c1..6dbfdb15a5 100644 --- a/ratis-proto/src/main/proto/Raft.proto +++ b/ratis-proto/src/main/proto/Raft.proto @@ -169,6 +169,7 @@ message RequestVoteReplyProto { RaftRpcReplyProto serverReply = 1; uint64 term = 2; bool shouldShutdown = 3; + TermIndexProto lastEntry = 4; // to determine if the voter log is empty. } message CommitInfoProto { diff --git a/ratis-server-api/src/main/java/org/apache/ratis/server/protocol/TermIndex.java b/ratis-server-api/src/main/java/org/apache/ratis/server/protocol/TermIndex.java index 6115bccadf..369aefc85f 100644 --- a/ratis-server-api/src/main/java/org/apache/ratis/server/protocol/TermIndex.java +++ b/ratis-server-api/src/main/java/org/apache/ratis/server/protocol/TermIndex.java @@ -21,9 +21,11 @@ import org.apache.ratis.proto.RaftProtos.TermIndexProto; import org.apache.ratis.server.raftlog.RaftLog; import org.apache.ratis.util.BiWeakValueCache; +import org.apache.ratis.util.MemoizedSupplier; import java.util.Comparator; import java.util.Optional; +import java.util.function.Supplier; /** The term and the log index defined in the Raft consensus algorithm. */ public interface TermIndex extends Comparable { @@ -37,6 +39,7 @@ public interface TermIndex extends Comparable { * are respectively 1 and 0 (= {@link RaftLog#LEAST_VALID_LOG_INDEX}). */ TermIndex INITIAL_VALUE = valueOf(0, RaftLog.INVALID_LOG_INDEX); + TermIndex PROTO_DEFAULT = valueOf(TermIndexProto.getDefaultInstance()); /** An empty {@link TermIndex} array. */ TermIndex[] EMPTY_ARRAY = {}; @@ -93,6 +96,8 @@ static BiWeakValueCache getCache() { private static TermIndex newTermIndex(long term, long index) { return new TermIndex() { + private final Supplier protoSupplier = MemoizedSupplier.valueOf(TermIndex.super::toProto); + @Override public long getTerm() { return term; @@ -121,12 +126,22 @@ public int hashCode() { return Long.hashCode(term) ^ Long.hashCode(index); } + @Override + public TermIndexProto toProto() { + return protoSupplier.get(); + } + private String longToString(long n) { return n >= 0L ? String.valueOf(n) : "~"; } @Override public String toString() { + if (this.equals(INITIAL_VALUE)) { + return ""; + } else if (this.equals(PROTO_DEFAULT)) { + return ""; + } return String.format("(t:%s, i:%s)", longToString(term), longToString(index)); } }; diff --git a/ratis-server/src/main/java/org/apache/ratis/server/impl/LeaderElection.java b/ratis-server/src/main/java/org/apache/ratis/server/impl/LeaderElection.java index 439405871d..9953e12aff 100644 --- a/ratis-server/src/main/java/org/apache/ratis/server/impl/LeaderElection.java +++ b/ratis-server/src/main/java/org/apache/ratis/server/impl/LeaderElection.java @@ -20,12 +20,14 @@ import org.apache.ratis.metrics.Timekeeper; import org.apache.ratis.proto.RaftProtos.RequestVoteReplyProto; import org.apache.ratis.proto.RaftProtos.RequestVoteRequestProto; +import org.apache.ratis.proto.RaftProtos.TermIndexProto; +import org.apache.ratis.protocol.RaftGroupMemberId; import org.apache.ratis.protocol.RaftPeer; import org.apache.ratis.protocol.RaftPeerId; -import org.apache.ratis.server.DivisionInfo; import org.apache.ratis.server.RaftConfiguration; import org.apache.ratis.server.RaftServerConfigKeys; import org.apache.ratis.server.protocol.TermIndex; +import org.apache.ratis.server.raftlog.RaftLog; import org.apache.ratis.server.util.ServerStringUtils; import org.apache.ratis.thirdparty.com.google.common.annotations.VisibleForTesting; import org.apache.ratis.util.Daemon; @@ -78,6 +80,121 @@ class LeaderElection implements Runnable { public static final Logger LOG = LoggerFactory.getLogger(LeaderElection.class); + interface ServerInterface { + default RaftPeerId getId() { + return getMemberId().getPeerId(); + } + + RaftGroupMemberId getMemberId(); + boolean isAlive(); + boolean isCandidate(); + + long getCurrentTerm(); + long getLastCommittedIndex(); + TermIndex getLastEntry(); + + boolean isPreVoteEnabled(); + ConfAndTerm initElection(Phase phase) throws IOException; + RequestVoteReplyProto requestVote(RequestVoteRequestProto r) throws IOException; + + void changeToLeader(); + void rejected(long term, ResultAndTerm result) throws IOException; + void shutdown(); + + Timekeeper getLeaderElectionTimer(); + void onNewLeaderElectionCompletion(); + + TimeDuration getRandomElectionTimeout(); + ThreadGroup getThreadGroup(); + + static ServerInterface get(RaftServerImpl server) { + final boolean preVote = RaftServerConfigKeys.LeaderElection.preVote(server.getRaftServer().getProperties()); + + return new ServerInterface() { + @Override + public RaftGroupMemberId getMemberId() { + return server.getMemberId(); + } + + @Override + public boolean isAlive() { + return server.getInfo().isAlive(); + } + + @Override + public boolean isCandidate() { + return server.getInfo().isCandidate(); + } + + @Override + public long getCurrentTerm() { + return server.getState().getCurrentTerm(); + } + + @Override + public long getLastCommittedIndex() { + return server.getRaftLog().getLastCommittedIndex(); + } + + @Override + public TermIndex getLastEntry() { + return server.getState().getLastEntry(); + } + + @Override + public boolean isPreVoteEnabled() { + return preVote; + } + + @Override + public ConfAndTerm initElection(Phase phase) throws IOException { + return server.getState().initElection(phase); + } + + @Override + public RequestVoteReplyProto requestVote(RequestVoteRequestProto r) throws IOException { + return server.getServerRpc().requestVote(r); + } + + @Override + public void changeToLeader() { + server.changeToLeader(); + } + + @Override + public void rejected(long term, ResultAndTerm result) throws IOException { + server.changeToFollowerAndPersistMetadata(term, false, result); + } + + @Override + public void shutdown() { + server.close(); + server.getStateMachine().event().notifyServerShutdown(server.getRoleInfoProto(), false); + } + + @Override + public Timekeeper getLeaderElectionTimer() { + return server.getLeaderElectionMetrics().getLeaderElectionTimer(); + } + + @Override + public void onNewLeaderElectionCompletion() { + server.getLeaderElectionMetrics().onNewLeaderElectionCompletion(); + } + + @Override + public TimeDuration getRandomElectionTimeout() { + return server.getRandomElectionTimeout(); + } + + @Override + public ThreadGroup getThreadGroup() { + return server.getThreadGroup(); + } + }; + } + } + private ResultAndTerm logAndReturn(Phase phase, Result result, Map responses, List exceptions) { return logAndReturn(phase, result, responses, exceptions, null); @@ -106,7 +223,7 @@ enum Phase { enum Result {PASSED, SINGLE_MODE_PASSED, REJECTED, TIMEOUT, DISCOVERED_A_NEW_TERM, SHUTDOWN, NOT_IN_CONF} - private static class ResultAndTerm { + static class ResultAndTerm { private final Result result; private final Long term; @@ -185,22 +302,24 @@ public String toString() { private final Daemon daemon; private final CompletableFuture stopped = new CompletableFuture<>(); - private final RaftServerImpl server; + private final ServerInterface server; private final boolean skipPreVote; private final ConfAndTerm round0; LeaderElection(RaftServerImpl server, boolean force) { + this(ServerInterface.get(server), force); + } + + LeaderElection(ServerInterface server, boolean force) { this.name = ServerStringUtils.generateUnifiedName(server.getMemberId(), getClass()) + COUNT.incrementAndGet(); this.lifeCycle = new LifeCycle(this); this.daemon = Daemon.newBuilder().setName(name).setRunnable(this) .setThreadGroup(server.getThreadGroup()).build(); this.server = server; - this.skipPreVote = force || - !RaftServerConfigKeys.LeaderElection.preVote( - server.getRaftServer().getProperties()); + this.skipPreVote = force || !server.isPreVoteEnabled(); try { // increase term of the candidate in advance if it's forced to election - this.round0 = force ? server.getState().initElection(Phase.ELECTION) : null; + this.round0 = force ? server.initElection(Phase.ELECTION) : null; } catch (IOException e) { throw new IllegalStateException(name + ": Failed to initialize election", e); } @@ -250,7 +369,7 @@ private void runImpl() { return; } - try (AutoCloseable ignored = Timekeeper.start(server.getLeaderElectionMetrics().getLeaderElectionTimer())) { + try (AutoCloseable ignored = Timekeeper.start(server.getLeaderElectionTimer())) { for (int round = 0; shouldRun(); round++) { if (skipPreVote || askForVotes(Phase.PRE_VOTE, round)) { if (askForVotes(Phase.ELECTION, round)) { @@ -264,10 +383,10 @@ private void runImpl() { } final LifeCycle.State state = lifeCycle.getCurrentState(); if (state.isClosingOrClosed()) { - LOG.info(this + ": since this is already " + state + ", safely ignore " + e); + LOG.info("{}: since this is already {}, safely ignore {}", this, state, e.toString()); } else { - if (!server.getInfo().isAlive()) { - LOG.info(this + ": since the server is not alive, safely ignore " + e); + if (!server.isAlive()) { + LOG.info("{}: since the server is not alive, safely ignore {}", this, e.toString()); } else { LOG.error("{}: Failed, state={}", this, state, e); } @@ -275,18 +394,17 @@ private void runImpl() { } } finally { // Update leader election completion metric(s). - server.getLeaderElectionMetrics().onNewLeaderElectionCompletion(); + server.onNewLeaderElectionCompletion(); lifeCycle.checkStateAndClose(() -> {}); } } private boolean shouldRun() { - final DivisionInfo info = server.getInfo(); - return lifeCycle.getCurrentState().isRunning() && info.isCandidate() && info.isAlive(); + return lifeCycle.getCurrentState().isRunning() && server.isCandidate() && server.isAlive(); } private boolean shouldRun(long electionTerm) { - return shouldRun() && server.getState().getCurrentTerm() == electionTerm; + return shouldRun() && server.getCurrentTerm() == electionTerm; } private ResultAndTerm submitRequestAndWaitResult(Phase phase, RaftConfigurationImpl conf, long electionTerm) @@ -299,7 +417,7 @@ private ResultAndTerm submitRequestAndWaitResult(Phase phase, RaftConfigurationI if (others.isEmpty()) { r = new ResultAndTerm(Result.PASSED, electionTerm); } else { - final TermIndex lastEntry = server.getState().getLastEntry(); + final TermIndex lastEntry = server.getLastEntry(); final Executor voteExecutor = new Executor(this, others.size()); try { final int submitted = submitRequests(phase, electionTerm, lastEntry, others, voteExecutor); @@ -322,8 +440,7 @@ private boolean askForVotes(Phase phase, int round) throws InterruptedException, } // If round0 is non-null, we have already called initElection in the constructor, // reuse round0 to avoid initElection again for the first round - final ConfAndTerm confAndTerm = (round == 0 && round0 != null) ? - round0 : server.getState().initElection(phase); + final ConfAndTerm confAndTerm = (round == 0 && round0 != null) ? round0 : server.initElection(phase); electionTerm = confAndTerm.getTerm(); conf = confAndTerm.getConf(); } @@ -343,15 +460,14 @@ private boolean askForVotes(Phase phase, int round) throws InterruptedException, return true; case NOT_IN_CONF: case SHUTDOWN: - server.close(); - server.getStateMachine().event().notifyServerShutdown(server.getRoleInfoProto(), false); + server.shutdown(); return false; case TIMEOUT: return false; // should retry case REJECTED: case DISCOVERED_A_NEW_TERM: - final long term = r.maxTerm(server.getState().getCurrentTerm()); - server.changeToFollowerAndPersistMetadata(term, false, r); + final long term = r.maxTerm(server.getCurrentTerm()); + server.rejected(term, r); return false; default: throw new IllegalArgumentException("Unable to process result " + r.result); } @@ -364,7 +480,7 @@ private int submitRequests(Phase phase, long electionTerm, TermIndex lastEntry, for (final RaftPeer peer : others) { final RequestVoteRequestProto r = ServerProtoUtils.toRequestVoteRequestProto( server.getMemberId(), peer.getId(), electionTerm, lastEntry, phase == Phase.PRE_VOTE); - voteExecutor.submit(() -> server.getServerRpc().requestVote(r)); + voteExecutor.submit(() -> server.requestVote(r)); submitted++; } return submitted; @@ -390,6 +506,9 @@ private ResultAndTerm waitForResults(Phase phase, long electionTerm, int submitt Set higherPriorityPeers = getHigherPriorityPeers(conf); final boolean singleMode = conf.isSingleMode(server.getId()); + // true iff this server does not have any commits + final boolean emptyCommit = server.getLastCommittedIndex() < RaftLog.LEAST_VALID_LOG_INDEX; + while (waitForNum > 0 && shouldRun(electionTerm)) { final TimeDuration waitTime = timeout.elapsedTime().apply(n -> -n); if (waitTime.isNonPositive()) { @@ -439,7 +558,10 @@ private ResultAndTerm waitForResults(Phase phase, long electionTerm, int submitt // all higher priority peers have replied higherPriorityPeers.remove(replierId); - if (r.getServerReply().getSuccess()) { + final boolean acceptVote = r.getServerReply().getSuccess() + // When the commits are non-empty, do not accept votes from empty log voters. + && (emptyCommit || nonEmptyLog(r)); + if (acceptVote) { votedPeers.add(replierId); // If majority and all peers with higher priority have voted, candidate pass vote if (higherPriorityPeers.isEmpty() && conf.hasMajority(votedPeers, server.getId())) { @@ -448,6 +570,7 @@ private ResultAndTerm waitForResults(Phase phase, long electionTerm, int submitt } else { rejectedPeers.add(replierId); if (conf.majorityRejectVotes(rejectedPeers)) { + LOG.info("rejectedPeers: {}, emptyCommit? {}", rejectedPeers, emptyCommit); return logAndReturn(phase, Result.REJECTED, responses, exceptions); } } @@ -467,6 +590,26 @@ private ResultAndTerm waitForResults(Phase phase, long electionTerm, int submitt } } + /** + * @return true if the given reply indicates that the voter has a non-empty raft log. + * Note that a voter running with an old version may not include the lastEntry in the reply. + * For compatibility, this method returns true for such case. + */ + static boolean nonEmptyLog(RequestVoteReplyProto reply) { + final TermIndexProto lastEntry = reply.getLastEntry(); + // valid term >= 1 and valid index >= 0; therefore, (0, 0) can only be the proto default + if (lastEntry.equals(TermIndexProto.getDefaultInstance())) { // default: (0,0) + LOG.info("Reply missing lastEntry: {} ", ServerStringUtils.toRequestVoteReplyString(reply)); + return true; // accept voters with an older version + } + if (lastEntry.getTerm() > 0) { // when log is empty, lastEntry is (0,-1). + return true; // accept voters with a non-empty log + } + + LOG.info("Replier log is empty: {} ", ServerStringUtils.toRequestVoteReplyString(reply)); + return false; // reject voters with an empty log + } + @Override public String toString() { return name; diff --git a/ratis-server/src/main/java/org/apache/ratis/server/impl/RaftServerImpl.java b/ratis-server/src/main/java/org/apache/ratis/server/impl/RaftServerImpl.java index b068265ce8..7f3ed98313 100644 --- a/ratis-server/src/main/java/org/apache/ratis/server/impl/RaftServerImpl.java +++ b/ratis-server/src/main/java/org/apache/ratis/server/impl/RaftServerImpl.java @@ -1450,7 +1450,7 @@ private RequestVoteReplyProto requestVote(Phase phase, shouldShutdown = true; } reply = toRequestVoteReplyProto(candidateId, getMemberId(), - voteGranted, state.getCurrentTerm(), shouldShutdown); + voteGranted, state.getCurrentTerm(), shouldShutdown, state.getLastEntry()); if (LOG.isInfoEnabled()) { LOG.info("{} replies to {} vote request: {}. Peer's state: {}", getMemberId(), phase, toRequestVoteReplyString(reply), state); diff --git a/ratis-server/src/main/java/org/apache/ratis/server/impl/ServerProtoUtils.java b/ratis-server/src/main/java/org/apache/ratis/server/impl/ServerProtoUtils.java index 29a42f65a0..e6a29189a8 100644 --- a/ratis-server/src/main/java/org/apache/ratis/server/impl/ServerProtoUtils.java +++ b/ratis-server/src/main/java/org/apache/ratis/server/impl/ServerProtoUtils.java @@ -43,11 +43,13 @@ private static RaftRpcReplyProto.Builder toRaftRpcReplyProtoBuilder( } static RequestVoteReplyProto toRequestVoteReplyProto( - RaftPeerId requestorId, RaftGroupMemberId replyId, boolean success, long term, boolean shouldShutdown) { + RaftPeerId requestorId, RaftGroupMemberId replyId, boolean success, long term, boolean shouldShutdown, + TermIndex lastEntry) { return RequestVoteReplyProto.newBuilder() .setServerReply(toRaftRpcReplyProtoBuilder(requestorId, replyId, success)) .setTerm(term) .setShouldShutdown(shouldShutdown) + .setLastEntry((lastEntry != null? lastEntry : TermIndex.INITIAL_VALUE).toProto()) .build(); } diff --git a/ratis-server/src/main/java/org/apache/ratis/server/util/ServerStringUtils.java b/ratis-server/src/main/java/org/apache/ratis/server/util/ServerStringUtils.java index 6601eddce0..3a5db62859 100644 --- a/ratis-server/src/main/java/org/apache/ratis/server/util/ServerStringUtils.java +++ b/ratis-server/src/main/java/org/apache/ratis/server/util/ServerStringUtils.java @@ -118,7 +118,8 @@ public static String toRequestVoteReplyString(RequestVoteReplyProto proto) { if (proto == null) { return null; } - return ProtoUtils.toString(proto.getServerReply()) + "-t" + proto.getTerm(); + return ProtoUtils.toString(proto.getServerReply()) + "-t" + proto.getTerm() + + "-last:" + TermIndex.valueOf(proto.getLastEntry()); } /** diff --git a/ratis-test/src/test/java/org/apache/ratis/server/impl/TestLeaderElectionServerInterface.java b/ratis-test/src/test/java/org/apache/ratis/server/impl/TestLeaderElectionServerInterface.java new file mode 100644 index 0000000000..876633db1d --- /dev/null +++ b/ratis-test/src/test/java/org/apache/ratis/server/impl/TestLeaderElectionServerInterface.java @@ -0,0 +1,193 @@ +/* + * 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.ratis.server.impl; + +import org.apache.ratis.BaseTest; +import org.apache.ratis.metrics.Timekeeper; +import org.apache.ratis.proto.RaftProtos.RequestVoteReplyProto; +import org.apache.ratis.proto.RaftProtos.RequestVoteRequestProto; +import org.apache.ratis.protocol.RaftGroup; +import org.apache.ratis.protocol.RaftGroupId; +import org.apache.ratis.protocol.RaftGroupMemberId; +import org.apache.ratis.protocol.RaftPeer; +import org.apache.ratis.protocol.RaftPeerId; +import org.apache.ratis.server.protocol.TermIndex; +import org.apache.ratis.util.TimeDuration; +import org.junit.jupiter.api.Test; + +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import java.util.concurrent.ThreadLocalRandom; +import java.util.concurrent.TimeUnit; +import java.util.stream.Collectors; +import java.util.stream.IntStream; + +import static org.junit.jupiter.api.Assertions.assertFalse; +import static org.junit.jupiter.api.Assertions.assertTrue; +import static org.junit.jupiter.api.Assertions.fail; + +public class TestLeaderElectionServerInterface extends BaseTest { + private final List peers = IntStream.range(0, 3).boxed() + .map(i -> RaftPeer.newBuilder().setId("s" + i).build()) + .collect(Collectors.toList()); + private final RaftGroup group = RaftGroup.valueOf(RaftGroupId.randomId(), peers); + private final RaftConfigurationImpl conf = RaftConfigurationImpl.newBuilder().setLogEntryIndex(0).setConf(peers).build(); + private final ThreadGroup threadGroup = new ThreadGroup("ServerInterface"); + + private final RaftGroupMemberId candidate = RaftGroupMemberId.valueOf(peers.get(0).getId(), group.getGroupId()); + + LeaderElection.ServerInterface newServerInterface(boolean expectToPass, + Map lastEntries) { + return new LeaderElection.ServerInterface() { + private volatile boolean isAlive = true; + + @Override + public RaftGroupMemberId getMemberId() { + return candidate; + } + + @Override + public boolean isAlive() { + return isAlive; + } + + @Override + public boolean isCandidate() { + return true; + } + + @Override + public long getCurrentTerm() { + final TermIndex lastEntry = getLastEntry(); + return lastEntry != null? lastEntry.getTerm() : TermIndex.INITIAL_VALUE.getTerm(); + } + + @Override + public long getLastCommittedIndex() { + final TermIndex lastEntry = getLastEntry(); + return lastEntry != null? lastEntry.getIndex() : TermIndex.INITIAL_VALUE.getIndex(); + } + + @Override + public TermIndex getLastEntry() { + return lastEntries.get(getId()); + } + + @Override + public boolean isPreVoteEnabled() { + return false; + } + + @Override + public LeaderElection.ConfAndTerm initElection(LeaderElection.Phase phase) { + return new LeaderElection.ConfAndTerm(conf, getCurrentTerm()); + } + + @Override + public RequestVoteReplyProto requestVote(RequestVoteRequestProto r) { + final RaftPeerId voterPeerId = RaftPeerId.valueOf(r.getServerRequest().getReplyId()); + final RaftGroupMemberId voter = RaftGroupMemberId.valueOf(voterPeerId, group.getGroupId()); + final TermIndex lastEntry = lastEntries.get(voterPeerId); + final long term = (lastEntry != null? lastEntry : TermIndex.INITIAL_VALUE).getTerm(); + + // voter replies to candidate + return ServerProtoUtils.toRequestVoteReplyProto(getId(), voter, true, term, false, lastEntry); + } + + @Override + public void changeToLeader() { + assertTrue(expectToPass); + isAlive = false; + } + + @Override + public void rejected(long term, LeaderElection.ResultAndTerm result) { + assertFalse(expectToPass); + isAlive = false; + } + + @Override + public void shutdown() { + fail(); + } + + @Override + public Timekeeper getLeaderElectionTimer() { + final long start = System.nanoTime(); + final Timekeeper.Context context = () -> System.nanoTime() - start; + return () -> context; + } + + @Override + public void onNewLeaderElectionCompletion() { + // no op + } + + @Override + public TimeDuration getRandomElectionTimeout() { + final int millis = 100 + ThreadLocalRandom.current().nextInt(100); + return TimeDuration.valueOf(millis, TimeUnit.MILLISECONDS); + } + + @Override + public ThreadGroup getThreadGroup() { + return threadGroup; + } + }; + } + + @Test + public void testVoterWithEmptyLog() { + // all the candidate and the voters have an empty log + // expect to pass: empty-log-candidate will accept votes from empty-log-voters + runTestVoterWithEmptyLog(true); + + // candidate: non-empty commit + // voter 1 : empty log + // voter 2 : empty log + // expect to fail: non-empty-commit-candidate will NOT accept votes from empty-log-voters + final TermIndex candidateLastEntry = TermIndex.valueOf(2, 9); + runTestVoterWithEmptyLog(false, candidateLastEntry); + + // candidate: non-empty commit + // voter 1 : non-empty log + // voter 2 : empty log + // expect to pass: non-empty-commit-candidate will accept votes from non-empty-log-voters + final TermIndex voterLastEntry = TermIndex.valueOf(2, 7); + runTestVoterWithEmptyLog(true, candidateLastEntry, voterLastEntry); + + // candidate: non-empty log + // voter 1 : older version + // voter 2 : empty log + // expect to pass: non-empty-commit-candidate will accept votes from older-version-voters + runTestVoterWithEmptyLog(true, candidateLastEntry, TermIndex.PROTO_DEFAULT); + } + + void runTestVoterWithEmptyLog(boolean expectToPass, TermIndex... lastEntries) { + LOG.info("expectToPass? {}, lastEntries={}", + expectToPass, lastEntries); + final Map map = new HashMap<>(); + for(int i = 0; i < lastEntries.length; i++) { + map.put(peers.get(i).getId(), lastEntries[i]); + } + final LeaderElection election = new LeaderElection(newServerInterface(expectToPass, map), false); + election.startInForeground(); + } + +} \ No newline at end of file From 0df68e6282d29f1df14774ac49400bbb6cbacc44 Mon Sep 17 00:00:00 2001 From: jianghuazhu <740087514@qq.com> Date: Sun, 18 May 2025 20:21:57 +0800 Subject: [PATCH 181/397] RATIS-2293. Fix typo in ClientProtoUtils (#1263) --- .../java/org/apache/ratis/client/impl/ClientProtoUtils.java | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/ratis-client/src/main/java/org/apache/ratis/client/impl/ClientProtoUtils.java b/ratis-client/src/main/java/org/apache/ratis/client/impl/ClientProtoUtils.java index 44e3a5cd63..36c0b3937f 100644 --- a/ratis-client/src/main/java/org/apache/ratis/client/impl/ClientProtoUtils.java +++ b/ratis-client/src/main/java/org/apache/ratis/client/impl/ClientProtoUtils.java @@ -179,11 +179,11 @@ static RaftClientRequest toRaftClientRequest(RaftClientRequestProto p) { final RaftClientRequest.Builder b = RaftClientRequest.newBuilder(); - final RaftPeerId perrId = RaftPeerId.valueOf(request.getReplyId()); + final RaftPeerId peerId = RaftPeerId.valueOf(request.getReplyId()); if (request.getToLeader()) { - b.setLeaderId(perrId); + b.setLeaderId(peerId); } else { - b.setServerId(perrId); + b.setServerId(peerId); } if (request.hasSlidingWindowEntry()) { b.setSlidingWindowEntry(request.getSlidingWindowEntry()); From 70a601c841d290edbd0abe0a645d2cbfb7fc54df Mon Sep 17 00:00:00 2001 From: slfan1989 <55643692+slfan1989@users.noreply.github.com> Date: Tue, 20 May 2025 01:52:24 +0800 Subject: [PATCH 182/397] RATIS-2294. Fix NettyClientRpc exception and timeout handling (#1264) --- .../ratis/netty/client/NettyClientRpc.java | 48 ++++++++++++++++++- .../java/org/apache/ratis/RaftAsyncTests.java | 20 +++++--- .../java/org/apache/ratis/RaftBasicTests.java | 2 +- 3 files changed, 61 insertions(+), 9 deletions(-) diff --git a/ratis-netty/src/main/java/org/apache/ratis/netty/client/NettyClientRpc.java b/ratis-netty/src/main/java/org/apache/ratis/netty/client/NettyClientRpc.java index 26ac41f7db..ef34caf17d 100644 --- a/ratis-netty/src/main/java/org/apache/ratis/netty/client/NettyClientRpc.java +++ b/ratis-netty/src/main/java/org/apache/ratis/netty/client/NettyClientRpc.java @@ -17,6 +17,7 @@ */ package org.apache.ratis.netty.client; +import org.apache.ratis.client.RaftClientConfigKeys; import org.apache.ratis.client.impl.ClientProtoUtils; import org.apache.ratis.client.impl.RaftClientRpcWithProxy; import org.apache.ratis.conf.RaftProperties; @@ -28,23 +29,40 @@ import org.apache.ratis.proto.RaftProtos.GroupManagementRequestProto; import org.apache.ratis.proto.RaftProtos.SetConfigurationRequestProto; import org.apache.ratis.proto.netty.NettyProtos.RaftNettyServerRequestProto; +import org.apache.ratis.protocol.exceptions.TimeoutIOException; import org.apache.ratis.util.JavaUtils; +import org.apache.ratis.util.TimeDuration; +import org.apache.ratis.util.TimeoutExecutor; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; import java.io.IOException; import java.util.concurrent.CompletableFuture; public class NettyClientRpc extends RaftClientRpcWithProxy { + + public static final Logger LOG = LoggerFactory.getLogger(NettyClientRpc.class); + + private ClientId clientId; + private final TimeDuration requestTimeout; + private final TimeoutExecutor scheduler = TimeoutExecutor.getInstance(); + public NettyClientRpc(ClientId clientId, RaftProperties properties) { super(new NettyRpcProxy.PeerMap(clientId.toString(), properties)); + this.clientId = clientId; + this.requestTimeout = RaftClientConfigKeys.Rpc.requestTimeout(properties); } @Override public CompletableFuture sendRequestAsync(RaftClientRequest request) { final RaftPeerId serverId = request.getServerId(); + long callId = request.getCallId(); try { final NettyRpcProxy proxy = getProxies().getProxy(serverId); final RaftNettyServerRequestProto serverRequestProto = buildRequestProto(request); - return proxy.sendAsync(serverRequestProto).thenApply(replyProto -> { + final CompletableFuture replyFuture = new CompletableFuture<>(); + + proxy.sendAsync(serverRequestProto).thenApply(replyProto -> { if (request instanceof GroupListRequest) { return ClientProtoUtils.toGroupListReply(replyProto.getGroupListReply()); } else if (request instanceof GroupInfoRequest) { @@ -52,7 +70,35 @@ public CompletableFuture sendRequestAsync(RaftClientRequest req } else { return ClientProtoUtils.toRaftClientReply(replyProto.getRaftClientReply()); } + }).whenComplete((reply, e) -> { + if (e == null) { + if (reply == null) { + e = new NullPointerException("Both reply==null && e==null"); + } + if (e == null) { + e = reply.getNotLeaderException(); + } + if (e == null) { + e = reply.getLeaderNotReadyException(); + } + } + + if (e != null) { + replyFuture.completeExceptionally(e); + } else { + replyFuture.complete(reply); + } }); + + scheduler.onTimeout(requestTimeout, () -> { + if (!replyFuture.isDone()) { + final String s = clientId + "->" + serverId + " request #" + + callId + " timeout " + requestTimeout.getDuration(); + replyFuture.completeExceptionally(new TimeoutIOException(s)); + } + }, LOG, () -> "Timeout check for client request #" + callId); + + return replyFuture; } catch (Throwable e) { return JavaUtils.completeExceptionally(e); } diff --git a/ratis-server/src/test/java/org/apache/ratis/RaftAsyncTests.java b/ratis-server/src/test/java/org/apache/ratis/RaftAsyncTests.java index 3a760a8065..a1c16df8f2 100644 --- a/ratis-server/src/test/java/org/apache/ratis/RaftAsyncTests.java +++ b/ratis-server/src/test/java/org/apache/ratis/RaftAsyncTests.java @@ -47,6 +47,7 @@ import org.apache.ratis.thirdparty.com.google.protobuf.ByteString; import org.apache.ratis.thirdparty.com.google.protobuf.InvalidProtocolBufferException; import org.apache.ratis.util.JavaUtils; +import org.apache.ratis.util.PlatformUtils; import org.apache.ratis.util.Slf4jUtils; import org.apache.ratis.util.TimeDuration; import org.apache.ratis.util.function.CheckedRunnable; @@ -83,6 +84,10 @@ public abstract class RaftAsyncTests extends Ba { getProperties().setClass(MiniRaftCluster.STATEMACHINE_CLASS_KEY, SimpleStateMachine4Testing.class, StateMachine.class); + if (!PlatformUtils.LINUX) { + getProperties().setBoolean("raft.netty.server.use-epoll", false); + getProperties().setBoolean("raft.netty.client.use-epoll", false); + } } @Test @@ -282,8 +287,8 @@ public void testStaleReadAsync() throws Exception { void runTestStaleReadAsync(CLUSTER cluster) throws Exception { final int numMessages = 10; - try (RaftClient client = cluster.createClient()) { - RaftTestUtil.waitForLeader(cluster); + RaftServer.Division division = waitForLeader(cluster); + try (RaftClient client = cluster.createClient(division.getId())) { // submit some messages final List> futures = new ArrayList<>(); @@ -304,6 +309,7 @@ void runTestStaleReadAsync(CLUSTER cluster) throws Exception { // Use a follower with the max commit index final RaftClientReply lastWriteReply = replies.get(replies.size() - 1); final RaftPeerId leader = lastWriteReply.getServerId(); + Assert.assertEquals(leader, lastWriteReply.getServerId()); LOG.info("leader = " + leader); final Collection commitInfos = lastWriteReply.getCommitInfos(); LOG.info("commitInfos = " + commitInfos); @@ -366,8 +372,8 @@ public void testWriteAsyncCustomReplicationLevel() throws Exception { void runTestWriteAsyncCustomReplicationLevel(CLUSTER cluster) throws Exception { final int numMessages = 20; - try (RaftClient client = cluster.createClient()) { - RaftTestUtil.waitForLeader(cluster); + final RaftPeerId leader = waitForLeader(cluster).getId(); + try (RaftClient client = cluster.createClient(leader)) { // submit some messages for (int i = 0; i < numMessages; i++) { @@ -417,13 +423,13 @@ void runTestAppendEntriesTimeout(CLUSTER cluster) throws Exception { LOG.info("Running testAppendEntriesTimeout"); final TimeDuration oldExpiryTime = RaftServerConfigKeys.RetryCache.expiryTime(getProperties()); RaftServerConfigKeys.RetryCache.setExpiryTime(getProperties(), TimeDuration.valueOf(20, TimeUnit.SECONDS)); - waitForLeader(cluster); + final RaftPeerId leader = waitForLeader(cluster).getId(); long time = System.currentTimeMillis(); long waitTime = 5000; try (final RaftClient client = cluster.createClient()) { // block append requests cluster.getServerAliveStream() - .filter(impl -> !impl.getInfo().isLeader()) + .filter(impl -> !impl.getInfo().isLeader() && !impl.getPeer().getId().equals(leader)) .map(SimpleStateMachine4Testing::get) .forEach(SimpleStateMachine4Testing::blockWriteStateMachineData); @@ -433,7 +439,7 @@ void runTestAppendEntriesTimeout(CLUSTER cluster) throws Exception { Assert.assertFalse(replyFuture.isDone()); // unblock append request. cluster.getServerAliveStream() - .filter(impl -> !impl.getInfo().isLeader()) + .filter(impl -> !impl.getInfo().isLeader() && !impl.getPeer().getId().equals(leader)) .map(SimpleStateMachine4Testing::get) .forEach(SimpleStateMachine4Testing::unblockWriteStateMachineData); diff --git a/ratis-server/src/test/java/org/apache/ratis/RaftBasicTests.java b/ratis-server/src/test/java/org/apache/ratis/RaftBasicTests.java index adef385674..156cecf0b0 100644 --- a/ratis-server/src/test/java/org/apache/ratis/RaftBasicTests.java +++ b/ratis-server/src/test/java/org/apache/ratis/RaftBasicTests.java @@ -463,7 +463,7 @@ public void testStateMachineMetrics() throws Exception { static void runTestStateMachineMetrics(boolean async, MiniRaftCluster cluster) throws Exception { RaftServer.Division leader = waitForLeader(cluster); - try (final RaftClient client = cluster.createClient()) { + try (final RaftClient client = cluster.createClient(leader.getId())) { Gauge appliedIndexGauge = getStatemachineGaugeWithName(leader, STATEMACHINE_APPLIED_INDEX_GAUGE); Gauge smAppliedIndexGauge = getStatemachineGaugeWithName(leader, From 5c663cecf026931d94719f93b18ffe02eefa1236 Mon Sep 17 00:00:00 2001 From: slfan1989 <55643692+slfan1989@users.noreply.github.com> Date: Wed, 21 May 2025 00:09:01 +0800 Subject: [PATCH 183/397] RATIS-2251. Migrate ratis-test tests to Junit 5 - Part 3. (#1227) --- .../InstallSnapshotNotificationTests.java | 80 +++++----- .../org/apache/ratis/LogAppenderTests.java | 14 +- .../apache/ratis/MessageStreamApiTests.java | 16 +- .../apache/ratis/OutputStreamBaseTest.java | 30 ++-- .../apache/ratis/RaftAsyncExceptionTests.java | 14 +- .../java/org/apache/ratis/RaftAsyncTests.java | 54 +++---- .../apache/ratis/RaftExceptionBaseTest.java | 20 +-- .../java/org/apache/ratis/RaftTestUtil.java | 23 ++- .../apache/ratis/ReadOnlyRequestTests.java | 46 +++--- .../ReadOnlyRequestWithLongTimeoutTests.java | 20 +-- .../ratis/RequestLimitAsyncBaseTest.java | 8 +- .../apache/ratis/TestReConfigProperty.java | 142 +++++++----------- .../org/apache/ratis/WatchRequestTests.java | 62 ++++---- .../ratis/server/impl/GroupInfoBaseTest.java | 12 +- .../server/impl/GroupManagementBaseTest.java | 70 ++++----- .../ratis/server/impl/LogMetadataTests.java | 6 +- .../impl/PreAppendLeaderStepDownTest.java | 14 +- .../impl/RaftReconfigurationBaseTest.java | 94 ++++++------ .../ratis/server/impl/RaftServerTestUtil.java | 22 +-- .../impl/RaftStateMachineExceptionTests.java | 32 ++-- .../ratis/server/impl/RetryCacheTestUtil.java | 4 +- .../server/impl/ServerPauseResumeTest.java | 22 +-- .../impl/TestRatisServerMetricsBase.java | 8 +- .../statemachine/RaftSnapshotBaseTest.java | 2 +- .../statemachine/SnapshotManagementTest.java | 44 +++--- ratis-test/pom.xml | 10 ++ .../ratis/grpc/TestRaftAsyncWithGrpc.java | 2 + .../ratis/netty/TestRaftAsyncWithNetty.java | 2 + 28 files changed, 431 insertions(+), 442 deletions(-) diff --git a/ratis-server/src/test/java/org/apache/ratis/InstallSnapshotNotificationTests.java b/ratis-server/src/test/java/org/apache/ratis/InstallSnapshotNotificationTests.java index ae5d792246..411c93120f 100644 --- a/ratis-server/src/test/java/org/apache/ratis/InstallSnapshotNotificationTests.java +++ b/ratis-server/src/test/java/org/apache/ratis/InstallSnapshotNotificationTests.java @@ -39,8 +39,8 @@ import org.apache.ratis.util.JavaUtils; import org.apache.ratis.util.Slf4jUtils; import org.apache.ratis.util.SizeInBytes; -import org.junit.Assert; -import org.junit.Test; +import org.junit.jupiter.api.Assertions; +import org.junit.jupiter.api.Test; import org.slf4j.Logger; import org.slf4j.LoggerFactory; import org.slf4j.event.Level; @@ -194,7 +194,7 @@ private void testAddNewFollowers(CLUSTER cluster, int numRequests) throws Except try(final RaftClient client = cluster.createClient(leaderId)) { for (; i < numRequests; i++) { final RaftClientReply reply = client.io().send(new RaftTestUtil.SimpleMessage("m" + i)); - Assert.assertTrue(reply.isSuccess()); + Assertions.assertTrue(reply.isSuccess()); } } @@ -206,7 +206,7 @@ private void testAddNewFollowers(CLUSTER cluster, int numRequests) throws Except final List snapshotFiles = RaftSnapshotBaseTest.getSnapshotFiles(cluster, nextIndex - SNAPSHOT_TRIGGER_THRESHOLD, nextIndex); JavaUtils.attemptRepeatedly(() -> { - Assert.assertTrue(snapshotFiles.stream().anyMatch(RaftSnapshotBaseTest::exists)); + Assertions.assertTrue(snapshotFiles.stream().anyMatch(RaftSnapshotBaseTest::exists)); return null; }, 10, ONE_SECOND, "snapshotFile.exist", LOG); logs = LogSegmentPath.getLogSegmentPaths(leader.getRaftStorage()); @@ -220,7 +220,7 @@ private void testAddNewFollowers(CLUSTER cluster, int numRequests) throws Except // delete the log segments from the leader LOG.info("Delete logs {}", logs); for (LogSegmentPath path : logs) { - FileUtils.deleteFully(path.getPath()); // the log may be already purged + FileUtils.deleteFully(path.getPath()); // the log may be already puged } // restart the peer @@ -231,13 +231,13 @@ private void testAddNewFollowers(CLUSTER cluster, int numRequests) throws Except // generate some more traffic try(final RaftClient client = cluster.createClient(cluster.getLeader().getId())) { - Assert.assertTrue(client.io().send(new RaftTestUtil.SimpleMessage("m" + i)).isSuccess()); + Assertions.assertTrue(client.io().send(new RaftTestUtil.SimpleMessage("m" + i)).isSuccess()); } final SnapshotInfo leaderSnapshotInfo = cluster.getLeader().getStateMachine().getLatestSnapshot(); LOG.info("LeaderSnapshotInfo: {}", leaderSnapshotInfo.getTermIndex()); final boolean set = LEADER_SNAPSHOT_INFO_REF.compareAndSet(null, leaderSnapshotInfo); - Assert.assertTrue(set); + Assertions.assertTrue(set); // add two more peers final MiniRaftCluster.PeerChanges change = cluster.addNewPeers(2, true, @@ -253,7 +253,7 @@ private void testAddNewFollowers(CLUSTER cluster, int numRequests) throws Except // leader snapshot. for (RaftServer.Division follower : cluster.getFollowers()) { final long expected = leaderSnapshotInfo.getIndex(); - Assert.assertEquals(expected, RaftServerTestUtil.getLatestInstalledSnapshotIndex(follower)); + Assertions.assertEquals(expected, RaftServerTestUtil.getLatestInstalledSnapshotIndex(follower)); RaftSnapshotBaseTest.assertLogContent(follower, false); } @@ -279,7 +279,7 @@ private void testRestartFollower(CLUSTER cluster) throws Exception { try (final RaftClient client = cluster.createClient(leaderId)) { for (; i < SNAPSHOT_TRIGGER_THRESHOLD * 2 - 1; i++) { final RaftClientReply reply = client.io().send(new RaftTestUtil.SimpleMessage("m" + i)); - Assert.assertTrue(reply.isSuccess()); + Assertions.assertTrue(reply.isSuccess()); } } @@ -290,7 +290,7 @@ private void testRestartFollower(CLUSTER cluster) throws Exception { final List snapshotFiles = RaftSnapshotBaseTest.getSnapshotFiles(cluster, oldLeaderNextIndex - SNAPSHOT_TRIGGER_THRESHOLD, oldLeaderNextIndex); JavaUtils.attemptRepeatedly(() -> { - Assert.assertTrue(snapshotFiles.stream().anyMatch(RaftSnapshotBaseTest::exists)); + Assertions.assertTrue(snapshotFiles.stream().anyMatch(RaftSnapshotBaseTest::exists)); return null; }, 10, ONE_SECOND, "snapshotFile.exist", LOG); } @@ -300,7 +300,7 @@ private void testRestartFollower(CLUSTER cluster) throws Exception { // generate some more traffic try (final RaftClient client = cluster.createClient(leader.getId())) { - Assert.assertTrue(client.io().send(new RaftTestUtil.SimpleMessage("m" + i)).isSuccess()); + Assertions.assertTrue(client.io().send(new RaftTestUtil.SimpleMessage("m" + i)).isSuccess()); } FIVE_SECONDS.sleep(); @@ -309,8 +309,8 @@ private void testRestartFollower(CLUSTER cluster) throws Exception { JavaUtils.attempt(() -> { final long newLeaderNextIndex = leader.getRaftLog().getNextIndex(); LOG.info("{}: newLeaderNextIndex = {}", leaderId, newLeaderNextIndex); - Assert.assertTrue(newLeaderNextIndex > oldLeaderNextIndex); - Assert.assertEquals(newLeaderNextIndex, follower.getRaftLog().getNextIndex()); + Assertions.assertTrue(newLeaderNextIndex > oldLeaderNextIndex); + Assertions.assertEquals(newLeaderNextIndex, follower.getRaftLog().getNextIndex()); }, 10, ONE_SECOND, "followerNextIndex", LOG); } @@ -331,14 +331,14 @@ private void testInstallSnapshotNotificationCount(CLUSTER cluster) throws Except // Let a few heartbeats pass. ONE_SECOND.sleep(); - Assert.assertEquals(0, numSnapshotRequests.get()); + Assertions.assertEquals(0, numSnapshotRequests.get()); // Generate data. try(final RaftClient client = cluster.createClient(leaderId)) { for (; i < 10; i++) { RaftClientReply reply = client.io().send(new RaftTestUtil.SimpleMessage("m" + i)); - Assert.assertTrue(reply.isSuccess()); + Assertions.assertTrue(reply.isSuccess()); } } @@ -349,42 +349,42 @@ private void testInstallSnapshotNotificationCount(CLUSTER cluster) throws Except // Take snapshot and check result. long snapshotIndex = cluster.getLeader().getStateMachine().takeSnapshot(); - Assert.assertEquals(20, snapshotIndex); + Assertions.assertEquals(20, snapshotIndex); final SnapshotInfo leaderSnapshotInfo = cluster.getLeader().getStateMachine().getLatestSnapshot(); - Assert.assertEquals(20, leaderSnapshotInfo.getIndex()); + Assertions.assertEquals(20, leaderSnapshotInfo.getIndex()); final boolean set = LEADER_SNAPSHOT_INFO_REF.compareAndSet(null, leaderSnapshotInfo); - Assert.assertTrue(set); + Assertions.assertTrue(set); // Wait for the snapshot to be done. final RaftServer.Division leader = cluster.getLeader(); final long nextIndex = leader.getRaftLog().getNextIndex(); - Assert.assertEquals(21, nextIndex); + Assertions.assertEquals(21, nextIndex); // End index is exclusive. final List snapshotFiles = RaftSnapshotBaseTest.getSnapshotFiles(cluster, 0, nextIndex); JavaUtils.attemptRepeatedly(() -> { - Assert.assertTrue(snapshotFiles.stream().anyMatch(RaftSnapshotBaseTest::exists)); + Assertions.assertTrue(snapshotFiles.stream().anyMatch(RaftSnapshotBaseTest::exists)); return null; }, 10, ONE_SECOND, "snapshotFile.exist", LOG); // Clear all log files and reset cached log start index. long snapshotInstallIndex = leader.getRaftLog().onSnapshotInstalled(leader.getRaftLog().getLastCommittedIndex()).get(); - Assert.assertEquals(20, snapshotInstallIndex); + Assertions.assertEquals(20, snapshotInstallIndex); // Check that logs are gone. - Assert.assertEquals(0, + Assertions.assertEquals(0, LogSegmentPath.getLogSegmentPaths(leader.getRaftStorage()).size()); - Assert.assertEquals(RaftLog.INVALID_LOG_INDEX, leader.getRaftLog().getStartIndex()); + Assertions.assertEquals(RaftLog.INVALID_LOG_INDEX, leader.getRaftLog().getStartIndex()); // Allow some heartbeats to go through, then make sure none of them had // snapshot requests. ONE_SECOND.sleep(); - Assert.assertEquals(0, numSnapshotRequests.get()); + Assertions.assertEquals(0, numSnapshotRequests.get()); // Make sure leader and followers are still up to date. for (RaftServer.Division follower : cluster.getFollowers()) { - Assert.assertEquals( + Assertions.assertEquals( leader.getRaftLog().getNextIndex(), follower.getRaftLog().getNextIndex()); } @@ -400,7 +400,7 @@ private void testInstallSnapshotNotificationCount(CLUSTER cluster) throws Except // Generate more data. try (final RaftClient client = cluster.createClient(leader.getId())) { - Assert.assertTrue(client.io().send(new RaftTestUtil.SimpleMessage("m" + i)).isSuccess()); + Assertions.assertTrue(client.io().send(new RaftTestUtil.SimpleMessage("m" + i)).isSuccess()); } // Make sure leader and followers are still up to date. @@ -412,7 +412,7 @@ private void testInstallSnapshotNotificationCount(CLUSTER cluster) throws Except } // Make sure each new peer got one snapshot notification. - Assert.assertEquals(2, numSnapshotRequests.get()); + Assertions.assertEquals(2, numSnapshotRequests.get()); } finally { cluster.shutdown(); @@ -437,7 +437,7 @@ private void testInstallSnapshotInstalledEvent(CLUSTER cluster) throws Exception for (; i < SNAPSHOT_TRIGGER_THRESHOLD * 2 - 1; i++) { RaftClientReply reply = client.io().send(new RaftTestUtil.SimpleMessage("m" + i)); - Assert.assertTrue(reply.isSuccess()); + Assertions.assertTrue(reply.isSuccess()); } } @@ -448,7 +448,7 @@ private void testInstallSnapshotInstalledEvent(CLUSTER cluster) throws Exception final List snapshotFiles = RaftSnapshotBaseTest.getSnapshotFiles(cluster, nextIndex - SNAPSHOT_TRIGGER_THRESHOLD, nextIndex); JavaUtils.attemptRepeatedly(() -> { - Assert.assertTrue(snapshotFiles.stream().anyMatch(RaftSnapshotBaseTest::exists)); + Assertions.assertTrue(snapshotFiles.stream().anyMatch(RaftSnapshotBaseTest::exists)); return null; }, 10, ONE_SECOND, "snapshotFile.exist", LOG); logs = LogSegmentPath.getLogSegmentPaths(leader.getRaftStorage()); @@ -470,13 +470,13 @@ private void testInstallSnapshotInstalledEvent(CLUSTER cluster) throws Exception // generate some more traffic try(final RaftClient client = cluster.createClient(cluster.getLeader().getId())) { - Assert.assertTrue(client.io().send(new RaftTestUtil.SimpleMessage("m" + i)).isSuccess()); + Assertions.assertTrue(client.io().send(new RaftTestUtil.SimpleMessage("m" + i)).isSuccess()); } final SnapshotInfo leaderSnapshotInfo = cluster.getLeader().getStateMachine().getLatestSnapshot(); LOG.info("LeaderSnapshotInfo: {}", leaderSnapshotInfo.getTermIndex()); final boolean set = LEADER_SNAPSHOT_INFO_REF.compareAndSet(null, leaderSnapshotInfo); - Assert.assertTrue(set); + Assertions.assertTrue(set); // add one new peer final MiniRaftCluster.PeerChanges change = cluster.addNewPeers(1, true, true); @@ -490,7 +490,7 @@ private void testInstallSnapshotInstalledEvent(CLUSTER cluster) throws Exception // Check the installed snapshot index on each Follower matches with the // leader snapshot. for (RaftServer.Division follower : cluster.getFollowers()) { - Assert.assertEquals(leaderSnapshotInfo.getIndex(), + Assertions.assertEquals(leaderSnapshotInfo.getIndex(), RaftServerTestUtil.getLatestInstalledSnapshotIndex(follower)); } @@ -500,9 +500,9 @@ private void testInstallSnapshotInstalledEvent(CLUSTER cluster) throws Exception (SimpleStateMachine4Testing) cluster.getFollowers().get(0).getStateMachine(); final File followerSnapshotFile = new File(followerStateMachine.getStateMachineDir(), leaderSnapshotFile.getName()); - Assert.assertEquals(numNotifyInstallSnapshotFinished.get(), 2); - Assert.assertTrue(leaderSnapshotFile.exists()); - Assert.assertFalse(followerSnapshotFile.exists()); + Assertions.assertEquals(numNotifyInstallSnapshotFinished.get(), 2); + Assertions.assertTrue(leaderSnapshotFile.exists()); + Assertions.assertFalse(followerSnapshotFile.exists()); // restart the peer and check if it can correctly handle conf change cluster.restartServer(cluster.getLeader().getId(), false); @@ -535,7 +535,7 @@ private void testInstallSnapshotDuringBootstrap(CLUSTER cluster) throws Exceptio for (; i < SNAPSHOT_TRIGGER_THRESHOLD * 2 - 1; i++) { RaftClientReply reply = client.io().send(new RaftTestUtil.SimpleMessage("m" + i)); - Assert.assertTrue(reply.isSuccess()); + Assertions.assertTrue(reply.isSuccess()); } } @@ -546,7 +546,7 @@ private void testInstallSnapshotDuringBootstrap(CLUSTER cluster) throws Exceptio final List snapshotFiles = RaftSnapshotBaseTest.getSnapshotFiles(cluster, nextIndex - SNAPSHOT_TRIGGER_THRESHOLD, nextIndex); JavaUtils.attemptRepeatedly(() -> { - Assert.assertTrue(snapshotFiles.stream().anyMatch(RaftSnapshotBaseTest::exists)); + Assertions.assertTrue(snapshotFiles.stream().anyMatch(RaftSnapshotBaseTest::exists)); return null; }, 10, ONE_SECOND, "snapshotFile.exist", LOG); @@ -554,7 +554,7 @@ private void testInstallSnapshotDuringBootstrap(CLUSTER cluster) throws Exceptio final SnapshotInfo leaderSnapshotInfo = cluster.getLeader().getStateMachine().getLatestSnapshot(); final boolean set = LEADER_SNAPSHOT_INFO_REF.compareAndSet(null, leaderSnapshotInfo); - Assert.assertTrue(set); + Assertions.assertTrue(set); // add two more peers final MiniRaftCluster.PeerChanges change = cluster.addNewPeers(2, true, @@ -568,12 +568,12 @@ private void testInstallSnapshotDuringBootstrap(CLUSTER cluster) throws Exceptio // Check the installed snapshot index on each Follower matches with the // leader snapshot. for (RaftServer.Division follower : cluster.getFollowers()) { - Assert.assertEquals(leaderSnapshotInfo.getIndex(), + Assertions.assertEquals(leaderSnapshotInfo.getIndex(), RaftServerTestUtil.getLatestInstalledSnapshotIndex(follower)); } // Make sure each new peer got at least one snapshot notification. - Assert.assertTrue(2 <= numSnapshotRequests.get()); + Assertions.assertTrue(2 <= numSnapshotRequests.get()); } finally { cluster.shutdown(); } diff --git a/ratis-server/src/test/java/org/apache/ratis/LogAppenderTests.java b/ratis-server/src/test/java/org/apache/ratis/LogAppenderTests.java index ce08e4a74f..c7a7849e6a 100644 --- a/ratis-server/src/test/java/org/apache/ratis/LogAppenderTests.java +++ b/ratis-server/src/test/java/org/apache/ratis/LogAppenderTests.java @@ -18,7 +18,7 @@ package org.apache.ratis; import static org.apache.ratis.RaftTestUtil.waitForLeader; -import static org.junit.Assert.assertTrue; +import static org.junit.jupiter.api.Assertions.assertTrue; import org.apache.ratis.RaftTestUtil.SimpleMessage; import org.apache.ratis.client.RaftClient; @@ -40,8 +40,8 @@ import org.apache.ratis.util.JavaUtils; import org.apache.ratis.util.Slf4jUtils; import org.apache.ratis.util.SizeInBytes; -import org.junit.Assert; -import org.junit.Test; +import org.junit.jupiter.api.Assertions; +import org.junit.jupiter.api.Test; import java.io.IOException; import java.util.ArrayList; @@ -199,7 +199,7 @@ void runTest(CLUSTER cluster) throws Exception { if (e != null) { throw e; } - Assert.assertTrue(s.succeed.get()); + Assertions.assertTrue(s.succeed.get()); } } finally { for (int i = 0; i < clients.size(); i ++) { @@ -215,11 +215,11 @@ void runTest(CLUSTER cluster) throws Exception { final RaftLog leaderLog = cluster.getLeader().getRaftLog(); final EnumMap counts = RaftTestUtil.countEntries(leaderLog); LOG.info("counts = " + counts); - Assert.assertEquals(6 * numMsgs * numClients, counts.get(LogEntryBodyCase.STATEMACHINELOGENTRY).get()); + Assertions.assertEquals(6 * numMsgs * numClients, counts.get(LogEntryBodyCase.STATEMACHINELOGENTRY).get()); final LogEntryProto last = RaftTestUtil.getLastEntry(LogEntryBodyCase.STATEMACHINELOGENTRY, leaderLog); LOG.info("last = {}", LogProtoUtils.toLogEntryString(last)); - Assert.assertNotNull(last); - Assert.assertTrue(last.getIndex() <= leader.getInfo().getLastAppliedIndex()); + Assertions.assertNotNull(last); + Assertions.assertTrue(last.getIndex() <= leader.getInfo().getLastAppliedIndex()); } } diff --git a/ratis-server/src/test/java/org/apache/ratis/MessageStreamApiTests.java b/ratis-server/src/test/java/org/apache/ratis/MessageStreamApiTests.java index 8166115738..50f59b2d20 100644 --- a/ratis-server/src/test/java/org/apache/ratis/MessageStreamApiTests.java +++ b/ratis-server/src/test/java/org/apache/ratis/MessageStreamApiTests.java @@ -31,8 +31,8 @@ import org.apache.ratis.thirdparty.com.google.protobuf.ByteString; import org.apache.ratis.util.Slf4jUtils; import org.apache.ratis.util.SizeInBytes; -import org.junit.Assert; -import org.junit.Test; +import org.junit.jupiter.api.Assertions; +import org.junit.jupiter.api.Test; import org.slf4j.event.Level; import java.nio.charset.StandardCharsets; @@ -75,11 +75,11 @@ void runTestStream(CLUSTER cluster) throws Exception { try(RaftClient client = cluster.createClient()) { final String k1 = k.substring(0, endOfRequest); final RaftClientReply r1= client.io().sendReadOnly(new SimpleMessage(k1)); - Assert.assertTrue(r1.isSuccess()); + Assertions.assertTrue(r1.isSuccess()); final String k2 = k.substring(endOfRequest); final RaftClientReply r2 = client.io().sendReadOnly(new SimpleMessage(k2)); - Assert.assertTrue(r2.isSuccess()); + Assertions.assertTrue(r2.isSuccess()); } } @@ -104,24 +104,24 @@ void runTestStreamAsync(CLUSTER cluster) throws Exception { final String s = (char)('A' + i) + "1234567"; LOG.info("s=" + s); final ByteString b = ByteString.copyFrom(s, StandardCharsets.UTF_8); - Assert.assertEquals(8, b.size()); + Assertions.assertEquals(8, b.size()); for(int j = 0; j < 128; j++) { bytes = bytes.concat(b); } i++; - Assert.assertEquals(i*SUBMESSAGE_SIZE.getSizeInt(), bytes.size()); + Assertions.assertEquals(i*SUBMESSAGE_SIZE.getSizeInt(), bytes.size()); } try(RaftClient client = cluster.createClient()) { final RaftClientReply reply = client.getMessageStreamApi().streamAsync(Message.valueOf(bytes)).get(); - Assert.assertTrue(reply.isSuccess()); + Assertions.assertTrue(reply.isSuccess()); } // check if all the parts are streamed as a single message. try(RaftClient client = cluster.createClient()) { final RaftClientReply reply = client.io().sendReadOnly(new SimpleMessage(bytes.toString(StandardCharsets.UTF_8))); - Assert.assertTrue(reply.isSuccess()); + Assertions.assertTrue(reply.isSuccess()); } } } diff --git a/ratis-server/src/test/java/org/apache/ratis/OutputStreamBaseTest.java b/ratis-server/src/test/java/org/apache/ratis/OutputStreamBaseTest.java index 0c1a5164fc..a17cdb0d58 100644 --- a/ratis-server/src/test/java/org/apache/ratis/OutputStreamBaseTest.java +++ b/ratis-server/src/test/java/org/apache/ratis/OutputStreamBaseTest.java @@ -26,8 +26,8 @@ import org.apache.ratis.server.raftlog.RaftLog; import org.apache.ratis.util.SizeInBytes; import org.apache.ratis.util.StringUtils; -import org.junit.Assert; -import org.junit.Test; +import org.junit.jupiter.api.Assertions; +import org.junit.jupiter.api.Test; import java.io.IOException; import java.io.OutputStream; @@ -47,7 +47,7 @@ import java.util.function.Supplier; import static org.apache.ratis.RaftTestUtil.waitForLeader; -import static org.junit.Assert.fail; +import static org.junit.jupiter.api.Assertions.fail; public abstract class OutputStreamBaseTest extends BaseTest @@ -92,7 +92,7 @@ private void runTestSimpleWrite(CLUSTER cluster) throws Exception { private void checkLog(RaftLog raftLog, long expectedCommittedIndex, Supplier s) throws IOException { long committedIndex = raftLog.getLastCommittedIndex(); - Assert.assertTrue(committedIndex >= expectedCommittedIndex); + Assertions.assertTrue(committedIndex >= expectedCommittedIndex); // check the log content final LogEntryHeader[] entries = raftLog.getEntries(0, Long.MAX_VALUE); int count = 0; @@ -106,10 +106,10 @@ private void checkLog(RaftLog raftLog, long expectedCommittedIndex, final String message = "log " + entry + " " + log.getLogEntryBodyCase() + " " + StringUtils.bytes2HexString(logData) + ", expected=" + StringUtils.bytes2HexString(expected); - Assert.assertArrayEquals(message, expected, logData); + Assertions.assertArrayEquals(expected, logData, message); count++; } - Assert.assertEquals(expectedCommittedIndex, count); + Assertions.assertEquals(expectedCommittedIndex, count); } @Test @@ -155,12 +155,12 @@ private void runTestWriteAndFlush(CLUSTER cluster) throws Exception { private RaftLog assertRaftLog(int expectedEntries, RaftServer.Division server) throws Exception { final RaftLog raftLog = server.getRaftLog(); final EnumMap counts = RaftTestUtil.countEntries(raftLog); - Assert.assertEquals(expectedEntries, counts.get(LogEntryBodyCase.STATEMACHINELOGENTRY).get()); + Assertions.assertEquals(expectedEntries, counts.get(LogEntryBodyCase.STATEMACHINELOGENTRY).get()); final LogEntryProto last = RaftTestUtil.getLastEntry(LogEntryBodyCase.STATEMACHINELOGENTRY, raftLog); - Assert.assertNotNull(last); - Assert.assertTrue(raftLog.getLastCommittedIndex() >= last.getIndex()); - Assert.assertTrue(server.getInfo().getLastAppliedIndex() >= last.getIndex()); + Assertions.assertNotNull(last); + Assertions.assertTrue(raftLog.getLastCommittedIndex() >= last.getIndex()); + Assertions.assertTrue(server.getInfo().getLastAppliedIndex() >= last.getIndex()); return raftLog; } @@ -249,12 +249,12 @@ private void runTestWriteWithOffset(CLUSTER cluster) throws Exception { final LogEntryProto e = raftLog.get(ti.getIndex()); if (e.hasStateMachineLogEntry()) { final byte[] eValue = e.getStateMachineLogEntry().getLogData().toByteArray(); - Assert.assertEquals(ByteValue.BUFFERSIZE, eValue.length); + Assertions.assertEquals(ByteValue.BUFFERSIZE, eValue.length); System.arraycopy(eValue, 0, actual, totalSize, eValue.length); totalSize += eValue.length; } } - Assert.assertArrayEquals(expected, actual); + Assertions.assertArrayEquals(expected, actual); } /** @@ -296,18 +296,18 @@ private void runTestKillLeader(CLUSTER cluster) throws Exception { Thread.sleep(500); RaftTestUtil.waitAndKillLeader(cluster); final RaftServer.Division newLeader = waitForLeader(cluster); - Assert.assertNotEquals(leader.getId(), newLeader.getId()); + Assertions.assertNotEquals(leader.getId(), newLeader.getId()); Thread.sleep(500); running.set(false); latch.await(5, TimeUnit.SECONDS); LOG.info("Writer success? " + success.get()); - Assert.assertTrue(success.get()); + Assertions.assertTrue(success.get()); // total number of tx should be >= result + 2, where 2 means two NoOp from // leaders. It may be larger than result+2 because the client may resend // requests and we do not have retry cache on servers yet. LOG.info("last applied index: {}. total number of requests: {}", newLeader.getInfo().getLastAppliedIndex(), result.get()); - Assert.assertTrue(newLeader.getInfo().getLastAppliedIndex() >= result.get() + 1); + Assertions.assertTrue(newLeader.getInfo().getLastAppliedIndex() >= result.get() + 1); } } diff --git a/ratis-server/src/test/java/org/apache/ratis/RaftAsyncExceptionTests.java b/ratis-server/src/test/java/org/apache/ratis/RaftAsyncExceptionTests.java index 33e0857338..044ddc3422 100644 --- a/ratis-server/src/test/java/org/apache/ratis/RaftAsyncExceptionTests.java +++ b/ratis-server/src/test/java/org/apache/ratis/RaftAsyncExceptionTests.java @@ -30,8 +30,8 @@ import org.apache.ratis.statemachine.impl.SimpleStateMachine4Testing; import org.apache.ratis.statemachine.StateMachine; import org.apache.ratis.util.Slf4jUtils; -import org.junit.Assert; -import org.junit.Test; +import org.junit.jupiter.api.Assertions; +import org.junit.jupiter.api.Test; import org.slf4j.event.Level; import java.util.ArrayList; @@ -62,13 +62,13 @@ private void runTestGroupMismatchException(CLUSTER cluster) throws Exception { // send a message to make sure the cluster is working try(RaftClient client = cluster.createClient()) { final RaftClientReply reply = client.async().send(new SimpleMessage("first")).get(); - Assert.assertTrue(reply.isSuccess()); + Assertions.assertTrue(reply.isSuccess()); } // create another group final RaftGroup clusterGroup = cluster.getGroup(); final RaftGroup anotherGroup = RaftGroup.valueOf(RaftGroupId.randomId(), clusterGroup.getPeers()); - Assert.assertNotEquals(clusterGroup.getGroupId(), anotherGroup.getGroupId()); + Assertions.assertNotEquals(clusterGroup.getGroupId(), anotherGroup.getGroupId()); // create another client using another group final SimpleMessage[] messages = SimpleMessage.create(5); @@ -78,7 +78,7 @@ private void runTestGroupMismatchException(CLUSTER cluster) throws Exception { for(SimpleMessage m : messages) { futures.add(client.async().send(m)); } - Assert.assertEquals(messages.length, futures.size()); + Assertions.assertEquals(messages.length, futures.size()); // check replies final Iterator> i = futures.iterator(); @@ -102,7 +102,7 @@ private void runTestTimeoutException(CLUSTER cluster) throws Exception { // send a message to make sure the cluster is working try(RaftClient client = cluster.createClient()) { final RaftClientReply reply = client.io().send(new SimpleMessage("m0")); - Assert.assertTrue(reply.isSuccess()); + Assertions.assertTrue(reply.isSuccess()); RaftClientConfigKeys.Rpc.setRequestTimeout(PROPERTIES.get(), ONE_SECOND); // Block StartTransaction @@ -118,7 +118,7 @@ private void runTestTimeoutException(CLUSTER cluster) throws Exception { .map(SimpleStateMachine4Testing::get) .forEach(SimpleStateMachine4Testing::unblockStartTransaction); // The request should succeed after start transaction is unblocked - Assert.assertTrue(replyFuture.get(FIVE_SECONDS.getDuration(), FIVE_SECONDS.getUnit()).isSuccess()); + Assertions.assertTrue(replyFuture.get(FIVE_SECONDS.getDuration(), FIVE_SECONDS.getUnit()).isSuccess()); } } } diff --git a/ratis-server/src/test/java/org/apache/ratis/RaftAsyncTests.java b/ratis-server/src/test/java/org/apache/ratis/RaftAsyncTests.java index a1c16df8f2..4119bea71f 100644 --- a/ratis-server/src/test/java/org/apache/ratis/RaftAsyncTests.java +++ b/ratis-server/src/test/java/org/apache/ratis/RaftAsyncTests.java @@ -51,8 +51,9 @@ import org.apache.ratis.util.Slf4jUtils; import org.apache.ratis.util.TimeDuration; import org.apache.ratis.util.function.CheckedRunnable; -import org.junit.Assert; -import org.junit.Test; +import org.junit.jupiter.api.Assertions; +import org.junit.jupiter.api.Test; +import org.junit.jupiter.api.Timeout; import org.slf4j.event.Level; import java.io.IOException; @@ -70,6 +71,7 @@ import static org.apache.ratis.RaftTestUtil.waitForLeader; +@Timeout(100) public abstract class RaftAsyncTests extends BaseTest implements MiniRaftCluster.Factory.Get { { @@ -112,9 +114,9 @@ public void testAsyncConfiguration() throws IOException { } static void assertRaftRetryFailureException(RaftRetryFailureException rfe, RetryPolicy retryPolicy, String name) { - Assert.assertNotNull(name + " does not have RaftRetryFailureException", rfe); - Assert.assertTrue(name + ": unexpected error message, rfe=" + rfe + ", retryPolicy=" + retryPolicy, - rfe.getMessage().contains(retryPolicy.toString())); + Assertions.assertNotNull(rfe, name + " does not have RaftRetryFailureException"); + Assertions.assertTrue(rfe.getMessage().contains(retryPolicy.toString()), + name + ": unexpected error message, rfe=" + rfe + ", retryPolicy=" + retryPolicy); } @Test @@ -172,7 +174,7 @@ void runTestRequestAsyncWithRetryFailure(boolean initialMessages, CLUSTER cluste for (; i < messages.length; i++) { replies.add(client.async().send(messages[i])); } - Assert.assertEquals(messages.length, replies.size()); + Assertions.assertEquals(messages.length, replies.size()); } // sleep again so that the first half calls will fail retries. @@ -229,7 +231,7 @@ void runTestAsyncRequestSemaphore(CLUSTER cluster) throws Exception { futures[i] = client.async().send(messages[i]); blockedRequestsCount.decrementAndGet(); } - Assert.assertEquals(0, blockedRequestsCount.get()); + Assertions.assertEquals(0, blockedRequestsCount.get()); futures[numMessages] = CompletableFuture.supplyAsync(() -> { blockedRequestsCount.incrementAndGet(); @@ -242,7 +244,7 @@ void runTestAsyncRequestSemaphore(CLUSTER cluster) throws Exception { while (blockedRequestsCount.get() != 1) { Thread.sleep(1000); } - Assert.assertEquals(1, blockedRequestsCount.get()); + Assertions.assertEquals(1, blockedRequestsCount.get()); //Since all semaphore permits are acquired the last message sent is in queue RaftClientTestUtil.assertAsyncRequestSemaphore(client, 0, 1); @@ -255,7 +257,7 @@ void runTestAsyncRequestSemaphore(CLUSTER cluster) throws Exception { for (int i = 0; i <= numMessages; i++) { futures[i].join(); } - Assert.assertEquals(0, blockedRequestsCount.get()); + Assertions.assertEquals(0, blockedRequestsCount.get()); } } @@ -297,11 +299,11 @@ void runTestStaleReadAsync(CLUSTER cluster) throws Exception { LOG.info("sendAsync " + s); futures.add(client.async().send(new SimpleMessage(s))); } - Assert.assertEquals(numMessages, futures.size()); + Assertions.assertEquals(numMessages, futures.size()); final List replies = new ArrayList<>(); for (CompletableFuture f : futures) { final RaftClientReply r = f.join(); - Assert.assertTrue(r.isSuccess()); + Assertions.assertTrue(r.isSuccess()); replies.add(r); } futures.clear(); @@ -309,7 +311,7 @@ void runTestStaleReadAsync(CLUSTER cluster) throws Exception { // Use a follower with the max commit index final RaftClientReply lastWriteReply = replies.get(replies.size() - 1); final RaftPeerId leader = lastWriteReply.getServerId(); - Assert.assertEquals(leader, lastWriteReply.getServerId()); + Assertions.assertEquals(leader, lastWriteReply.getServerId()); LOG.info("leader = " + leader); final Collection commitInfos = lastWriteReply.getCommitInfos(); LOG.info("commitInfos = " + commitInfos); @@ -356,7 +358,7 @@ void runTestStaleReadAsync(CLUSTER cluster) throws Exception { throw new CompletionException(e); } - Assert.assertEquals("log entry mismatch for query=" + query, expected, computed); + Assertions.assertEquals(expected, computed, "log entry mismatch for query=" + query); return null; })); } @@ -383,14 +385,14 @@ void runTestWriteAsyncCustomReplicationLevel(CLUSTER cluster) throws Exception { if (exception != null) { LOG.error("Failed to send message " + s, exception); // reply should be null in case of exception - Assert.assertNull(reply); + Assertions.assertNull(reply); return; } - Assert.assertTrue(reply.isSuccess()); - Assert.assertNull(reply.getException()); + Assertions.assertTrue(reply.isSuccess()); + Assertions.assertNull(reply.getException()); // verify that all servers have caught up to log index when the reply is returned reply.getCommitInfos().forEach(commitInfoProto -> - Assert.assertTrue(commitInfoProto.getCommitIndex() >= reply.getLogIndex())); + Assertions.assertTrue(commitInfoProto.getCommitIndex() >= reply.getLogIndex())); }); } } @@ -436,15 +438,15 @@ void runTestAppendEntriesTimeout(CLUSTER cluster) throws Exception { CompletableFuture replyFuture = client.async().send(new SimpleMessage("abc")); Thread.sleep(waitTime); // replyFuture should not be completed until append request is unblocked. - Assert.assertFalse(replyFuture.isDone()); + Assertions.assertFalse(replyFuture.isDone()); // unblock append request. cluster.getServerAliveStream() .filter(impl -> !impl.getInfo().isLeader() && !impl.getPeer().getId().equals(leader)) .map(SimpleStateMachine4Testing::get) .forEach(SimpleStateMachine4Testing::unblockWriteStateMachineData); - Assert.assertTrue(replyFuture.get().isSuccess()); - Assert.assertTrue(System.currentTimeMillis() - time > waitTime); + Assertions.assertTrue(replyFuture.get().isSuccess()); + Assertions.assertTrue(System.currentTimeMillis() - time > waitTime); } //reset for the other tests @@ -481,7 +483,7 @@ void runTestCheckLeadershipFailure(CLUSTER cluster) throws Exception { // previous leader should not there. cluster.getServerAliveStream() .map(RaftServer.Division::getInfo) - .forEach(info -> Assert.assertTrue(!info.isLeader() || info.getCurrentTerm() > termOfPrevLeader)); + .forEach(info -> Assertions.assertTrue(!info.isLeader() || info.getCurrentTerm() > termOfPrevLeader)); } finally { // unblock append entries request @@ -494,7 +496,7 @@ void runTestCheckLeadershipFailure(CLUSTER cluster) throws Exception { LOG.info("Current Leader is elected on term {}", termOfCurrLeader); // leader on termOfPrevLeader should step-down. - Assert.assertTrue(termOfPrevLeader < termOfCurrLeader); + Assertions.assertTrue(termOfPrevLeader < termOfCurrLeader); } @Test @@ -507,10 +509,10 @@ public void testNoRetryWaitOnNotLeaderException() throws Exception { private void runTestNoRetryWaitOnNotLeaderException(MiniRaftCluster cluster) throws Exception { final RaftServer.Division leader = waitForLeader(cluster); final List followers = cluster.getFollowers(); - Assert.assertNotNull(followers); - Assert.assertEquals(2, followers.size()); - Assert.assertNotSame(leader, followers.get(0)); - Assert.assertNotSame(leader, followers.get(1)); + Assertions.assertNotNull(followers); + Assertions.assertEquals(2, followers.size()); + Assertions.assertNotSame(leader, followers.get(0)); + Assertions.assertNotSame(leader, followers.get(1)); // send a message to make sure that the leader is ready try (final RaftClient client = cluster.createClient(leader.getId())) { diff --git a/ratis-server/src/test/java/org/apache/ratis/RaftExceptionBaseTest.java b/ratis-server/src/test/java/org/apache/ratis/RaftExceptionBaseTest.java index 9b6d811211..3d2837cd6b 100644 --- a/ratis-server/src/test/java/org/apache/ratis/RaftExceptionBaseTest.java +++ b/ratis-server/src/test/java/org/apache/ratis/RaftExceptionBaseTest.java @@ -33,9 +33,9 @@ import org.apache.ratis.util.JavaUtils; import org.apache.ratis.util.Slf4jUtils; import org.apache.ratis.util.SizeInBytes; -import org.junit.Assert; +import org.junit.jupiter.api.Assertions; import org.junit.Assume; -import org.junit.Test; +import org.junit.jupiter.api.Test; import org.slf4j.event.Level; import java.io.IOException; @@ -83,17 +83,17 @@ RaftClientReply assertNotLeaderException(RaftPeerId expectedSuggestedLeader, String messageId, RaftPeerId server, RaftClientRpc rpc, CLUSTER cluster) throws IOException { final SimpleMessage message = new SimpleMessage(messageId); final RaftClientReply reply = rpc.sendRequest(cluster.newRaftClientRequest(ClientId.randomId(), server, message)); - Assert.assertNotNull(reply); + Assertions.assertNotNull(reply); Assume.assumeFalse(reply.isSuccess()); final NotLeaderException nle = reply.getNotLeaderException(); Objects.requireNonNull(nle); - Assert.assertEquals(expectedSuggestedLeader, nle.getSuggestedLeader().getId()); + Assertions.assertEquals(expectedSuggestedLeader, nle.getSuggestedLeader().getId()); return reply; } static void sendMessage(String message, RaftClient client) throws IOException { final RaftClientReply reply = client.io().send(new SimpleMessage(message)); - Assert.assertTrue(reply.isSuccess()); + Assertions.assertTrue(reply.isSuccess()); } @Test @@ -115,7 +115,7 @@ void runTestNotLeaderExceptionWithReconf(CLUSTER cluster) throws Exception { LOG.info("Start changing the configuration: {}", Arrays.asList(change.allPeersInNewConf)); try (final RaftClient c2 = cluster.createClient(newLeader)) { RaftClientReply reply = c2.admin().setConfiguration(change.allPeersInNewConf); - Assert.assertTrue(reply.isSuccess()); + Assertions.assertTrue(reply.isSuccess()); } LOG.info(cluster.printServers()); @@ -127,9 +127,9 @@ void runTestNotLeaderExceptionWithReconf(CLUSTER cluster) throws Exception { final Collection peers = cluster.getPeers(); final Collection peersFromReply = reply.getNotLeaderException().getPeers(); - Assert.assertEquals(peers.size(), peersFromReply.size()); + Assertions.assertEquals(peers.size(), peersFromReply.size()); for (RaftPeer p : peersFromReply) { - Assert.assertTrue(peers.contains(p)); + Assertions.assertTrue(peers.contains(p)); } sendMessage("m2", client); @@ -143,10 +143,10 @@ public void testGroupMismatchException() throws Exception { void runTestGroupMismatchException(CLUSTER cluster) throws Exception { final RaftGroup clusterGroup = cluster.getGroup(); - Assert.assertEquals(NUM_PEERS, clusterGroup.getPeers().size()); + Assertions.assertEquals(NUM_PEERS, clusterGroup.getPeers().size()); final RaftGroup anotherGroup = RaftGroup.valueOf(RaftGroupId.randomId(), clusterGroup.getPeers()); - Assert.assertNotEquals(clusterGroup.getGroupId(), anotherGroup.getGroupId()); + Assertions.assertNotEquals(clusterGroup.getGroupId(), anotherGroup.getGroupId()); // Create client using another group try(RaftClient client = cluster.createClient(anotherGroup)) { diff --git a/ratis-server/src/test/java/org/apache/ratis/RaftTestUtil.java b/ratis-server/src/test/java/org/apache/ratis/RaftTestUtil.java index 58dedaf8c6..f8d92cf08a 100644 --- a/ratis-server/src/test/java/org/apache/ratis/RaftTestUtil.java +++ b/ratis-server/src/test/java/org/apache/ratis/RaftTestUtil.java @@ -43,7 +43,6 @@ import org.apache.ratis.util.Preconditions; import org.apache.ratis.util.ProtoUtils; import org.apache.ratis.util.TimeDuration; -import org.junit.Assert; import org.junit.AssumptionViolatedException; import org.junit.jupiter.api.Assertions; import org.slf4j.Logger; @@ -142,7 +141,7 @@ static RaftServer.Division waitForLeader(MiniRaftCluster cluster, RaftGroupId gr static RaftPeerId waitAndKillLeader(MiniRaftCluster cluster) throws InterruptedException { final RaftServer.Division leader = waitForLeader(cluster); - Assert.assertNotNull(leader); + Assertions.assertNotNull(leader); LOG.info("killing leader = " + leader); cluster.killServer(leader.getId()); @@ -229,7 +228,7 @@ static void checkLogEntries(RaftLog log, SimpleMessage[] expectedMessages, e = log.get(termIndices[i].getIndex()); if (Arrays.equals(expectedMessages[j].getContent().toByteArray(), e.getStateMachineLogEntry().getLogData().toByteArray())) { - Assert.assertTrue(predicate.test(e)); + Assertions.assertTrue(predicate.test(e)); } } catch (IOException exception) { exception.printStackTrace(); @@ -342,16 +341,16 @@ static Void assertLogEntries(RaftLog log, long expectedTerm, SimpleMessage[] exp static void assertLogEntries(List entries, long expectedTerm, SimpleMessage... expectedMessages) { long logIndex = 0; - Assert.assertEquals(expectedMessages.length, entries.size()); + Assertions.assertEquals(expectedMessages.length, entries.size()); for (int i = 0; i < expectedMessages.length; i++) { final LogEntryProto e = entries.get(i); - Assert.assertTrue(e.getTerm() >= expectedTerm); + Assertions.assertTrue(e.getTerm() >= expectedTerm); if (e.getTerm() > expectedTerm) { expectedTerm = e.getTerm(); } - Assert.assertTrue(e.getIndex() > logIndex); + Assertions.assertTrue(e.getIndex() > logIndex); logIndex = e.getIndex(); - Assert.assertEquals(expectedMessages[i].getContent(), e.getStateMachineLogEntry().getLogData()); + Assertions.assertEquals(expectedMessages[i].getContent(), e.getStateMachineLogEntry().getLogData()); } } @@ -553,11 +552,11 @@ static Thread sendMessageInNewThread(MiniRaftCluster cluster, RaftPeerId leaderI } static void assertSameLog(RaftLog expected, RaftLog computed) throws Exception { - Assert.assertEquals(expected.getLastEntryTermIndex(), computed.getLastEntryTermIndex()); + Assertions.assertEquals(expected.getLastEntryTermIndex(), computed.getLastEntryTermIndex()); final long lastIndex = expected.getNextIndex() - 1; - Assert.assertEquals(expected.getLastEntryTermIndex().getIndex(), lastIndex); + Assertions.assertEquals(expected.getLastEntryTermIndex().getIndex(), lastIndex); for(long i = 0; i < lastIndex; i++) { - Assert.assertEquals(expected.get(i), computed.get(i)); + Assertions.assertEquals(expected.get(i), computed.get(i)); } } @@ -588,8 +587,8 @@ static void assertSuccessReply(CompletableFuture reply) throws } static void assertSuccessReply(RaftClientReply reply) { - Assert.assertNotNull("reply == null", reply); - Assert.assertTrue("reply is not success: " + reply, reply.isSuccess()); + Assertions.assertNotNull(reply, "reply == null"); + Assertions.assertTrue(reply.isSuccess(), "reply is not success: " + reply); } static void gc() throws InterruptedException { diff --git a/ratis-server/src/test/java/org/apache/ratis/ReadOnlyRequestTests.java b/ratis-server/src/test/java/org/apache/ratis/ReadOnlyRequestTests.java index eea75592ef..17fd28cbe6 100644 --- a/ratis-server/src/test/java/org/apache/ratis/ReadOnlyRequestTests.java +++ b/ratis-server/src/test/java/org/apache/ratis/ReadOnlyRequestTests.java @@ -35,9 +35,9 @@ import org.apache.ratis.statemachine.impl.BaseStateMachine; import org.apache.ratis.util.Slf4jUtils; import org.apache.ratis.util.TimeDuration; -import org.junit.Assert; -import org.junit.Before; -import org.junit.Test; +import org.junit.jupiter.api.Assertions; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.Test; import org.slf4j.event.Level; import java.nio.charset.StandardCharsets; @@ -63,7 +63,7 @@ public abstract class ReadOnlyRequestTests final Message waitAndIncrementMessage = new RaftTestUtil.SimpleMessage(WAIT_AND_INCREMENT); final Message queryMessage = new RaftTestUtil.SimpleMessage(QUERY); - @Before + @BeforeEach public void setup() { final RaftProperties p = getProperties(); p.setClass(MiniRaftCluster.STATEMACHINE_CLASS_KEY, @@ -90,9 +90,9 @@ private void testReadOnlyImpl(CLUSTER cluster) throws Exception { try (final RaftClient client = cluster.createClient(leaderId)) { for (int i = 1; i <= 10; i++) { RaftClientReply reply = client.io().send(incrementMessage); - Assert.assertTrue(reply.isSuccess()); + Assertions.assertTrue(reply.isSuccess()); reply = client.io().sendReadOnly(queryMessage); - Assert.assertEquals(i, retrieve(reply)); + Assertions.assertEquals(i, retrieve(reply)); } } } finally { @@ -123,10 +123,10 @@ private void testReadOnlyTimeoutImpl(CLUSTER cluster) throws Exception { CompletableFuture result = client.async().send(incrementMessage); client.admin().transferLeadership(null, 200); - Assert.assertThrows(ReadIndexException.class, () -> { + Assertions.assertThrows(ReadIndexException.class, () -> { RaftClientReply timeoutReply = noRetry.io().sendReadOnly(queryMessage); - Assert.assertNotNull(timeoutReply.getException()); - Assert.assertTrue(timeoutReply.getException() instanceof ReadException); + Assertions.assertNotNull(timeoutReply.getException()); + Assertions.assertTrue(timeoutReply.getException() instanceof ReadException); }); } @@ -152,18 +152,18 @@ private void testFollowerReadOnlyImpl(CLUSTER cluster) throws Exception { RaftTestUtil.waitForLeader(cluster); List followers = cluster.getFollowers(); - Assert.assertEquals(2, followers.size()); + Assertions.assertEquals(2, followers.size()); final RaftPeerId f0 = followers.get(0).getId(); final RaftPeerId f1 = followers.get(1).getId(); try (RaftClient client = cluster.createClient(cluster.getLeader().getId())) { for (int i = 1; i <= 10; i++) { final RaftClientReply reply = client.io().send(incrementMessage); - Assert.assertTrue(reply.isSuccess()); + Assertions.assertTrue(reply.isSuccess()); final RaftClientReply read1 = client.io().sendReadOnly(queryMessage, f0); - Assert.assertEquals(i, retrieve(read1)); + Assertions.assertEquals(i, retrieve(read1)); final CompletableFuture read2 = client.async().sendReadOnly(queryMessage, f1); - Assert.assertEquals(i, retrieve(read2.get(1, TimeUnit.SECONDS))); + Assertions.assertEquals(i, retrieve(read2.get(1, TimeUnit.SECONDS))); } } } finally { @@ -188,7 +188,7 @@ private void testFollowerReadOnlyParallelImpl(CLUSTER cluster) throws Exception RaftTestUtil.waitForLeader(cluster); List followers = cluster.getFollowers(); - Assert.assertEquals(2, followers.size()); + Assertions.assertEquals(2, followers.size()); try (RaftClient leaderClient = cluster.createClient(cluster.getLeader().getId()); RaftClient followerClient1 = cluster.createClient(followers.get(0).getId())) { @@ -198,7 +198,7 @@ private void testFollowerReadOnlyParallelImpl(CLUSTER cluster) throws Exception Thread.sleep(100); RaftClientReply clientReply = followerClient1.io().sendReadOnly(queryMessage, followers.get(0).getId()); - Assert.assertEquals(2, retrieve(clientReply)); + Assertions.assertEquals(2, retrieve(clientReply)); } } finally { @@ -223,20 +223,20 @@ private void testFollowerReadOnlyFailWhenLeaderDownImpl(CLUSTER cluster) throws RaftTestUtil.waitForLeader(cluster); List followers = cluster.getFollowers(); - Assert.assertEquals(2, followers.size()); + Assertions.assertEquals(2, followers.size()); try (RaftClient leaderClient = cluster.createClient(cluster.getLeader().getId()); RaftClient followerClient1 = cluster.createClient(followers.get(0).getId(), RetryPolicies.noRetry())) { leaderClient.io().send(incrementMessage); RaftClientReply clientReply = followerClient1.io().sendReadOnly(queryMessage); - Assert.assertEquals(1, retrieve(clientReply)); + Assertions.assertEquals(1, retrieve(clientReply)); // kill the leader // read timeout quicker than election timeout leaderClient.admin().transferLeadership(null, 200); - Assert.assertThrows(ReadIndexException.class, () -> { + Assertions.assertThrows(ReadIndexException.class, () -> { followerClient1.io().sendReadOnly(queryMessage, followers.get(0).getId()); }); } @@ -273,14 +273,14 @@ private void testFollowerReadOnlyRetryWhenLeaderDown(CLUSTER cluster) throws Exc client.io().send(incrementMessage); final RaftClientReply clientReply = client.io().sendReadOnly(queryMessage); - Assert.assertEquals(1, retrieve(clientReply)); + Assertions.assertEquals(1, retrieve(clientReply)); // kill the leader client.admin().transferLeadership(null, 200); // readOnly will success after re-election final RaftClientReply replySuccess = client.io().sendReadOnly(queryMessage); - Assert.assertEquals(1, retrieve(clientReply)); + Assertions.assertEquals(1, retrieve(clientReply)); } } @@ -295,12 +295,12 @@ private void testReadAfterWriteImpl(CLUSTER cluster) throws Exception { // test blocking read-after-write client.io().send(incrementMessage); final RaftClientReply blockReply = client.io().sendReadAfterWrite(queryMessage); - Assert.assertEquals(1, retrieve(blockReply)); + Assertions.assertEquals(1, retrieve(blockReply)); // test asynchronous read-after-write client.async().send(incrementMessage); client.async().sendReadAfterWrite(queryMessage).thenAccept(reply -> { - Assert.assertEquals(2, retrieve(reply)); + Assertions.assertEquals(2, retrieve(reply)); }); for (int i = 0; i < 20; i++) { @@ -311,7 +311,7 @@ private void testReadAfterWriteImpl(CLUSTER cluster) throws Exception { CompletableFuture.allOf(linearizable, readAfterWrite).get(); // read-after-write is more consistent than linearizable read - Assert.assertTrue(retrieve(readAfterWrite.get()) >= retrieve(linearizable.get())); + Assertions.assertTrue(retrieve(readAfterWrite.get()) >= retrieve(linearizable.get())); } } diff --git a/ratis-server/src/test/java/org/apache/ratis/ReadOnlyRequestWithLongTimeoutTests.java b/ratis-server/src/test/java/org/apache/ratis/ReadOnlyRequestWithLongTimeoutTests.java index 14aa1bca26..4a6498f939 100644 --- a/ratis-server/src/test/java/org/apache/ratis/ReadOnlyRequestWithLongTimeoutTests.java +++ b/ratis-server/src/test/java/org/apache/ratis/ReadOnlyRequestWithLongTimeoutTests.java @@ -32,9 +32,9 @@ import org.apache.ratis.statemachine.StateMachine; import org.apache.ratis.util.Slf4jUtils; import org.apache.ratis.util.TimeDuration; -import org.junit.Assert; -import org.junit.Before; -import org.junit.Test; +import org.junit.jupiter.api.Assertions; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.Test; import org.slf4j.event.Level; import java.util.concurrent.CompletableFuture; @@ -59,7 +59,7 @@ public abstract class ReadOnlyRequestWithLongTimeoutTests asyncTimeoutReply = client.async().send(timeoutMessage); Thread.sleep(100); - Assert.assertThrows(ReadException.class, () -> { + Assertions.assertThrows(ReadException.class, () -> { final RaftClientReply timeoutReply = client.io().sendReadOnly(queryMessage); - Assert.assertTrue(timeoutReply.getException().getCause() instanceof TimeoutIOException); + Assertions.assertTrue(timeoutReply.getException().getCause() instanceof TimeoutIOException); }); asyncTimeoutReply.join(); diff --git a/ratis-server/src/test/java/org/apache/ratis/RequestLimitAsyncBaseTest.java b/ratis-server/src/test/java/org/apache/ratis/RequestLimitAsyncBaseTest.java index 19a46a0652..ce339faff9 100644 --- a/ratis-server/src/test/java/org/apache/ratis/RequestLimitAsyncBaseTest.java +++ b/ratis-server/src/test/java/org/apache/ratis/RequestLimitAsyncBaseTest.java @@ -31,8 +31,8 @@ import org.apache.ratis.server.impl.RaftServerTestUtil; import org.apache.ratis.statemachine.impl.SimpleStateMachine4Testing; import org.apache.ratis.util.Slf4jUtils; -import org.junit.Assert; -import org.junit.Test; +import org.junit.jupiter.api.Assertions; +import org.junit.jupiter.api.Test; import org.slf4j.event.Level; import java.util.ArrayList; @@ -74,7 +74,7 @@ void runTestWriteElementLimit(CLUSTER cluster) throws Exception { final SimpleMessage message = new SimpleMessage("first"); final CompletableFuture future = c1.async().send(message); final RaftClientReply reply = getWithDefaultTimeout(future); - Assert.assertTrue(reply.isSuccess()); + Assertions.assertTrue(reply.isSuccess()); } // collecting futures returned from StateMachine.applyTransaction @@ -118,7 +118,7 @@ void runTestWriteElementLimit(CLUSTER cluster) throws Exception { // check replies for(CompletableFuture f : writeFutures) { final RaftClientReply reply = getWithDefaultTimeout(f); - Assert.assertTrue(reply.isSuccess()); + Assertions.assertTrue(reply.isSuccess()); } } } diff --git a/ratis-server/src/test/java/org/apache/ratis/TestReConfigProperty.java b/ratis-server/src/test/java/org/apache/ratis/TestReConfigProperty.java index 4535406a77..7d1aec901f 100644 --- a/ratis-server/src/test/java/org/apache/ratis/TestReConfigProperty.java +++ b/ratis-server/src/test/java/org/apache/ratis/TestReConfigProperty.java @@ -27,9 +27,10 @@ import org.apache.ratis.statemachine.StateMachine; import org.apache.ratis.statemachine.impl.SimpleStateMachine4Testing; import org.apache.ratis.util.Slf4jUtils; -import org.junit.Assert; -import org.junit.Before; -import org.junit.Test; +import org.junit.jupiter.api.Assertions; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.Test; +import org.junit.jupiter.api.Timeout; import org.slf4j.event.Level; import java.io.IOException; @@ -59,7 +60,7 @@ public abstract class TestReConfigProperty exte private static final String VAL2 = "val2"; private static final String DEFAULT = "default"; - @Before + @BeforeEach public void setup () { conf1 = new RaftProperties(); conf2 = new RaftProperties(); @@ -81,8 +82,7 @@ public void testGetChangedProperty() { Collection changes = ReconfigurationBase.getChangedProperties(conf2, conf1); - Assert.assertTrue("expected 3 changed properties but got " + changes.size(), - changes.size() == 3); + Assertions.assertEquals(3, changes.size(), "Unexpected changed size"); boolean changeFound = false; boolean unsetFound = false; @@ -100,8 +100,7 @@ public void testGetChangedProperty() { setFound = true; } } - Assert.assertTrue("not all changes have been applied", - changeFound && unsetFound && setFound); + Assertions.assertTrue(changeFound && unsetFound && setFound, "not all changes have been applied"); } /** @@ -157,161 +156,133 @@ public void run() { public void testReconfigure() { ReconfigurableDummy dummy = new ReconfigurableDummy(conf1); - Assert.assertEquals(PROP1 + " set to wrong value ", VAL1, dummy.getProperties().get(PROP1)); - Assert.assertEquals(PROP2 + " set to wrong value ", VAL1, dummy.getProperties().get(PROP2)); - Assert.assertEquals(PROP3 + " set to wrong value ", VAL1, dummy.getProperties().get(PROP3)); - Assert.assertNull(PROP4 + " set to wrong value ", dummy.getProperties().get(PROP4)); - Assert.assertNull(PROP5 + " set to wrong value ", dummy.getProperties().get(PROP5)); - - Assert.assertTrue(PROP1 + " should be reconfigurable ", - dummy.isPropertyReconfigurable(PROP1)); - Assert.assertTrue(PROP2 + " should be reconfigurable ", - dummy.isPropertyReconfigurable(PROP2)); - Assert.assertFalse(PROP3 + " should not be reconfigurable ", - dummy.isPropertyReconfigurable(PROP3)); - Assert.assertTrue(PROP4 + " should be reconfigurable ", - dummy.isPropertyReconfigurable(PROP4)); - Assert.assertFalse(PROP5 + " should not be reconfigurable ", - dummy.isPropertyReconfigurable(PROP5)); + Assertions.assertEquals(VAL1, dummy.getProperties().get(PROP1), PROP1 + " set to wrong value "); + Assertions.assertEquals(VAL1, dummy.getProperties().get(PROP2), PROP2 + " set to wrong value "); + Assertions.assertEquals(VAL1, dummy.getProperties().get(PROP3), PROP3 + " set to wrong value "); + Assertions.assertNull(dummy.getProperties().get(PROP4), PROP4 + " set to wrong value "); + Assertions.assertNull(dummy.getProperties().get(PROP5), PROP5 + " set to wrong value "); + + Assertions.assertTrue(dummy.isPropertyReconfigurable(PROP1), PROP1 + " should be reconfigurable "); + Assertions.assertTrue(dummy.isPropertyReconfigurable(PROP2), PROP2 + " should be reconfigurable "); + Assertions.assertFalse(dummy.isPropertyReconfigurable(PROP3), PROP3 + " should not be reconfigurable "); + Assertions.assertTrue(dummy.isPropertyReconfigurable(PROP4), PROP4 + " should be reconfigurable "); + Assertions.assertFalse(dummy.isPropertyReconfigurable(PROP5), PROP5 + " should not be reconfigurable "); // change something to the same value as before { - boolean exceptionCaught = false; try { dummy.reconfigureProperty(PROP1, VAL1); dummy.startReconfiguration(); RaftTestUtil.waitFor(() -> dummy.getReconfigurationStatus().ended(), 100, 60000); - Assert.assertEquals(PROP1 + " set to wrong value ", VAL1, dummy.getProperties().get(PROP1)); + Assertions.assertEquals(VAL1, dummy.getProperties().get(PROP1), PROP1 + " set to wrong value "); } catch (ReconfigurationException | IOException | TimeoutException | InterruptedException e) { - exceptionCaught = true; + Assertions.fail("Unexpected exception", e); } - Assert.assertFalse("received unexpected exception", - exceptionCaught); } // change something to null { - boolean exceptionCaught = false; try { dummy.reconfigureProperty(PROP1, null); dummy.startReconfiguration(); RaftTestUtil.waitFor(() -> dummy.getReconfigurationStatus().ended(), 100, 60000); - Assert.assertEquals(PROP1 + "set to wrong value ", DEFAULT, - dummy.getProperties().get(PROP1)); + Assertions.assertEquals(DEFAULT +, dummy.getProperties().get(PROP1), PROP1 + "set to wrong value "); } catch (ReconfigurationException | IOException | InterruptedException | TimeoutException e) { - exceptionCaught = true; + Assertions.fail("Unexpected exception", e); } - Assert.assertFalse("received unexpected exception", - exceptionCaught); } // change something to a different value than before { - boolean exceptionCaught = false; try { dummy.reconfigureProperty(PROP1, VAL2); dummy.startReconfiguration(); RaftTestUtil.waitFor(() -> dummy.getReconfigurationStatus().ended(), 100, 60000); - Assert.assertEquals(PROP1 + "set to wrong value ", VAL2, dummy.getProperties().get(PROP1)); + Assertions.assertEquals(VAL2, dummy.getProperties().get(PROP1), PROP1 + "set to wrong value "); } catch (ReconfigurationException | IOException | InterruptedException | TimeoutException e) { - exceptionCaught = true; + Assertions.fail("Unexpected exception", e); } - Assert.assertFalse("received unexpected exception", - exceptionCaught); } // set unset property to null { - boolean exceptionCaught = false; try { dummy.reconfigureProperty(PROP4, null); dummy.startReconfiguration(); RaftTestUtil.waitFor(() -> dummy.getReconfigurationStatus().ended(), 100, 60000); - Assert.assertSame(PROP4 + "set to wrong value ", DEFAULT, dummy.getProperties().get(PROP4)); + Assertions.assertSame(DEFAULT, dummy.getProperties().get(PROP4), PROP4 + "set to wrong value "); } catch (ReconfigurationException | IOException | InterruptedException | TimeoutException e) { - exceptionCaught = true; + Assertions.fail("Unexpected exception", e); } - Assert.assertFalse("received unexpected exception", - exceptionCaught); } // set unset property { - boolean exceptionCaught = false; try { dummy.reconfigureProperty(PROP4, VAL1); dummy.startReconfiguration(); RaftTestUtil.waitFor(() -> dummy.getReconfigurationStatus().ended(), 100, 60000); - Assert.assertEquals(PROP4 + "set to wrong value ", VAL1, dummy.getProperties().get(PROP4)); + Assertions.assertEquals(VAL1, dummy.getProperties().get(PROP4), PROP4 + "set to wrong value "); } catch (ReconfigurationException | IOException | InterruptedException | TimeoutException e) { - exceptionCaught = true; + Assertions.fail("Unexpected exception", e); } - Assert.assertFalse("received unexpected exception", - exceptionCaught); } // try to set unset property to null (not reconfigurable) { - boolean exceptionCaught = false; try { dummy.reconfigureProperty(PROP5, null); dummy.startReconfiguration(); RaftTestUtil.waitFor(() -> dummy.getReconfigurationStatus().ended(), 100, 60000); } catch (ReconfigurationException | IOException | InterruptedException | TimeoutException e) { - exceptionCaught = true; + Assertions.fail("Unexpected exception", e); } - Assert.assertTrue("did not receive expected exception", - dummy.getReconfigurationStatus().getChanges() + Assertions.assertTrue(dummy.getReconfigurationStatus().getChanges() .get(new PropertyChange(PROP5, DEFAULT, null)) - .getMessage().contains("Property is not reconfigurable.") && !exceptionCaught); + .getMessage().contains("Property is not reconfigurable."), "did not receive expected exception"); } // try to set unset property to value (not reconfigurable) { - boolean exceptionCaught = false; try { dummy.reconfigureProperty(PROP5, VAL1); dummy.startReconfiguration(); RaftTestUtil.waitFor(() -> dummy.getReconfigurationStatus().ended(), 100, 60000); } catch (ReconfigurationException | IOException | InterruptedException | TimeoutException e) { - exceptionCaught = true; + Assertions.fail("Unexpected exception", e); } - Assert.assertTrue("did not receive expected exception", - dummy.getReconfigurationStatus().getChanges() + Assertions.assertTrue(dummy.getReconfigurationStatus().getChanges() .get(new PropertyChange(PROP5, VAL1, null)) - .getMessage().contains("Property is not reconfigurable.") && !exceptionCaught); + .getMessage().contains("Property is not reconfigurable."), "did not receive expected exception"); } // try to change property to value (not reconfigurable) { - boolean exceptionCaught = false; try { dummy.reconfigureProperty(PROP3, VAL2); dummy.startReconfiguration(); RaftTestUtil.waitFor(() -> dummy.getReconfigurationStatus().ended(), 100, 60000); } catch (ReconfigurationException | IOException | InterruptedException | TimeoutException e) { - exceptionCaught = true; + Assertions.fail("Unexpected exception", e); } - Assert.assertTrue("did not receive expected exception", - dummy.getReconfigurationStatus().getChanges() + Assertions.assertTrue(dummy.getReconfigurationStatus().getChanges() .get(new PropertyChange(PROP3, VAL2, VAL1)) - .getMessage().contains("Property is not reconfigurable.") && !exceptionCaught); + .getMessage().contains("Property is not reconfigurable."), "did not receive expected exception"); } // try to change property to null (not reconfigurable) { - boolean exceptionCaught = false; try { dummy.reconfigureProperty(PROP3, null); dummy.startReconfiguration(); RaftTestUtil.waitFor(() -> dummy.getReconfigurationStatus().ended(), 100, 60000); } catch (ReconfigurationException | IOException | InterruptedException | TimeoutException e) { - exceptionCaught = true; + Assertions.fail("Unexpected exception", e); } - Assert.assertTrue("did not receive expected exception", - dummy.getReconfigurationStatus().getChanges() + Assertions.assertTrue(dummy.getReconfigurationStatus().getChanges() .get(new PropertyChange(PROP3, DEFAULT, VAL1)) - .getMessage().contains("Property is not reconfigurable.") && !exceptionCaught); + .getMessage().contains("Property is not reconfigurable."), "did not receive expected exception"); } } @@ -321,7 +292,7 @@ public void testReconfigure() { @Test public void testThread() throws ReconfigurationException, IOException { ReconfigurableDummy dummy = new ReconfigurableDummy(conf1); - Assert.assertEquals(VAL1, dummy.getProperties().get(PROP1)); + Assertions.assertEquals(VAL1, dummy.getProperties().get(PROP1)); Thread dummyThread = new Thread(dummy); dummyThread.start(); try { @@ -341,25 +312,23 @@ public void testThread() throws ReconfigurationException, IOException { } } - Assert.assertFalse("dummy thread should not be alive", - dummyThread.isAlive()); + Assertions.assertFalse(dummyThread.isAlive(), "dummy thread should not be alive"); dummy.running = false; try { dummyThread.join(); } catch (InterruptedException ignore) { // do nothing } - Assert.assertTrue(PROP1 + " is set to wrong value", - dummy.getProperties().get(PROP1).equals(VAL2)); + Assertions.assertEquals(VAL2, dummy.getProperties().get(PROP1), PROP1 + " is set to wrong value"); } /** * Ensure that {@link ReconfigurationBase#reconfigureProperty} updates the * parent's cached configuration on success. - * @throws IOException */ - @Test (timeout=300000) + @Test + @Timeout(value = 300) public void testConfIsUpdatedOnSuccess() throws ReconfigurationException, IOException, InterruptedException, TimeoutException { final String property = "FOO"; @@ -377,15 +346,15 @@ public void testConfIsUpdatedOnSuccess() reconfigurable.reconfigureProperty(property, value2); reconfigurable.startReconfiguration(); RaftTestUtil.waitFor(() -> reconfigurable.getReconfigurationStatus().ended(), 100, 60000); - Assert.assertEquals(value2, reconfigurable.getProperties().get(property)); + Assertions.assertEquals(value2, reconfigurable.getProperties().get(property)); } /** * Ensure that {@link ReconfigurationBase#startReconfiguration} updates * its parent's cached configuration on success. - * @throws IOException */ - @Test (timeout=300000) + @Test + @Timeout(value = 300) public void testConfIsUpdatedOnSuccessAsync() throws InterruptedException, IOException, TimeoutException { final String property = "FOO"; @@ -404,7 +373,7 @@ public void testConfIsUpdatedOnSuccessAsync() reconfigurable.startReconfiguration(); RaftTestUtil.waitFor(() -> reconfigurable.getReconfigurationStatus().ended(), 100, 60000); - Assert.assertEquals(value2, reconfigurable.getProperties().get(property)); + Assertions.assertEquals(value2, reconfigurable.getProperties().get(property)); } /** @@ -412,7 +381,8 @@ public void testConfIsUpdatedOnSuccessAsync() * property in its parent's configuration when the new value is null. * @throws IOException */ - @Test (timeout=300000) + @Test + @Timeout(value = 300) public void testConfIsUnset() throws InterruptedException, TimeoutException, IOException { final String property = "FOO"; @@ -427,15 +397,15 @@ public void testConfIsUnset() reconfigurable.startReconfiguration(); RaftTestUtil.waitFor(() -> reconfigurable.getReconfigurationStatus().ended(), 100, 60000); - Assert.assertNull(reconfigurable.getProperties().get(property)); + Assertions.assertNull(reconfigurable.getProperties().get(property)); } /** * Ensure that {@link ReconfigurationBase#startReconfiguration} unsets the * property in its parent's configuration when the new value is null. - * @throws IOException */ - @Test (timeout=300000) + @Test + @Timeout(value = 300) public void testConfIsUnsetAsync() throws ReconfigurationException, IOException, TimeoutException, InterruptedException { final String property = "FOO"; @@ -451,7 +421,7 @@ public void testConfIsUnsetAsync() throws ReconfigurationException, // Kick off a reconfiguration task and wait until it completes. reconfigurable.startReconfiguration(); RaftTestUtil.waitFor(() -> reconfigurable.getReconfigurationStatus().ended(), 100, 60000); - Assert.assertNull(reconfigurable.getProperties().get(property)); + Assertions.assertNull(reconfigurable.getProperties().get(property)); } private ReconfigurationBase makeReconfigurable( diff --git a/ratis-server/src/test/java/org/apache/ratis/WatchRequestTests.java b/ratis-server/src/test/java/org/apache/ratis/WatchRequestTests.java index b842ee9db5..a2c0cd2e10 100644 --- a/ratis-server/src/test/java/org/apache/ratis/WatchRequestTests.java +++ b/ratis-server/src/test/java/org/apache/ratis/WatchRequestTests.java @@ -42,9 +42,9 @@ import org.apache.ratis.util.TimeDuration; import org.apache.ratis.util.function.CheckedConsumer; import org.apache.ratis.util.function.CheckedSupplier; -import org.junit.Assert; -import org.junit.Before; -import org.junit.Test; +import org.junit.jupiter.api.Assertions; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.Test; import org.slf4j.Logger; import org.slf4j.event.Level; @@ -57,7 +57,7 @@ import java.util.concurrent.TimeUnit; import java.util.stream.Stream; -import static org.junit.Assert.fail; +import static org.junit.jupiter.api.Assertions.fail; public abstract class WatchRequestTests extends BaseTest @@ -70,7 +70,7 @@ public abstract class WatchRequestTests static final int NUM_SERVERS = 3; static final int GET_TIMEOUT_SECOND = 10; - @Before + @BeforeEach public void setup() { final RaftProperties p = getProperties(); p.setClass(MiniRaftCluster.STATEMACHINE_CLASS_KEY, @@ -128,7 +128,7 @@ CompletableFuture sendWatchRequest(long logIndex, RetryPolicy p CompletableFuture reply = watchClient.async().send(new RaftTestUtil.SimpleMessage("message")); long writeIndex = reply.get().getLogIndex(); - Assert.assertTrue(writeIndex > 0); + Assertions.assertTrue(writeIndex > 0); watchClient.async().watch(writeIndex, ReplicationLevel.MAJORITY_COMMITTED); return watchClient.async().watch(logIndex, ReplicationLevel.MAJORITY); } @@ -212,8 +212,8 @@ RaftClientReply get(CompletableFuture f, String name) throws Ex } log.info("{}-Watch({}) returns {}", name, logIndex, reply); - Assert.assertTrue(reply.isSuccess()); - Assert.assertTrue(reply.getLogIndex() >= logIndex); + Assertions.assertTrue(reply.isSuccess()); + Assertions.assertTrue(reply.getLogIndex() >= logIndex); return reply; } } @@ -240,8 +240,8 @@ static void runTestWatchRequestAsync(TestParameters p) throws Exception { p.sendRequests(replies, watches); - Assert.assertEquals(numMessages, replies.size()); - Assert.assertEquals(numMessages, watches.size()); + Assertions.assertEquals(numMessages, replies.size()); + Assertions.assertEquals(numMessages, watches.size()); // since leader is blocked, nothing can be done. TimeUnit.SECONDS.sleep(1); @@ -254,7 +254,7 @@ static void runTestWatchRequestAsync(TestParameters p) throws Exception { checkMajority(replies, watches, log); - Assert.assertEquals(numMessages, watches.size()); + Assertions.assertEquals(numMessages, watches.size()); // but not replicated/committed to all. TimeUnit.SECONDS.sleep(1); @@ -273,26 +273,26 @@ static void checkMajority(List> replies, final RaftClientReply reply = replies.get(i).get(GET_TIMEOUT_SECOND, TimeUnit.SECONDS); log.info("checkMajority {}: receive {}", i, reply); final long logIndex = reply.getLogIndex(); - Assert.assertTrue(reply.isSuccess()); + Assertions.assertTrue(reply.isSuccess()); final WatchReplies watchReplies = watches.get(i).get(GET_TIMEOUT_SECOND, TimeUnit.SECONDS); - Assert.assertEquals(logIndex, watchReplies.logIndex); + Assertions.assertEquals(logIndex, watchReplies.logIndex); final RaftClientReply watchMajorityReply = watchReplies.getMajority(); final RaftClientReply watchMajorityCommittedReply = watchReplies.getMajorityCommitted(); { // check commit infos final Collection commitInfos = watchMajorityCommittedReply.getCommitInfos(); final String message = "logIndex=" + logIndex + ", " + ProtoUtils.toString(commitInfos); - Assert.assertEquals(NUM_SERVERS, commitInfos.size()); + Assertions.assertEquals(NUM_SERVERS, commitInfos.size()); // One follower has not committed, so min must be less than logIndex final long min = commitInfos.stream().map(CommitInfoProto::getCommitIndex).min(Long::compare).get(); - Assert.assertTrue(message, logIndex > min); + Assertions.assertTrue(logIndex > min, message); // All other followers have committed commitInfos.stream() .map(CommitInfoProto::getCommitIndex).sorted(Long::compare) - .skip(1).forEach(ci -> Assert.assertTrue(message, logIndex <= ci)); + .skip(1).forEach(ci -> Assertions.assertTrue(logIndex <= ci, message)); } } } @@ -308,8 +308,8 @@ static void checkAll(List> watches, Logger log) { // check commit infos final Collection commitInfos = watchAllCommittedReply.getCommitInfos(); final String message = "logIndex=" + logIndex + ", " + ProtoUtils.toString(commitInfos); - Assert.assertEquals(NUM_SERVERS, commitInfos.size()); - commitInfos.forEach(info -> Assert.assertTrue(message, logIndex <= info.getCommitIndex())); + Assertions.assertEquals(NUM_SERVERS, commitInfos.size()); + commitInfos.forEach(info -> Assertions.assertTrue(logIndex <= info.getCommitIndex(), message)); } } } @@ -352,8 +352,8 @@ static void runTestWatchRequestAsyncChangeLeader(TestParameters p) throws Except p.sendRequests(replies, watches); - Assert.assertEquals(numMessages, replies.size()); - Assert.assertEquals(numMessages, watches.size()); + Assertions.assertEquals(numMessages, replies.size()); + Assertions.assertEquals(numMessages, watches.size()); // since only one follower is blocked commit, requests can be committed MAJORITY and ALL but not ALL_COMMITTED. checkMajority(replies, watches, log); @@ -410,8 +410,8 @@ static void runTestWatchRequestTimeout(TestParameters p) throws Exception { p.sendRequests(replies, watches); - Assert.assertEquals(numMessages, replies.size()); - Assert.assertEquals(numMessages, watches.size()); + Assertions.assertEquals(numMessages, replies.size()); + Assertions.assertEquals(numMessages, watches.size()); watchTimeout.sleep(); watchTimeoutDenomination.sleep(); // for roundup error @@ -461,11 +461,11 @@ static void runTestWatchRequestClientTimeout(TestParameters p) throws Exception fail("runTestWatchRequestClientTimeout failed"); } catch (Exception ex) { log.error("error occurred", ex); - Assert.assertTrue(ex.getCause().getClass() == AlreadyClosedException.class || + Assertions.assertTrue(ex.getCause().getClass() == AlreadyClosedException.class || ex.getCause().getClass() == RaftRetryFailureException.class); if (ex.getCause() != null) { if (ex.getCause().getCause() != null) { - Assert.assertEquals(TimeoutIOException.class, + Assertions.assertEquals(TimeoutIOException.class, ex.getCause().getCause().getClass()); } } @@ -531,10 +531,10 @@ static void checkTimeout(List> replies, final RaftClientReply reply = replies.get(i).get(GET_TIMEOUT_SECOND, TimeUnit.SECONDS); log.info("checkTimeout {}: receive {}", i, reply); final long logIndex = reply.getLogIndex(); - Assert.assertTrue(reply.isSuccess()); + Assertions.assertTrue(reply.isSuccess()); final WatchReplies watchReplies = watches.get(i).get(GET_TIMEOUT_SECOND, TimeUnit.SECONDS); - Assert.assertEquals(logIndex, watchReplies.logIndex); + Assertions.assertEquals(logIndex, watchReplies.logIndex); assertNotReplicatedException(logIndex, ReplicationLevel.ALL, watchReplies::getAll); @@ -554,11 +554,11 @@ static void assertNotReplicatedException(long logIndex, ReplicationLevel replica } static void assertNotReplicatedException(long logIndex, ReplicationLevel replication, Throwable t) { - Assert.assertSame(NotReplicatedException.class, t.getClass()); + Assertions.assertSame(NotReplicatedException.class, t.getClass()); final NotReplicatedException nre = (NotReplicatedException) t; - Assert.assertNotNull(nre); - Assert.assertEquals(logIndex, nre.getLogIndex()); - Assert.assertEquals(replication, nre.getRequiredReplication()); - Assert.assertNotNull(nre.getCommitInfos()); + Assertions.assertNotNull(nre); + Assertions.assertEquals(logIndex, nre.getLogIndex()); + Assertions.assertEquals(replication, nre.getRequiredReplication()); + Assertions.assertNotNull(nre.getCommitInfos()); } } diff --git a/ratis-server/src/test/java/org/apache/ratis/server/impl/GroupInfoBaseTest.java b/ratis-server/src/test/java/org/apache/ratis/server/impl/GroupInfoBaseTest.java index 72eedf0f32..218b6e067d 100644 --- a/ratis-server/src/test/java/org/apache/ratis/server/impl/GroupInfoBaseTest.java +++ b/ratis-server/src/test/java/org/apache/ratis/server/impl/GroupInfoBaseTest.java @@ -24,8 +24,8 @@ import org.apache.ratis.proto.RaftProtos.CommitInfoProto; import org.apache.ratis.server.RaftServer; import org.apache.ratis.util.Slf4jUtils; -import org.junit.Assert; -import org.junit.Test; +import org.junit.jupiter.api.Assertions; +import org.junit.jupiter.api.Test; import org.slf4j.event.Level; import java.util.Collection; @@ -92,7 +92,7 @@ private void runTest(CLUSTER cluster) throws Exception { final RaftClientReply reply = sendMessages(numMessages, cluster); for(CommitInfoProto i : reply.getCommitInfos()) { if (!RaftPeerId.valueOf(i.getServer().getId()).equals(killedFollower)) { - Assert.assertTrue(i.getCommitIndex() > maxCommit); + Assertions.assertTrue(i.getCommitIndex() > maxCommit); } } } @@ -104,12 +104,12 @@ private void runTest(CLUSTER cluster) throws Exception { } try(final RaftClient client = cluster.createClient(peer.getId())) { final GroupListReply info = client.getGroupManagementApi(peer.getId()).list(); - Assert.assertEquals(1, info.getGroupIds().stream().filter(id -> group.getGroupId().equals(id)).count()); + Assertions.assertEquals(1, info.getGroupIds().stream().filter(id -> group.getGroupId().equals(id)).count()); for(CommitInfoProto i : info.getCommitInfos()) { if (RaftPeerId.valueOf(i.getServer().getId()).equals(killedFollower)) { - Assert.assertTrue(i.getCommitIndex() <= maxCommit); + Assertions.assertTrue(i.getCommitIndex() <= maxCommit); } else { - Assert.assertTrue(i.getCommitIndex() > maxCommit); + Assertions.assertTrue(i.getCommitIndex() > maxCommit); } } } diff --git a/ratis-server/src/test/java/org/apache/ratis/server/impl/GroupManagementBaseTest.java b/ratis-server/src/test/java/org/apache/ratis/server/impl/GroupManagementBaseTest.java index a06336f912..a86ab2a515 100644 --- a/ratis-server/src/test/java/org/apache/ratis/server/impl/GroupManagementBaseTest.java +++ b/ratis-server/src/test/java/org/apache/ratis/server/impl/GroupManagementBaseTest.java @@ -36,8 +36,8 @@ import org.apache.ratis.util.Slf4jUtils; import org.apache.ratis.util.TimeDuration; import org.apache.ratis.util.function.CheckedBiConsumer; -import org.junit.Assert; -import org.junit.Test; +import org.junit.jupiter.api.Assertions; +import org.junit.jupiter.api.Test; import org.slf4j.Logger; import org.slf4j.LoggerFactory; import org.slf4j.event.Level; @@ -97,7 +97,7 @@ public void testGroupWithPriority() throws Exception { // Make sure that there are no leaders. TimeUnit.SECONDS.sleep(1); LOG.info("start: " + cluster.printServers()); - Assert.assertNull(cluster.getLeader()); + Assertions.assertNull(cluster.getLeader()); // Add groups List peers = cluster.getPeers(); @@ -108,7 +108,7 @@ public void testGroupWithPriority() throws Exception { LOG.info("add new group: " + newGroup); try (final RaftClient client = cluster.createClient(newGroup)) { // Before request, client try leader with the highest priority - Assert.assertTrue(client.getLeaderId() == peersWithPriority.get(suggestedLeaderIndex).getId()); + Assertions.assertTrue(client.getLeaderId() == peersWithPriority.get(suggestedLeaderIndex).getId()); for (RaftPeer p : newGroup.getPeers()) { client.getGroupManagementApi(p.getId()).add(newGroup); } @@ -116,7 +116,7 @@ public void testGroupWithPriority() throws Exception { JavaUtils.attempt(() -> { final RaftServer.Division leader = RaftTestUtil.waitForLeader(cluster, newGroup.getGroupId()); - Assert.assertTrue(leader.getId() == peers.get(suggestedLeaderIndex).getId()); + Assertions.assertTrue(leader.getId() == peers.get(suggestedLeaderIndex).getId()); }, 10, TimeDuration.valueOf(1, TimeUnit.SECONDS), "testMultiGroupWithPriority", LOG); String suggestedLeader = peers.get(suggestedLeaderIndex).getId().toString(); @@ -130,7 +130,7 @@ public void testGroupWithPriority() throws Exception { JavaUtils.attempt(() -> { final RaftServer.Division leader = RaftTestUtil.waitForLeader(cluster, newGroup.getGroupId()); - Assert.assertTrue(leader.getId() != peers.get(suggestedLeaderIndex).getId()); + Assertions.assertTrue(leader.getId() != peers.get(suggestedLeaderIndex).getId()); }, 10, TimeDuration.valueOf(1, TimeUnit.SECONDS), "testMultiGroupWithPriority", LOG); // send request so that suggested leader's log lag behind new leader's, @@ -138,7 +138,7 @@ public void testGroupWithPriority() throws Exception { try (final RaftClient client = cluster.createClient(newGroup)) { for (int i = 0; i < 10; i ++) { RaftClientReply reply = client.io().send(new RaftTestUtil.SimpleMessage("m" + i)); - Assert.assertTrue(reply.isSuccess()); + Assertions.assertTrue(reply.isSuccess()); } } @@ -150,7 +150,7 @@ public void testGroupWithPriority() throws Exception { // leadership to suggested leader when suggested leader catch up the log. JavaUtils.attempt(() -> { final RaftServer.Division leader = RaftTestUtil.waitForLeader(cluster, newGroup.getGroupId()); - Assert.assertTrue(leader.getId() == peers.get(suggestedLeaderIndex).getId()); + Assertions.assertTrue(leader.getId() == peers.get(suggestedLeaderIndex).getId()); }, 10, TimeDuration.valueOf(1, TimeUnit.SECONDS), "testMultiGroupWithPriority", LOG); // change the suggest leader @@ -158,18 +158,18 @@ public void testGroupWithPriority() throws Exception { List peersWithNewPriority = getPeersWithPriority(peers, peers.get(newSuggestedLeaderIndex)); try (final RaftClient client = cluster.createClient(newGroup)) { RaftClientReply reply = client.admin().setConfiguration(peersWithNewPriority.toArray(new RaftPeer[0])); - Assert.assertTrue(reply.isSuccess()); + Assertions.assertTrue(reply.isSuccess()); } JavaUtils.attempt(() -> { final RaftServer.Division leader = RaftTestUtil.waitForLeader(cluster, newGroup.getGroupId()); - Assert.assertTrue(leader.getId() == peers.get(newSuggestedLeaderIndex).getId()); + Assertions.assertTrue(leader.getId() == peers.get(newSuggestedLeaderIndex).getId()); }, 10, TimeDuration.valueOf(1, TimeUnit.SECONDS), "testMultiGroupWithPriority", LOG); cluster.killServer(peers.get(newSuggestedLeaderIndex).getId()); JavaUtils.attempt(() -> { final RaftServer.Division leader = RaftTestUtil.waitForLeader(cluster, newGroup.getGroupId()); - Assert.assertTrue(leader.getId() != peers.get(newSuggestedLeaderIndex).getId()); + Assertions.assertTrue(leader.getId() != peers.get(newSuggestedLeaderIndex).getId()); }, 10, TimeDuration.valueOf(1, TimeUnit.SECONDS), "testMultiGroupWithPriority", LOG); cluster.shutdown(); @@ -191,7 +191,7 @@ public void testSingleGroupRestart() throws Exception { // Make sure that there are no leaders. TimeUnit.SECONDS.sleep(1); LOG.info("start: " + cluster.printServers()); - Assert.assertNull(cluster.getLeader()); + Assertions.assertNull(cluster.getLeader()); // Add groups final RaftGroup newGroup = RaftGroup.valueOf(RaftGroupId.randomId(), cluster.getPeers()); @@ -201,7 +201,7 @@ public void testSingleGroupRestart() throws Exception { client.getGroupManagementApi(p.getId()).add(newGroup); } } - Assert.assertNotNull(RaftTestUtil.waitForLeader(cluster)); + Assertions.assertNotNull(RaftTestUtil.waitForLeader(cluster)); TimeUnit.SECONDS.sleep(1); // restart the servers with null group @@ -211,7 +211,7 @@ public void testSingleGroupRestart() throws Exception { } // the servers should retrieve the conf from the log. - Assert.assertNotNull(RaftTestUtil.waitForLeader(cluster)); + Assertions.assertNotNull(RaftTestUtil.waitForLeader(cluster)); cluster.shutdown(); } @@ -267,7 +267,7 @@ public static void runMultiGroupTest( // Make sure that there are no leaders. TimeUnit.SECONDS.sleep(1); LOG.info("start: " + cluster.printServers()); - Assert.assertNull(cluster.getLeader()); + Assertions.assertNull(cluster.getLeader()); try { // Reinitialize servers to three groups @@ -286,7 +286,7 @@ public static void runMultiGroupTest( client.getGroupManagementApi(p.getId()).add(groups[i]); } } - Assert.assertNotNull(RaftTestUtil.waitForLeader(cluster, gid)); + Assertions.assertNotNull(RaftTestUtil.waitForLeader(cluster, gid)); checker.accept(cluster, groups[i]); } printThreadCount(type, "start groups"); @@ -302,15 +302,15 @@ public static void runMultiGroupTest( for (RaftPeer p : g.getPeers()) { final RaftServer.Division d = cluster.getDivision(p.getId(), g.getGroupId()); final File root = d.getRaftStorage().getStorageDir().getRoot(); - Assert.assertTrue(root.exists()); - Assert.assertTrue(root.isDirectory()); + Assertions.assertTrue(root.exists()); + Assertions.assertTrue(root.isDirectory()); final RaftClientReply r; try (final RaftClient client = cluster.createClient(p.getId(), g)) { r = client.getGroupManagementApi(p.getId()).remove(g.getGroupId(), true, false); } - Assert.assertTrue(r.isSuccess()); - Assert.assertFalse(root.exists()); + Assertions.assertTrue(r.isSuccess()); + Assertions.assertFalse(root.exists()); } } } @@ -335,7 +335,7 @@ public static void runMultiGroupTest( peers -> client.admin().setConfiguration(peers.toArray(RaftPeer.emptyArray()))); } - Assert.assertNotNull(RaftTestUtil.waitForLeader(cluster)); + Assertions.assertNotNull(RaftTestUtil.waitForLeader(cluster)); checker.accept(cluster, groups[chosen]); LOG.info("update groups: " + cluster.printServers()); printThreadCount(type, "update groups"); @@ -358,15 +358,15 @@ public void testGroupAlreadyExists() throws Exception { final RaftPeerId peerId = peer.getId(); final RaftGroup group = RaftGroup.valueOf(cluster.getGroupId(), peer); try (final RaftClient client = cluster.createClient()) { - Assert.assertEquals(group, cluster.getDivision(peerId).getGroup()); + Assertions.assertEquals(group, cluster.getDivision(peerId).getGroup()); try { client.getGroupManagementApi(peer.getId()).add(group); } catch (IOException ex) { // HadoopRPC throws RemoteException, which makes it hard to check if // the exception is instance of AlreadyExistsException - Assert.assertTrue(ex.toString().contains(AlreadyExistsException.class.getCanonicalName())); + Assertions.assertTrue(ex.toString().contains(AlreadyExistsException.class.getCanonicalName())); } - Assert.assertEquals(group, cluster.getDivision(peerId).getGroup()); + Assertions.assertEquals(group, cluster.getDivision(peerId).getGroup()); cluster.shutdown(); } } @@ -383,32 +383,32 @@ public void testGroupRemoveWhenRename() throws Exception { final RaftGroup group1 = RaftGroup.valueOf(cluster1.getGroupId(), peer1); final RaftGroup group2 = RaftGroup.valueOf(cluster2.getGroupId(), peer1); try (final RaftClient client = cluster1.createClient()) { - Assert.assertEquals(group1, cluster1.getDivision(peerId1).getGroup()); + Assertions.assertEquals(group1, cluster1.getDivision(peerId1).getGroup()); try { // Group2 is added to one of the peers in Group1 final GroupManagementApi api1 = client.getGroupManagementApi(peerId1); api1.add(group2); List groupIds1 = cluster1.getServer(peerId1).getGroupIds(); - Assert.assertEquals(groupIds1.size(), 2); + Assertions.assertEquals(groupIds1.size(), 2); // Group2 is renamed from the peer1 of Group1 api1.remove(group2.getGroupId(), false, true); groupIds1 = cluster1.getServer(peerId1).getGroupIds(); - Assert.assertEquals(groupIds1.size(), 1); + Assertions.assertEquals(groupIds1.size(), 1); cluster1.restart(false); List groupIdsAfterRestart = cluster1.getServer(peerId1).getGroupIds(); - Assert.assertEquals(groupIds1.size(), groupIdsAfterRestart.size()); + Assertions.assertEquals(groupIds1.size(), groupIdsAfterRestart.size()); File renamedGroup = new File(RaftServerConfigKeys.removedGroupsDir( cluster1.getProperties()), group2.getGroupId().getUuid().toString()); - Assert.assertTrue(renamedGroup.isDirectory()); + Assertions.assertTrue(renamedGroup.isDirectory()); } catch (IOException ex) { - Assert.fail(); + Assertions.fail(); } finally { cluster1.shutdown(); // Clean up @@ -430,7 +430,7 @@ public void testGroupRemoveWhenDelete() throws Exception { final RaftGroup group1 = RaftGroup.valueOf(cluster1.getGroupId(), peer1); final RaftGroup group2 = RaftGroup.valueOf(cluster2.getGroupId(), peer1); try (final RaftClient client = cluster1.createClient()) { - Assert.assertEquals(group1, + Assertions.assertEquals(group1, cluster1.getDivision(peerId1).getGroup()); try { @@ -438,20 +438,20 @@ public void testGroupRemoveWhenDelete() throws Exception { final GroupManagementApi api1 = client.getGroupManagementApi(peerId1); api1.add(group2); List groupIds1 = cluster1.getServer(peerId1).getGroupIds(); - Assert.assertEquals(groupIds1.size(), 2); + Assertions.assertEquals(groupIds1.size(), 2); // Group2 is deleted from the peer1 of Group1 api1.remove(group2.getGroupId(), true, false); groupIds1 = cluster1.getServer(peerId1).getGroupIds(); - Assert.assertEquals(groupIds1.size(), 1); + Assertions.assertEquals(groupIds1.size(), 1); cluster1.restart(false); List groupIdsAfterRestart = cluster1.getServer(peerId1).getGroupIds(); - Assert.assertEquals(groupIds1.size(), groupIdsAfterRestart.size()); + Assertions.assertEquals(groupIds1.size(), groupIdsAfterRestart.size()); } catch (IOException ex) { - Assert.fail(); + Assertions.fail(); } finally { cluster1.shutdown(); FileUtils.deleteFully(RaftServerConfigKeys.removedGroupsDir( diff --git a/ratis-server/src/test/java/org/apache/ratis/server/impl/LogMetadataTests.java b/ratis-server/src/test/java/org/apache/ratis/server/impl/LogMetadataTests.java index a834e27144..537ece931d 100644 --- a/ratis-server/src/test/java/org/apache/ratis/server/impl/LogMetadataTests.java +++ b/ratis-server/src/test/java/org/apache/ratis/server/impl/LogMetadataTests.java @@ -29,8 +29,8 @@ import org.apache.ratis.server.RaftServer; import org.apache.ratis.server.RaftServerConfigKeys; import org.apache.ratis.server.raftlog.RaftLog; -import org.junit.Assert; -import org.junit.Test; +import org.junit.jupiter.api.Assertions; +import org.junit.jupiter.api.Test; public abstract class LogMetadataTests extends BaseTest @@ -72,7 +72,7 @@ public void testLogMetadataBasicTest(boolean logMetadata, Predicate checke long commitIndex = cluster.getServer(leaderId).getDivision(cluster.getGroupId()).getRaftLog() .getLastCommittedIndex(); - Assert.assertTrue(checker.test(commitIndex)); + Assertions.assertTrue(checker.test(commitIndex)); } finally { cluster.shutdown(); } diff --git a/ratis-server/src/test/java/org/apache/ratis/server/impl/PreAppendLeaderStepDownTest.java b/ratis-server/src/test/java/org/apache/ratis/server/impl/PreAppendLeaderStepDownTest.java index 4a62844c0a..0211bfed00 100644 --- a/ratis-server/src/test/java/org/apache/ratis/server/impl/PreAppendLeaderStepDownTest.java +++ b/ratis-server/src/test/java/org/apache/ratis/server/impl/PreAppendLeaderStepDownTest.java @@ -34,8 +34,8 @@ import org.apache.ratis.util.JavaUtils; import org.apache.ratis.util.Slf4jUtils; import org.apache.ratis.util.TimeDuration; -import org.junit.Assert; -import org.junit.Test; +import org.junit.jupiter.api.Assertions; +import org.junit.jupiter.api.Test; import org.slf4j.event.Level; import java.io.IOException; @@ -105,9 +105,9 @@ private void runTestLeaderStepDown(CLUSTER cluster) throws Exception { RaftTestUtil.waitForLeader(cluster).getRaftLog().getLastEntryTermIndex().getTerm(); if (leaderShouldStepDown) { - Assert.assertTrue(newTerm > oldTerm); + Assertions.assertTrue(newTerm > oldTerm); } else { - Assert.assertEquals(newTerm, oldTerm); + Assertions.assertEquals(newTerm, oldTerm); } cluster.shutdown(); @@ -124,11 +124,11 @@ void runTestLeaderStepDownAsync(CLUSTER cluster) throws IOException, Interrupted RaftPeerId leaderId = leader.getId(); RaftServerImpl l = (RaftServerImpl) leader; try (RaftClient client = cluster.createClient(leader.getId())) { - JavaUtils.attempt(() -> Assert.assertEquals(leaderId, leader.getId()), + JavaUtils.attempt(() -> Assertions.assertEquals(leaderId, leader.getId()), 20, ONE_SECOND, "check leader id", LOG); RaftClientReply reply = client.admin().transferLeadership(null, 3000); - Assert.assertTrue(reply.isSuccess()); - Assert.assertEquals(2, ((RaftServerImpl) leader).getRole().getCurrentRole().getNumber()); + Assertions.assertTrue(reply.isSuccess()); + Assertions.assertEquals(2, ((RaftServerImpl) leader).getRole().getCurrentRole().getNumber()); } } } diff --git a/ratis-server/src/test/java/org/apache/ratis/server/impl/RaftReconfigurationBaseTest.java b/ratis-server/src/test/java/org/apache/ratis/server/impl/RaftReconfigurationBaseTest.java index 843c271b1c..68ad4ebd24 100644 --- a/ratis-server/src/test/java/org/apache/ratis/server/impl/RaftReconfigurationBaseTest.java +++ b/ratis-server/src/test/java/org/apache/ratis/server/impl/RaftReconfigurationBaseTest.java @@ -42,12 +42,14 @@ import org.apache.ratis.server.raftlog.RaftLog; import org.apache.ratis.server.raftlog.RaftLogBase; import org.apache.ratis.server.storage.RaftStorageTestUtils; +import org.apache.ratis.test.tag.Flaky; import org.apache.ratis.util.JavaUtils; import org.apache.ratis.util.LifeCycle; import org.apache.ratis.util.Slf4jUtils; import org.apache.ratis.util.TimeDuration; -import org.junit.Assert; -import org.junit.Test; +import org.junit.jupiter.api.Assertions; +import org.junit.jupiter.api.Test; +import org.junit.jupiter.api.Timeout; import org.slf4j.event.Level; import java.io.IOException; @@ -64,7 +66,7 @@ import static java.util.Arrays.asList; import static org.apache.ratis.server.impl.RaftServerTestUtil.waitAndCheckNewConf; -import static org.junit.Assert.assertThrows; +import static org.junit.jupiter.api.Assertions.assertThrows; public abstract class RaftReconfigurationBaseTest extends BaseTest @@ -94,7 +96,7 @@ private void checkPriority(CLUSTER cluster, RaftGroupId groupId, List for (int j = 0; j < peersWithPriority.size(); j ++) { int priorityInConf = conf.getPeer(peersWithPriority.get(j).getId()).getPriority(); - Assert.assertEquals(priorityInConf, peersWithPriority.get(j).getPriority()); + Assertions.assertEquals(priorityInConf, peersWithPriority.get(j).getPriority()); } } } @@ -169,15 +171,15 @@ public void testLeaderElectionWhenChangeFromSingleToHA() throws Exception { .setConf(new PeerConfiguration(Arrays.asList(curPeer, newPeer))) .setLogEntryIndex(Long.MAX_VALUE / 2) .build(); - Assert.assertTrue(oldNewConf.isSingleMode(curPeer.getId())); + Assertions.assertTrue(oldNewConf.isSingleMode(curPeer.getId())); RaftServerTestUtil.setRaftConf(leaderServer, groupId, oldNewConf); try(RaftClient client = cluster.createClient()) { client.admin().transferLeadership(null, leaderServer.getId(), 1000); } final RaftServer.Division newLeader = RaftTestUtil.waitForLeader(cluster); - Assert.assertEquals(leaderServer.getId(), newLeader.getId()); - Assert.assertEquals(oldNewConf, newLeader.getRaftConf()); + Assertions.assertEquals(leaderServer.getId(), newLeader.getId()); + Assertions.assertEquals(oldNewConf, newLeader.getRaftConf()); }); } @@ -195,8 +197,8 @@ public void testChangeMajority() throws Exception { .setServersInNewConf(c1.allPeersInNewConf) .setMode(SetConfigurationRequest.Mode.COMPARE_AND_SET) .build(); - assertThrows("Expect change majority error.", SetConfigurationException.class, - () -> client.admin().setConfiguration(arguments)); + assertThrows(SetConfigurationException.class, () -> client.admin().setConfiguration(arguments), + "Expect change majority error."); } }); } @@ -262,13 +264,13 @@ private void runTestSetConfigurationInAddMode(CLUSTER cluster) throws Exception try (final RaftClient client = cluster.createClient(leader.getId())) { for (int i = 0; i < 10; i++) { RaftClientReply reply = client.io().send(new SimpleMessage("m" + i)); - Assert.assertTrue(reply.isSuccess()); + Assertions.assertTrue(reply.isSuccess()); } RaftClientReply reply = client.admin().setConfiguration( SetConfigurationRequest.Arguments.newBuilder() .setServersInNewConf(peers) .setMode(SetConfigurationRequest.Mode.ADD).build()); - Assert.assertTrue(reply.isSuccess()); + Assertions.assertTrue(reply.isSuccess()); waitAndCheckNewConf(cluster, change.allPeersInNewConf, 0, null); } cluster.close(); @@ -289,7 +291,7 @@ private void runTestSetConfigurationInCasMode(CLUSTER cluster) throws Exception try (final RaftClient client = cluster.createClient(leader.getId())) { for (int i = 0; i < 10; i++) { RaftClientReply reply = client.io().send(new SimpleMessage("m" + i)); - Assert.assertTrue(reply.isSuccess()); + Assertions.assertTrue(reply.isSuccess()); } testFailureCase("Can't set configuration in CAS mode ", @@ -306,7 +308,7 @@ private void runTestSetConfigurationInCasMode(CLUSTER cluster) throws Exception .setServersInCurrentConf(oldPeers) .setMode(SetConfigurationRequest.Mode.COMPARE_AND_SET) .build()); - Assert.assertTrue(reply.isSuccess()); + Assertions.assertTrue(reply.isSuccess()); waitAndCheckNewConf(cluster, change.allPeersInNewConf, 0, null); } cluster.close(); @@ -314,7 +316,8 @@ private void runTestSetConfigurationInCasMode(CLUSTER cluster) throws Exception - @Test(timeout = 30000) + @Test + @Timeout(value = 30) public void testReconfTwice() throws Exception { runWithNewCluster(3, this::runTestReconfTwice); } @@ -326,7 +329,7 @@ void runTestReconfTwice(CLUSTER cluster) throws Exception { // submit some msgs before reconf for (int i = 0; i < STAGING_CATCHUP_GAP * 2; i++) { RaftClientReply reply = client.io().send(new SimpleMessage("m" + i)); - Assert.assertTrue(reply.isSuccess()); + Assertions.assertTrue(reply.isSuccess()); } final AtomicBoolean reconf1 = new AtomicBoolean(false); @@ -360,8 +363,8 @@ void runTestReconfTwice(CLUSTER cluster) throws Exception { clientThread.start(); latch.await(); - Assert.assertTrue(reconf1.get()); - Assert.assertTrue(reconf2.get()); + Assertions.assertTrue(reconf1.get()); + Assertions.assertTrue(reconf2.get()); waitAndCheckNewConf(cluster, finalPeers.get(), 2, null); final RaftPeerId leader2 = RaftTestUtil.waitForLeader(cluster).getId(); @@ -373,7 +376,7 @@ void runTestReconfTwice(CLUSTER cluster) throws Exception { // each leader change generates one configuration. // expectedConf = 1 (init) + 2*2 (two conf changes) + #leader final int expectedConf = leader2.equals(leaderId) ? 6 : 7; - Assert.assertEquals(server.getId() + ": " + confManager, expectedConf, confManager.numOfConf()); + Assertions.assertEquals(expectedConf, confManager.numOfConf(), server.getId() + ": " + confManager); }); } } @@ -392,17 +395,17 @@ void runTestReconfTimeout(CLUSTER cluster) throws Exception { LOG.info("Start changing the configuration: {}", asList(c1.allPeersInNewConf)); - Assert.assertFalse(((RaftConfigurationImpl)cluster.getLeader().getRaftConf()).isTransitional()); + Assertions.assertFalse(((RaftConfigurationImpl)cluster.getLeader().getRaftConf()).isTransitional()); final RaftClientRpc sender = client.getClientRpc(); final SetConfigurationRequest request = cluster.newSetConfigurationRequest( client.getId(), leaderId, c1.allPeersInNewConf); try { RaftClientReply reply = sender.sendRequest(request); - Assert.fail("did not get expected exception " + reply.toString()); + Assertions.fail("did not get expected exception " + reply.toString()); } catch (IOException e) { - Assert.assertTrue("Got exception " + e, - e instanceof ReconfigurationTimeoutException); + Assertions.assertTrue( + e instanceof ReconfigurationTimeoutException, "Got exception " + e); } // the two new peers have not started yet, the bootstrapping must timeout @@ -412,10 +415,10 @@ void runTestReconfTimeout(CLUSTER cluster) throws Exception { // state so that we still get timeout instead of in-progress exception try { sender.sendRequest(request); - Assert.fail("did not get expected exception"); + Assertions.fail("did not get expected exception"); } catch (IOException e) { - Assert.assertTrue("Got exception " + e, - e instanceof ReconfigurationTimeoutException); + Assertions.assertTrue( + e instanceof ReconfigurationTimeoutException, "Got exception " + e); } // start the two new peers @@ -423,7 +426,7 @@ void runTestReconfTimeout(CLUSTER cluster) throws Exception { for (RaftPeer np : c1.newPeers) { cluster.restartServer(np.getId(), false); } - Assert.assertTrue(client.admin().setConfiguration(c1.allPeersInNewConf).isSuccess()); + Assertions.assertTrue(client.admin().setConfiguration(c1.allPeersInNewConf).isSuccess()); } } @@ -443,8 +446,8 @@ public void testBootstrapReconfWithSingleNodeAddTwo() throws Exception { try (final RaftClient client = cluster.createClient(leaderId)) { final PeerChanges c1 = cluster.addNewPeers(2, true); - assertThrows("Expect change majority error.", SetConfigurationException.class, - () -> client.admin().setConfiguration(c1.allPeersInNewConf)); + assertThrows(SetConfigurationException.class, () -> client.admin().setConfiguration(c1.allPeersInNewConf), + "Expect change majority error."); } }); } @@ -465,7 +468,7 @@ void runTestBootstrapReconf(int numNewPeer, boolean startNewPeer, CLUSTER cluste // submit some msgs before reconf for (int i = 0; i < STAGING_CATCHUP_GAP * 2; i++) { RaftClientReply reply = client.io().send(new SimpleMessage("m" + i)); - Assert.assertTrue(reply.isSuccess()); + Assertions.assertTrue(reply.isSuccess()); } final PeerChanges c1 = cluster.addNewPeers(numNewPeer, startNewPeer); @@ -501,7 +504,7 @@ void runTestBootstrapReconf(int numNewPeer, boolean startNewPeer, CLUSTER cluste final RaftServer.Division d = cluster.getDivision(newPeer.getId()); RaftTestUtil.waitFor(() -> leaderLog.getEntries(0, Long.MAX_VALUE).length == d.getRaftLog().getEntries(0, Long.MAX_VALUE).length, 300, 15000); - Assert.assertArrayEquals(leaderLog.getEntries(0, Long.MAX_VALUE), + Assertions.assertArrayEquals(leaderLog.getEntries(0, Long.MAX_VALUE), d.getRaftLog().getEntries(0, Long.MAX_VALUE)); } } @@ -512,6 +515,7 @@ void runTestBootstrapReconf(int numNewPeer, boolean startNewPeer, CLUSTER cluste * retrying. */ @Test + @Flaky("RATIS-2251") public void testKillLeaderDuringReconf() throws Exception { // originally 3 peers runWithNewCluster(3, this::runTestKillLeaderDuringReconf); @@ -549,7 +553,7 @@ void runTestKillLeaderDuringReconf(CLUSTER cluster) throws Exception { // the leader cannot generate the (old, new) conf, and it will keep // bootstrapping the 1 new peer since it has not started yet. - Assert.assertFalse(((RaftConfigurationImpl)cluster.getLeader().getRaftConf()).isTransitional()); + Assertions.assertFalse(((RaftConfigurationImpl)cluster.getLeader().getRaftConf()).isTransitional()); // (0) the first conf entry, (1) the 1st setConf entry, (2) a metadata entry // (3) new current conf entry (4) a metadata entry @@ -559,11 +563,11 @@ void runTestKillLeaderDuringReconf(CLUSTER cluster) throws Exception { LOG.info("{}", LogProtoUtils.toLogEntryString(e)); } final long commitIndex = leaderLog.getLastCommittedIndex(); - Assert.assertTrue("commitIndex = " + commitIndex + " > 2", commitIndex <= 2); + Assertions.assertTrue(commitIndex <= 2, "commitIndex = " + commitIndex + " > 2"); } final RaftPeerId killed = RaftTestUtil.waitAndKillLeader(cluster); - Assert.assertEquals(leaderId, killed); + Assertions.assertEquals(leaderId, killed); final RaftPeerId newLeaderId = RaftTestUtil.waitForLeader(cluster).getId(); LOG.info("newLeaderId: {}", newLeaderId); TimeDuration.valueOf(1500, TimeUnit.MILLISECONDS).sleep(); @@ -615,13 +619,13 @@ void runTestNoChangeRequest(CLUSTER cluster) throws Exception { // no real configuration change in the request final RaftClientReply reply = client.admin().setConfiguration(cluster.getPeers().toArray(RaftPeer.emptyArray())); - Assert.assertTrue(reply.isSuccess()); + Assertions.assertTrue(reply.isSuccess()); final long newCommittedIndex = leaderLog.getLastCommittedIndex(); for(long i = committedIndex + 1; i <= newCommittedIndex; i++) { final LogEntryProto e = leaderLog.get(i); - Assert.assertTrue(e.hasMetadataEntry()); + Assertions.assertTrue(e.hasMetadataEntry()); } - Assert.assertSame(confBefore, cluster.getLeader().getRaftConf()); + Assertions.assertSame(confBefore, cluster.getLeader().getRaftConf()); } } @@ -691,8 +695,8 @@ void runTestOverlappedSetConfRequests(CLUSTER cluster) throws Exception { for (int i = 0; i < 10 && !confChanged.get(); i++) { Thread.sleep(1000); } - Assert.assertTrue(confChanged.get()); - Assert.assertTrue(caughtException.get()); + Assertions.assertTrue(confChanged.get()); + Assertions.assertTrue(caughtException.get()); } finally { LOG_SYNC_DELAY.clear(); } @@ -758,11 +762,11 @@ void runTestRevertConfigurationChange(CLUSTER cluster) throws Exception { // wait till the old leader persist the new conf JavaUtils.attemptRepeatedly(() -> { - Assert.assertTrue(log.getFlushIndex() >= confIndex); + Assertions.assertTrue(log.getFlushIndex() >= confIndex); return null; }, 10, sleepTime, "FLUSH", LOG); final long committed = log.getLastCommittedIndex(); - Assert.assertTrue(committed < confIndex); + Assertions.assertTrue(committed < confIndex); // unblock the old leader BlockRequestHandlingInjection.getInstance().unblockReplier(leaderId.toString()); @@ -770,14 +774,14 @@ void runTestRevertConfigurationChange(CLUSTER cluster) throws Exception { // the client should get NotLeaderException clientThread.join(5000); - Assert.assertTrue(gotNotLeader.get()); + Assertions.assertTrue(gotNotLeader.get()); // the old leader should have truncated the setConf from the log JavaUtils.attemptRepeatedly(() -> { - Assert.assertTrue(log.getLastCommittedIndex() >= confIndex); + Assertions.assertTrue(log.getLastCommittedIndex() >= confIndex); return null; }, 10, ONE_SECOND, "COMMIT", LOG); - Assert.assertTrue(log.get(confIndex).hasConfigurationEntry()); + Assertions.assertTrue(log.get(confIndex).hasConfigurationEntry()); log2 = null; } finally { RaftStorageTestUtils.printLog(log2, s -> LOG.info(s)); @@ -833,8 +837,8 @@ public void testLeaderNotReadyException() throws Exception { for (int i = 0; !success.get() && i < 5; i++) { Thread.sleep(1000); } - Assert.assertTrue(success.get()); - Assert.assertTrue(caughtNotReady.get()); + Assertions.assertTrue(success.get()); + Assertions.assertTrue(caughtNotReady.get()); } finally { LEADER_PLACE_HOLDER_DELAY.clear(); cluster.shutdown(); diff --git a/ratis-server/src/test/java/org/apache/ratis/server/impl/RaftServerTestUtil.java b/ratis-server/src/test/java/org/apache/ratis/server/impl/RaftServerTestUtil.java index 0054e16018..db22c4854a 100644 --- a/ratis-server/src/test/java/org/apache/ratis/server/impl/RaftServerTestUtil.java +++ b/ratis-server/src/test/java/org/apache/ratis/server/impl/RaftServerTestUtil.java @@ -40,7 +40,7 @@ import org.apache.ratis.util.Slf4jUtils; import org.apache.ratis.util.TimeDuration; import org.apache.ratis.util.function.CheckedConsumer; -import org.junit.Assert; +import org.junit.jupiter.api.Assertions; import org.mockito.Mockito; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -99,7 +99,7 @@ public static void waitAndCheckNewConf(MiniRaftCluster cluster, private static void waitAndCheckNewConf(MiniRaftCluster cluster, Collection peers, Collection deadPeers) { LOG.info("waitAndCheckNewConf: peers={}, deadPeers={}, {}", peers, deadPeers, cluster.printServers()); - Assert.assertNotNull(cluster.getLeader()); + Assertions.assertNotNull(cluster.getLeader()); int numIncluded = 0; int deadIncluded = 0; @@ -117,16 +117,16 @@ private static void waitAndCheckNewConf(MiniRaftCluster cluster, final RaftConfigurationImpl conf = server.getState().getRaftConf(); if (current.containsInConf(server.getId())) { numIncluded++; - Assert.assertTrue(conf.isStable()); - Assert.assertTrue(conf.hasNoChange(peers, Collections.emptyList())); + Assertions.assertTrue(conf.isStable()); + Assertions.assertTrue(conf.hasNoChange(peers, Collections.emptyList())); } else if (server.getInfo().isAlive()) { // The server is successfully removed from the conf // It may not be shutdown since it may not be able to talk to the new leader (who is not in its conf). - Assert.assertTrue(conf.isStable()); - Assert.assertFalse(conf.containsInConf(server.getId())); + Assertions.assertTrue(conf.isStable()); + Assertions.assertFalse(conf.containsInConf(server.getId())); } } - Assert.assertEquals(peers.size(), numIncluded + deadIncluded); + Assertions.assertEquals(peers.size(), numIncluded + deadIncluded); } public static long getNextIndex(RaftServer.Division server) { @@ -176,8 +176,8 @@ public static Stream getLogAppenders(RaftServer.Division server) { public static void assertLeaderLease(RaftServer.Division leader, boolean hasLease) { final LeaderStateImpl l = getLeaderState(leader).orElse(null); - Assert.assertNotNull(l); - Assert.assertEquals(l.hasLease(), hasLease); + Assertions.assertNotNull(l); + Assertions.assertEquals(l.hasLease(), hasLease); } public static void restartLogAppenders(RaftServer.Division server) { @@ -200,8 +200,8 @@ public static DataStreamMap newDataStreamMap(Object name) { public static void assertLostMajorityHeartbeatsRecently(RaftServer.Division leader) { final FollowerState f = ((RaftServerImpl)leader).getRole().getFollowerState().orElse(null); - Assert.assertNotNull(f); - Assert.assertTrue(f.lostMajorityHeartbeatsRecently()); + Assertions.assertNotNull(f); + Assertions.assertTrue(f.lostMajorityHeartbeatsRecently()); } public static SegmentedRaftLog newSegmentedRaftLog(RaftGroupMemberId memberId, DivisionInfo info, diff --git a/ratis-server/src/test/java/org/apache/ratis/server/impl/RaftStateMachineExceptionTests.java b/ratis-server/src/test/java/org/apache/ratis/server/impl/RaftStateMachineExceptionTests.java index be857141cd..3a58f4e7c6 100644 --- a/ratis-server/src/test/java/org/apache/ratis/server/impl/RaftStateMachineExceptionTests.java +++ b/ratis-server/src/test/java/org/apache/ratis/server/impl/RaftStateMachineExceptionTests.java @@ -33,15 +33,15 @@ import org.apache.ratis.statemachine.TransactionContext; import org.apache.ratis.util.JavaUtils; import org.apache.ratis.util.Slf4jUtils; -import org.junit.Assert; -import org.junit.Test; +import org.junit.jupiter.api.Assertions; +import org.junit.jupiter.api.Test; import org.slf4j.event.Level; import java.io.IOException; import java.util.Objects; import java.util.concurrent.CompletableFuture; -import static org.junit.Assert.fail; +import static org.junit.jupiter.api.Assertions.fail; public abstract class RaftStateMachineExceptionTests extends BaseTest implements MiniRaftCluster.Factory.Get { @@ -92,7 +92,7 @@ private void runTestHandleStateMachineException(CLUSTER cluster) throws Exceptio fail("Exception expected"); } catch (StateMachineException e) { e.printStackTrace(); - Assert.assertTrue(e.getCause().getMessage().contains("Fake Exception")); + Assertions.assertTrue(e.getCause().getMessage().contains("Fake Exception")); } cluster.shutdown(); } @@ -114,23 +114,23 @@ private void runTestRetryOnStateMachineException(CLUSTER cluster) throws Excepti final SimpleMessage message = new SimpleMessage("message"); final RaftClientRequest r = cluster.newRaftClientRequest(client.getId(), leaderId, callId, message); RaftClientReply reply = rpc.sendRequest(r); - Assert.assertFalse(reply.isSuccess()); - Assert.assertNotNull(reply.getStateMachineException()); + Assertions.assertFalse(reply.isSuccess()); + Assertions.assertNotNull(reply.getStateMachineException()); // retry with the same callId for (int i = 0; i < 5; i++) { reply = rpc.sendRequest(r); - Assert.assertEquals(client.getId(), reply.getClientId()); - Assert.assertEquals(callId, reply.getCallId()); - Assert.assertFalse(reply.isSuccess()); - Assert.assertNotNull(reply.getStateMachineException()); + Assertions.assertEquals(client.getId(), reply.getClientId()); + Assertions.assertEquals(callId, reply.getCallId()); + Assertions.assertFalse(reply.isSuccess()); + Assertions.assertNotNull(reply.getStateMachineException()); } for (RaftServer.Division server : cluster.iterateDivisions()) { LOG.info("check server " + server.getId()); JavaUtils.attemptRepeatedly(() -> { - Assert.assertNotNull(RetryCacheTestUtil.get(server, client.getId(), callId)); + Assertions.assertNotNull(RetryCacheTestUtil.get(server, client.getId(), callId)); return null; }, 5, BaseTest.ONE_SECOND, "GetRetryEntry", LOG); @@ -161,8 +161,8 @@ private void runTestRetryOnExceptionDuringReplication(CLUSTER cluster) throws Ex Objects.requireNonNull(reply.getStateMachineException()); final RetryCache.Entry oldEntry = RetryCacheTestUtil.get(oldLeader, client.getId(), callId); - Assert.assertNotNull(oldEntry); - Assert.assertTrue(RetryCacheTestUtil.isFailed(oldEntry)); + Assertions.assertNotNull(oldEntry); + Assertions.assertTrue(RetryCacheTestUtil.isFailed(oldEntry)); Thread.sleep(100); // At this point of time the old leader would have stepped down. wait for leader election to complete @@ -173,9 +173,9 @@ private void runTestRetryOnExceptionDuringReplication(CLUSTER cluster) throws Ex Objects.requireNonNull(reply.getStateMachineException()); final RetryCache.Entry currentEntry = RetryCacheTestUtil.get(leader, client.getId(), callId); - Assert.assertNotNull(currentEntry); - Assert.assertTrue(RetryCacheTestUtil.isFailed(currentEntry)); - Assert.assertNotEquals(oldEntry, currentEntry); + Assertions.assertNotNull(currentEntry); + Assertions.assertTrue(RetryCacheTestUtil.isFailed(currentEntry)); + Assertions.assertNotEquals(oldEntry, currentEntry); failPreAppend = false; } } diff --git a/ratis-server/src/test/java/org/apache/ratis/server/impl/RetryCacheTestUtil.java b/ratis-server/src/test/java/org/apache/ratis/server/impl/RetryCacheTestUtil.java index 439245b496..f90f914179 100644 --- a/ratis-server/src/test/java/org/apache/ratis/server/impl/RetryCacheTestUtil.java +++ b/ratis-server/src/test/java/org/apache/ratis/server/impl/RetryCacheTestUtil.java @@ -27,7 +27,7 @@ import org.apache.ratis.server.raftlog.segmented.SegmentedRaftLog; import org.apache.ratis.server.storage.RaftStorage; import org.apache.ratis.util.TimeDuration; -import org.junit.Assert; +import org.junit.jupiter.api.Assertions; import java.util.concurrent.TimeUnit; @@ -59,7 +59,7 @@ public static boolean isFailed(RetryCache.Entry entry) { public static void assertFailure(RetryCache cache, LogEntryProto logEntry, boolean isFailed) { if(logEntry.hasStateMachineLogEntry()) { final ClientInvocationId invocationId = ClientInvocationId.valueOf(logEntry.getStateMachineLogEntry()); - Assert.assertEquals(isFailed, get(cache, invocationId).isFailed()); + Assertions.assertEquals(isFailed, get(cache, invocationId).isFailed()); } } diff --git a/ratis-server/src/test/java/org/apache/ratis/server/impl/ServerPauseResumeTest.java b/ratis-server/src/test/java/org/apache/ratis/server/impl/ServerPauseResumeTest.java index d2584c6312..185b0e3cdb 100644 --- a/ratis-server/src/test/java/org/apache/ratis/server/impl/ServerPauseResumeTest.java +++ b/ratis-server/src/test/java/org/apache/ratis/server/impl/ServerPauseResumeTest.java @@ -29,8 +29,8 @@ import org.apache.ratis.server.RaftServer; import org.apache.ratis.server.RaftServerConfigKeys; import org.apache.ratis.server.raftlog.RaftLog; -import org.junit.Assert; -import org.junit.Test; +import org.junit.jupiter.api.Assertions; +import org.junit.jupiter.api.Test; /** Test server pause and resume. */ public abstract class ServerPauseResumeTest @@ -50,7 +50,7 @@ void runTestPauseResume(CLUSTER cluster) throws InterruptedException, IOExceptio final RaftServer.Division leader = waitForLeader(cluster); RaftPeerId leaderId = leader.getId(); final List followers = cluster.getFollowers(); - Assert.assertTrue(followers.size() >= 1); + Assertions.assertTrue(followers.size() >= 1); final RaftServerImpl follower = (RaftServerImpl)followers.get(0); SimpleMessage[] batch1 = SimpleMessage.create(100, "batch1"); @@ -60,15 +60,15 @@ void runTestPauseResume(CLUSTER cluster) throws InterruptedException, IOExceptio Thread.sleep(cluster.getTimeoutMax().toLong(TimeUnit.MILLISECONDS) * 5); final RaftLog leaderLog = leader.getRaftLog(); // leader should contain all logs. - Assert.assertTrue(RaftTestUtil.logEntriesContains(leaderLog, batch1)); + Assertions.assertTrue(RaftTestUtil.logEntriesContains(leaderLog, batch1)); RaftLog followerLog = follower.getRaftLog(); // follower should contain all logs. - Assert.assertTrue(RaftTestUtil.logEntriesContains(followerLog, batch1)); + Assertions.assertTrue(RaftTestUtil.logEntriesContains(followerLog, batch1)); // pause follower. boolean isSuccess = follower.pause(); - Assert.assertTrue(isSuccess); - Assert.assertTrue(follower.getInfo().getLifeCycleState().isPausingOrPaused()); + Assertions.assertTrue(isSuccess); + Assertions.assertTrue(follower.getInfo().getLifeCycleState().isPausingOrPaused()); SimpleMessage[] batch2 = SimpleMessage.create(100, "batch2"); Thread writeThread2 = RaftTestUtil.sendMessageInNewThread(cluster, leaderId, batch2); @@ -76,15 +76,15 @@ void runTestPauseResume(CLUSTER cluster) throws InterruptedException, IOExceptio writeThread2.join(); Thread.sleep(cluster.getTimeoutMax().toLong(TimeUnit.MILLISECONDS) * 5); // paused follower should not have any batch2 message in its raftlog. - Assert.assertTrue(RaftTestUtil.logEntriesNotContains(followerLog, batch2)); + Assertions.assertTrue(RaftTestUtil.logEntriesNotContains(followerLog, batch2)); // resume follower. isSuccess = follower.resume(); - Assert.assertTrue(isSuccess); - Assert.assertFalse(follower.getInfo().getLifeCycleState().isPausingOrPaused()); + Assertions.assertTrue(isSuccess); + Assertions.assertFalse(follower.getInfo().getLifeCycleState().isPausingOrPaused()); Thread.sleep(cluster.getTimeoutMax().toLong(TimeUnit.MILLISECONDS) * 5); // follower should contain all logs. - Assert.assertTrue(RaftTestUtil.logEntriesContains(followerLog, batch2)); + Assertions.assertTrue(RaftTestUtil.logEntriesContains(followerLog, batch2)); } } diff --git a/ratis-server/src/test/java/org/apache/ratis/server/impl/TestRatisServerMetricsBase.java b/ratis-server/src/test/java/org/apache/ratis/server/impl/TestRatisServerMetricsBase.java index a465b3c4dc..2fe2ed0b6e 100644 --- a/ratis-server/src/test/java/org/apache/ratis/server/impl/TestRatisServerMetricsBase.java +++ b/ratis-server/src/test/java/org/apache/ratis/server/impl/TestRatisServerMetricsBase.java @@ -17,7 +17,7 @@ */ package org.apache.ratis.server.impl; -import static org.junit.Assert.assertEquals; +import static org.junit.jupiter.api.Assertions.assertEquals; import java.io.IOException; import java.util.concurrent.CompletableFuture; @@ -32,8 +32,8 @@ import org.apache.ratis.server.RaftServer; import org.apache.ratis.server.metrics.RaftServerMetricsImpl; import org.apache.ratis.util.Slf4jUtils; -import org.junit.Assert; -import org.junit.Test; +import org.junit.jupiter.api.Assertions; +import org.junit.jupiter.api.Test; import org.slf4j.event.Level; /** Tests on Ratis server metrics. */ @@ -66,7 +66,7 @@ void runTestClientFailedRequest(CLUSTER cluster) .setType(RaftClientRequest.staleReadRequestType(Long.MAX_VALUE)) .build(); final CompletableFuture f = leaderImpl.getRaftServer().submitClientRequestAsync(r); - Assert.assertFalse(f.get().isSuccess()); + Assertions.assertFalse(f.get().isSuccess()); assertEquals(1L, ((RaftServerMetricsImpl)leaderImpl.getRaftServerMetrics()) .getNumFailedClientStaleRead().getCount()); } diff --git a/ratis-server/src/test/java/org/apache/ratis/statemachine/RaftSnapshotBaseTest.java b/ratis-server/src/test/java/org/apache/ratis/statemachine/RaftSnapshotBaseTest.java index 507dd63ad5..2c4ac2eeeb 100644 --- a/ratis-server/src/test/java/org/apache/ratis/statemachine/RaftSnapshotBaseTest.java +++ b/ratis-server/src/test/java/org/apache/ratis/statemachine/RaftSnapshotBaseTest.java @@ -49,8 +49,8 @@ import org.apache.ratis.util.JavaUtils; import org.apache.ratis.util.LifeCycle; import org.apache.ratis.util.Slf4jUtils; -import org.junit.jupiter.api.Assertions; import org.junit.jupiter.api.AfterEach; +import org.junit.jupiter.api.Assertions; import org.junit.jupiter.api.BeforeEach; import org.junit.jupiter.api.Test; import org.slf4j.Logger; diff --git a/ratis-server/src/test/java/org/apache/ratis/statemachine/SnapshotManagementTest.java b/ratis-server/src/test/java/org/apache/ratis/statemachine/SnapshotManagementTest.java index dffeb42286..8be669bf12 100644 --- a/ratis-server/src/test/java/org/apache/ratis/statemachine/SnapshotManagementTest.java +++ b/ratis-server/src/test/java/org/apache/ratis/statemachine/SnapshotManagementTest.java @@ -31,9 +31,9 @@ import org.apache.ratis.server.raftlog.RaftLog; import org.apache.ratis.statemachine.impl.SimpleStateMachine4Testing; import org.apache.ratis.util.Slf4jUtils; -import org.junit.Assert; -import org.junit.Before; -import org.junit.Test; +import org.junit.jupiter.api.Assertions; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.Test; import org.slf4j.Logger; import org.slf4j.LoggerFactory; import org.slf4j.event.Level; @@ -51,7 +51,7 @@ public abstract class SnapshotManagementTest static final Logger LOG = LoggerFactory.getLogger(SnapshotManagementTest.class); - @Before + @BeforeEach public void setup() { final RaftProperties p = getProperties(); p.setClass(MiniRaftCluster.STATEMACHINE_CLASS_KEY, @@ -74,18 +74,18 @@ void runTestTakeSnapshot(CLUSTER cluster) throws Exception { try (final RaftClient client = cluster.createClient(leaderId)) { for (int i = 0; i < RaftServerConfigKeys.Snapshot.creationGap(getProperties()); i++) { RaftClientReply reply = client.io().send(new RaftTestUtil.SimpleMessage("m" + i)); - Assert.assertTrue(reply.isSuccess()); + Assertions.assertTrue(reply.isSuccess()); } snapshotReply = client.getSnapshotManagementApi().create(3000); } - Assert.assertTrue(snapshotReply.isSuccess()); + Assertions.assertTrue(snapshotReply.isSuccess()); final long snapshotIndex = snapshotReply.getLogIndex(); LOG.info("snapshotIndex = {}", snapshotIndex); final File snapshotFile = SimpleStateMachine4Testing.get(leader) .getStateMachineStorage().getSnapshotFile(leader.getInfo().getCurrentTerm(), snapshotIndex); - Assert.assertTrue(snapshotFile.exists()); + Assertions.assertTrue(snapshotFile.exists()); } void runTestTakeSnapshotWithConfigurableGap(CLUSTER cluster) throws Exception { @@ -95,29 +95,29 @@ void runTestTakeSnapshotWithConfigurableGap(CLUSTER cluster) throws Exception { try (final RaftClient client = cluster.createClient(leaderId)) { for (int i = 0; i < RaftServerConfigKeys.Snapshot.creationGap(getProperties())/2-1; i++) { RaftClientReply reply = client.io().send(new RaftTestUtil.SimpleMessage("m" + i)); - Assert.assertTrue(reply.isSuccess()); + Assertions.assertTrue(reply.isSuccess()); } - Assert.assertTrue(leader.getStateMachine().getLastAppliedTermIndex().getIndex() + Assertions.assertTrue(leader.getStateMachine().getLastAppliedTermIndex().getIndex() < RaftServerConfigKeys.Snapshot.creationGap(getProperties())); snapshotReply = client.getSnapshotManagementApi(leaderId).create(3000); - Assert.assertTrue(snapshotReply.isSuccess()); - Assert.assertEquals(0,snapshotReply.getLogIndex()); + Assertions.assertTrue(snapshotReply.isSuccess()); + Assertions.assertEquals(0,snapshotReply.getLogIndex()); for (int i = 0; i < RaftServerConfigKeys.Snapshot.creationGap(getProperties())/2-1; i++) { RaftClientReply reply = client.io().send(new RaftTestUtil.SimpleMessage("m" + i)); - Assert.assertTrue(reply.isSuccess()); + Assertions.assertTrue(reply.isSuccess()); } final SnapshotManagementRequest r1 = SnapshotManagementRequest.newCreate(client.getId(), leaderId, cluster.getGroupId(), CallId.getAndIncrement(), 3000); snapshotReply = client.getSnapshotManagementApi(leaderId).create(3000); } - Assert.assertTrue(snapshotReply.isSuccess()); + Assertions.assertTrue(snapshotReply.isSuccess()); final long snapshotIndex = snapshotReply.getLogIndex(); LOG.info("snapshotIndex = {}", snapshotIndex); final File snapshotFile = SimpleStateMachine4Testing.get(leader) .getStateMachineStorage() .getSnapshotFile(leader.getInfo().getCurrentTerm(), snapshotIndex); - Assert.assertTrue(snapshotFile.exists()); + Assertions.assertTrue(snapshotFile.exists()); } void runTestTakeSnapshotOnSpecificServer(CLUSTER cluster) throws Exception { @@ -125,23 +125,23 @@ void runTestTakeSnapshotOnSpecificServer(CLUSTER cluster) throws Exception { final RaftServer.Division leader = RaftTestUtil.waitForLeader(cluster); final RaftServer.Division follower = cluster.getFollowers().get(0); final RaftPeerId followerId = follower.getId(); - Assert.assertTrue(follower.getInfo().isFollower()); + Assertions.assertTrue(follower.getInfo().isFollower()); try (final RaftClient client = cluster.createClient(followerId)) { for (int i = 0; i < RaftServerConfigKeys.Snapshot.creationGap(getProperties()); i++) { RaftClientReply reply = client.io().send(new RaftTestUtil.SimpleMessage("m" + i)); - Assert.assertTrue(reply.isSuccess()); + Assertions.assertTrue(reply.isSuccess()); } snapshotReply = client.getSnapshotManagementApi(followerId).create(3000); } - Assert.assertTrue(snapshotReply.isSuccess()); + Assertions.assertTrue(snapshotReply.isSuccess()); final long snapshotIndex = snapshotReply.getLogIndex(); LOG.info("snapshotIndex = {} on {} server {}", snapshotIndex, follower.getInfo().getCurrentRole(), follower.getId()); final File snapshotFile = SimpleStateMachine4Testing.get(follower) .getStateMachineStorage().getSnapshotFile(follower.getInfo().getCurrentTerm(), snapshotIndex); - Assert.assertTrue(snapshotFile.exists()); + Assertions.assertTrue(snapshotFile.exists()); } @@ -155,22 +155,22 @@ void runTestReceiveLogAndTakeSnapshotOnListener(CLUSTER cluster) throws Exceptio final RaftServer.Division leader = RaftTestUtil.waitForLeader(cluster); final RaftServer.Division listener = cluster.getListeners().get(0); final RaftPeerId listenerId = listener.getId(); - Assert.assertTrue(listener.getInfo().isListener()); + Assertions.assertTrue(listener.getInfo().isListener()); try (final RaftClient client = cluster.createClient(listenerId)) { for (int i = 0; i < RaftServerConfigKeys.Snapshot.creationGap(getProperties()); i++) { RaftClientReply reply = client.io().send(new RaftTestUtil.SimpleMessage("m" + i)); - Assert.assertTrue(reply.isSuccess()); + Assertions.assertTrue(reply.isSuccess()); } snapshotReply = client.getSnapshotManagementApi(listenerId).create(3000); } - Assert.assertTrue(snapshotReply.isSuccess()); + Assertions.assertTrue(snapshotReply.isSuccess()); final long snapshotIndex = snapshotReply.getLogIndex(); LOG.info("snapshotIndex = {} on {} server {}", snapshotIndex, listener.getInfo().getCurrentRole(), listener.getId()); final File snapshotFile = SimpleStateMachine4Testing.get(listener) .getStateMachineStorage().getSnapshotFile(listener.getInfo().getCurrentTerm(), snapshotIndex); - Assert.assertTrue(snapshotFile.exists()); + Assertions.assertTrue(snapshotFile.exists()); } } diff --git a/ratis-test/pom.xml b/ratis-test/pom.xml index 69470bf0db..a47e9bd359 100644 --- a/ratis-test/pom.xml +++ b/ratis-test/pom.xml @@ -151,6 +151,16 @@ junit-jupiter-params test + + org.junit.vintage + junit-vintage-engine + test + + + org.junit.platform + junit-platform-launcher + test + org.mockito mockito-core diff --git a/ratis-test/src/test/java/org/apache/ratis/grpc/TestRaftAsyncWithGrpc.java b/ratis-test/src/test/java/org/apache/ratis/grpc/TestRaftAsyncWithGrpc.java index a12c52f100..9f9cda701c 100644 --- a/ratis-test/src/test/java/org/apache/ratis/grpc/TestRaftAsyncWithGrpc.java +++ b/ratis-test/src/test/java/org/apache/ratis/grpc/TestRaftAsyncWithGrpc.java @@ -18,7 +18,9 @@ package org.apache.ratis.grpc; import org.apache.ratis.RaftAsyncTests; +import org.junit.jupiter.api.Timeout; +@Timeout(100) public class TestRaftAsyncWithGrpc extends RaftAsyncTests implements MiniRaftClusterWithGrpc.FactoryGet { } \ No newline at end of file diff --git a/ratis-test/src/test/java/org/apache/ratis/netty/TestRaftAsyncWithNetty.java b/ratis-test/src/test/java/org/apache/ratis/netty/TestRaftAsyncWithNetty.java index ebaa33d505..c09b07876f 100644 --- a/ratis-test/src/test/java/org/apache/ratis/netty/TestRaftAsyncWithNetty.java +++ b/ratis-test/src/test/java/org/apache/ratis/netty/TestRaftAsyncWithNetty.java @@ -18,7 +18,9 @@ package org.apache.ratis.netty; import org.apache.ratis.RaftAsyncTests; +import org.junit.jupiter.api.Timeout; +@Timeout(100) public class TestRaftAsyncWithNetty extends RaftAsyncTests implements MiniRaftClusterWithNetty.FactoryGet { From 41760077ebe65d713e7144b39b6540cb830aa39e Mon Sep 17 00:00:00 2001 From: Tsz-Wo Nicholas Sze Date: Thu, 22 May 2025 00:59:14 -0700 Subject: [PATCH 184/397] RATIS-2304. SnapshotManager should validate snapshot file path (#1268) --- .../java/org/apache/ratis/util/FileUtils.java | 33 +++++++++++++++++++ .../org/apache/ratis/util/TestFileUtils.java | 26 +++++++++++++++ .../ratis/server/storage/SnapshotManager.java | 8 ++--- 3 files changed, 63 insertions(+), 4 deletions(-) diff --git a/ratis-common/src/main/java/org/apache/ratis/util/FileUtils.java b/ratis-common/src/main/java/org/apache/ratis/util/FileUtils.java index 4b9d9e3b28..315010c626 100644 --- a/ratis-common/src/main/java/org/apache/ratis/util/FileUtils.java +++ b/ratis-common/src/main/java/org/apache/ratis/util/FileUtils.java @@ -60,6 +60,39 @@ static T attempt(CheckedSupplier op, Supplier name) throw } } + /** @return true iff the given dir is an ancestor of the given sub path. */ + static boolean isAncestor(File dir, File sub) throws IOException { + Objects.requireNonNull(dir, "dir == null"); + Objects.requireNonNull(sub, "sub == null"); + + String dirPath = dir.getCanonicalPath(); + final String subPath = sub.getCanonicalPath(); + if (dirPath.equals(subPath)) { + return true; + } else if (!dirPath.endsWith(File.separator)) { + dirPath += File.separator; + } + LOG.debug("dirPath: {}", dirPath); + LOG.debug("subPath: {}", subPath); + return subPath.startsWith(dirPath); + } + + /** + * Resolve the full path from the given dir and sub, + * where dir is supposed to be an ancestor of the resolved path. + * + * @return the full path + * @throws IOException if the dir is not an ancestor of the resolved path. + */ + static File resolveFullPath(File dir, String sub) throws IOException { + final File full = new File(dir, sub); + if (!isAncestor(dir, full)) { + throw new IOException("The dir is not an ancestor of the full path: dir=" + dir + + ", sub=" + sub + ", full=" + full); + } + return full; + } + static void truncateFile(File f, long target) throws IOException { final long original = f.length(); LogUtils.runAndLog(LOG, diff --git a/ratis-common/src/test/java/org/apache/ratis/util/TestFileUtils.java b/ratis-common/src/test/java/org/apache/ratis/util/TestFileUtils.java index 3171756b87..8f64ff294c 100644 --- a/ratis-common/src/test/java/org/apache/ratis/util/TestFileUtils.java +++ b/ratis-common/src/test/java/org/apache/ratis/util/TestFileUtils.java @@ -26,6 +26,32 @@ /** Test methods of {@link FileUtils}. */ public class TestFileUtils extends BaseTest { + @Test + public void testIsAncestor() throws IOException { + runTestIsAncestor(true, "/a", "/a/b"); + runTestIsAncestor(true, "/a", "/a/"); + runTestIsAncestor(true, "/a", "/a"); + runTestIsAncestor(true, "a", "a/b"); + runTestIsAncestor(true, "a", "a/"); + runTestIsAncestor(true, "a", "a"); + + runTestIsAncestor(false, "/a", "/c"); + runTestIsAncestor(false, "/a", "/abc"); + runTestIsAncestor(false, "/a", "/a/../c"); + runTestIsAncestor(false, "a", "a/../c"); + runTestIsAncestor(false, "a", "/c"); + } + + static void runTestIsAncestor(boolean expected, String ancestor, String path) throws IOException { + final boolean computed = isAncestor(ancestor, path); + System.out.printf("isAncestor(%2s, %-9s)? %s, expected? %s%n", + ancestor, path, computed, expected); + Assertions.assertSame(expected, computed); + } + + static boolean isAncestor(String ancestor, String path) throws IOException { + return FileUtils.isAncestor(new File(ancestor), new File(path)); + } @Test public void testRenameToCorrupt() throws IOException { diff --git a/ratis-server/src/main/java/org/apache/ratis/server/storage/SnapshotManager.java b/ratis-server/src/main/java/org/apache/ratis/server/storage/SnapshotManager.java index 91c1ba5b95..2d10c53a45 100644 --- a/ratis-server/src/main/java/org/apache/ratis/server/storage/SnapshotManager.java +++ b/ratis-server/src/main/java/org/apache/ratis/server/storage/SnapshotManager.java @@ -31,6 +31,7 @@ import org.apache.ratis.util.MemoizedSupplier; import org.apache.ratis.util.Preconditions; import org.apache.ratis.util.StringUtils; +import org.apache.ratis.util.function.CheckedFunction; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -43,7 +44,6 @@ import java.nio.file.StandardOpenOption; import java.security.MessageDigest; import java.util.Optional; -import java.util.function.Function; import java.util.function.Supplier; /** @@ -61,7 +61,7 @@ public class SnapshotManager { private final Supplier snapshotDir; private final Supplier snapshotTmpDir; - private final Function getRelativePath; + private final CheckedFunction getRelativePath; private MessageDigest digester; SnapshotManager(RaftPeerId selfId, Supplier dir, StateMachineStorage smStorage) { @@ -73,7 +73,7 @@ public class SnapshotManager { final Supplier smDir = MemoizedSupplier.valueOf(() -> dir.get().getStateMachineDir().toPath()); this.getRelativePath = c -> smDir.get().relativize( - new File(dir.get().getRoot(), c.getFilename()).toPath()).toString(); + FileUtils.resolveFullPath(dir.get().getRoot(), c.getFilename()).toPath()).toString(); } @SuppressWarnings({"squid:S2095"}) // Suppress closeable warning @@ -121,7 +121,7 @@ public void installSnapshot(InstallSnapshotRequestProto request, StateMachine st + " with endIndex >= lastIncludedIndex " + lastIncludedIndex); } - final File tmpSnapshotFile = new File(tmpDir, getRelativePath.apply(chunk)); + final File tmpSnapshotFile = FileUtils.resolveFullPath(tmpDir, getRelativePath.apply(chunk)); FileUtils.createDirectoriesDeleteExistingNonDirectory(tmpSnapshotFile.getParentFile()); try (FileChannel out = open(chunk, tmpSnapshotFile)) { From d71451832ad17a1a0a66aa82b9c6356ee6ca9c52 Mon Sep 17 00:00:00 2001 From: "Doroszlai, Attila" <6454655+adoroszlai@users.noreply.github.com> Date: Thu, 22 May 2025 17:20:00 +0200 Subject: [PATCH 185/397] RATIS-2297. Set default commit message to PR title (#1265) --- .asf.yaml | 1 + 1 file changed, 1 insertion(+) diff --git a/.asf.yaml b/.asf.yaml index 23fccc71e8..677adaeb0d 100644 --- a/.asf.yaml +++ b/.asf.yaml @@ -22,6 +22,7 @@ github: - java enabled_merge_buttons: squash: true + squash_commit_message: PR_TITLE merge: false rebase: false autolink_jira: From b4d7c9a6ab702e210222c56bc4c22dcb99b3eb8f Mon Sep 17 00:00:00 2001 From: slfan1989 <55643692+slfan1989@users.noreply.github.com> Date: Fri, 23 May 2025 00:21:14 +0800 Subject: [PATCH 186/397] RATIS-2303. Migrate ratis-examples tests to Junit 5. (#1267) --- .../filestore/FileStoreAsyncBaseTest.java | 2 +- .../examples/filestore/FileStoreBaseTest.java | 30 +++++++++---------- .../filestore/FileStoreStreamingBaseTest.java | 12 ++++---- .../examples/filestore/FileStoreWriter.java | 20 ++++++------- .../ratis/server/impl/TestReadAfterWrite.java | 20 ++++++------- .../netty/client/NettyClientStreamRpc.java | 5 ++-- .../ratis/datastream/DataStreamTestUtils.java | 2 +- 7 files changed, 45 insertions(+), 46 deletions(-) diff --git a/ratis-examples/src/test/java/org/apache/ratis/examples/filestore/FileStoreAsyncBaseTest.java b/ratis-examples/src/test/java/org/apache/ratis/examples/filestore/FileStoreAsyncBaseTest.java index 0b5071c448..ae3d7fc7b0 100644 --- a/ratis-examples/src/test/java/org/apache/ratis/examples/filestore/FileStoreAsyncBaseTest.java +++ b/ratis-examples/src/test/java/org/apache/ratis/examples/filestore/FileStoreAsyncBaseTest.java @@ -25,7 +25,7 @@ import org.apache.ratis.statemachine.StateMachine; import org.apache.ratis.util.LogUtils; import org.apache.ratis.util.SizeInBytes; -import org.junit.Test; +import org.junit.jupiter.api.Test; import org.slf4j.Logger; import org.slf4j.LoggerFactory; diff --git a/ratis-examples/src/test/java/org/apache/ratis/examples/filestore/FileStoreBaseTest.java b/ratis-examples/src/test/java/org/apache/ratis/examples/filestore/FileStoreBaseTest.java index 5bcc500391..07668e0f2d 100644 --- a/ratis-examples/src/test/java/org/apache/ratis/examples/filestore/FileStoreBaseTest.java +++ b/ratis-examples/src/test/java/org/apache/ratis/examples/filestore/FileStoreBaseTest.java @@ -28,8 +28,8 @@ import org.apache.ratis.util.SizeInBytes; import org.apache.ratis.util.TimeDuration; import org.apache.ratis.util.function.CheckedSupplier; -import org.junit.Assert; -import org.junit.Test; +import org.junit.jupiter.api.Assertions; +import org.junit.jupiter.api.Test; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -96,22 +96,22 @@ void runTestWatch(int n, CLUSTER cluster) throws Exception { }); firstList.add(f); final CompletableFuture s = client.watchAsync(pathSecond + i).whenComplete((reply, e) -> { - Assert.assertNotNull(reply); - Assert.assertNull(e); - Assert.assertTrue(isStarted.get()); + Assertions.assertNotNull(reply); + Assertions.assertNull(e); + Assertions.assertTrue(isStarted.get()); completionOrder.add(index); }); watchSecond.add(s); - Assert.assertFalse(f.isDone()); - Assert.assertFalse(s.isDone()); - Assert.assertFalse(isStarted.get()); + Assertions.assertFalse(f.isDone()); + Assertions.assertFalse(s.isDone()); + Assertions.assertFalse(isStarted.get()); } TimeDuration.valueOf(ThreadLocalRandom.current().nextLong(500) + 100, TimeUnit.MILLISECONDS) .sleep(s -> LOG.info("{}", s)); - firstList.stream().map(CompletableFuture::isDone).forEach(Assert::assertFalse); - watchSecond.stream().map(CompletableFuture::isDone).forEach(Assert::assertFalse); - Assert.assertFalse(isStarted.get()); + firstList.stream().map(CompletableFuture::isDone).forEach(Assertions::assertFalse); + watchSecond.stream().map(CompletableFuture::isDone).forEach(Assertions::assertFalse); + Assertions.assertFalse(isStarted.get()); isStarted.set(true); for (int i : randomIndices) { @@ -121,12 +121,12 @@ void runTestWatch(int n, CLUSTER cluster) throws Exception { for (int i = 0; i < n; i++) { final ReadReplyProto reply = watchSecond.get(i).get(100, TimeUnit.MILLISECONDS); LOG.info("reply {}: {}", i, reply); - Assert.assertNotNull(reply); - Assert.assertEquals(pathSecond + i, reply.getResolvedPath().toStringUtf8()); + Assertions.assertNotNull(reply); + Assertions.assertEquals(pathSecond + i, reply.getResolvedPath().toStringUtf8()); } LOG.info("completionOrder {}", completionOrder); - Assert.assertEquals(randomIndices, completionOrder); - firstList.stream().map(CompletableFuture::isDone).forEach(Assert::assertFalse); + Assertions.assertEquals(randomIndices, completionOrder); + firstList.stream().map(CompletableFuture::isDone).forEach(Assertions::assertFalse); } } diff --git a/ratis-examples/src/test/java/org/apache/ratis/examples/filestore/FileStoreStreamingBaseTest.java b/ratis-examples/src/test/java/org/apache/ratis/examples/filestore/FileStoreStreamingBaseTest.java index 490f12c8da..cdcee0ef0e 100644 --- a/ratis-examples/src/test/java/org/apache/ratis/examples/filestore/FileStoreStreamingBaseTest.java +++ b/ratis-examples/src/test/java/org/apache/ratis/examples/filestore/FileStoreStreamingBaseTest.java @@ -30,8 +30,8 @@ import org.apache.ratis.util.LogUtils; import org.apache.ratis.util.SizeInBytes; import org.apache.ratis.util.function.CheckedSupplier; -import org.junit.Assert; -import org.junit.Test; +import org.junit.jupiter.api.Assertions; +import org.junit.jupiter.api.Test; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -69,7 +69,7 @@ public void testFileStoreStreamSingleFile() throws Exception { final RaftGroup raftGroup = cluster.getGroup(); final Collection peers = raftGroup.getPeers(); - Assert.assertEquals(NUM_PEERS, peers.size()); + Assertions.assertEquals(NUM_PEERS, peers.size()); RaftPeer primary = peers.iterator().next(); final CheckedSupplier newClient = @@ -91,7 +91,7 @@ public void testFileStoreStreamMultipleFiles() throws Exception { final RaftGroup raftGroup = cluster.getGroup(); final Collection peers = raftGroup.getPeers(); - Assert.assertEquals(NUM_PEERS, peers.size()); + Assertions.assertEquals(NUM_PEERS, peers.size()); RaftPeer primary = peers.iterator().next(); final CheckedSupplier newClient = @@ -114,7 +114,7 @@ private void testSingleFile( .setFileSize(fileLength) .setBufferSize(bufferSize) .setFileStoreClientSupplier(newClient) - .build().streamWriteAndVerify(routingTable); + .build().streamWriteAndVerify(routingTable).close(); } private void testMultipleFiles(String pathBase, int numFile, SizeInBytes fileLength, @@ -136,7 +136,7 @@ private void testMultipleFiles(String pathBase, int numFile, SizeInBytes fileLen writerFutures.add(executor.submit(callable)); } for (Future future : writerFutures) { - future.get(); + future.get().close(); } } } diff --git a/ratis-examples/src/test/java/org/apache/ratis/examples/filestore/FileStoreWriter.java b/ratis-examples/src/test/java/org/apache/ratis/examples/filestore/FileStoreWriter.java index c2b52fa633..c0f7d08097 100644 --- a/ratis-examples/src/test/java/org/apache/ratis/examples/filestore/FileStoreWriter.java +++ b/ratis-examples/src/test/java/org/apache/ratis/examples/filestore/FileStoreWriter.java @@ -29,7 +29,7 @@ import org.apache.ratis.util.SizeInBytes; import org.apache.ratis.util.StringUtils; import org.apache.ratis.util.function.CheckedSupplier; -import org.junit.Assert; +import org.junit.jupiter.api.Assertions; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -129,7 +129,7 @@ FileStoreWriter write(boolean sync) throws IOException { LOG.trace("write {}, offset={}, length={}, close? {}", fileName, offset, length, close); final long written = client.write(fileName, offset, close, b, sync); - Assert.assertEquals(length, written); + Assertions.assertEquals(length, written); offset += length; } return this; @@ -156,15 +156,15 @@ public FileStoreWriter streamWriteAndVerify(RoutingTable routingTable) { } DataStreamReply reply = dataStreamOutput.closeAsync().join(); - Assert.assertTrue(reply.isSuccess()); + Assertions.assertTrue(reply.isSuccess()); // TODO: handle when any of the writeAsync has failed. // check writeAsync requests for (int i = 0; i < futures.size(); i++) { reply = futures.get(i).join(); - Assert.assertTrue(reply.isSuccess()); - Assert.assertEquals(sizes.get(i).longValue(), reply.getBytesWritten()); - Assert.assertEquals(reply.getType(), RaftProtos.DataStreamPacketHeaderProto.Type.STREAM_DATA); + Assertions.assertTrue(reply.isSuccess()); + Assertions.assertEquals(sizes.get(i).longValue(), reply.getBytesWritten()); + Assertions.assertEquals(reply.getType(), RaftProtos.DataStreamPacketHeaderProto.Type.STREAM_DATA); } return this; @@ -192,7 +192,7 @@ CompletableFuture writeAsync(boolean sync) { LOG.trace("writeAsync {}, offset={}, length={}, close? {}", fileName, offset, length, close); client.writeAsync(fileName, offset, close, b, sync) - .thenAcceptAsync(written -> Assert.assertEquals(length, (long)written), asyncExecutor) + .thenAcceptAsync(written -> Assertions.assertEquals(length, (long)written), asyncExecutor) .thenRun(() -> { final int count = callCount.decrementAndGet(); LOG.trace("writeAsync {}, offset={}, length={}, close? {}: n={}, callCount={}", @@ -256,12 +256,12 @@ CompletableFuture verifyAsync() { return null; }); } - Assert.assertEquals(size, n.get()); + Assertions.assertEquals(size, n.get()); return returnFuture; } void verify(ByteString read, int offset, int length, ByteBuffer expected) { - Assert.assertEquals(length, read.size()); + Assertions.assertEquals(length, read.size()); assertBuffers(offset, length, expected, read.asReadOnlyByteBuffer()); } @@ -282,7 +282,7 @@ public void close() throws IOException { static void assertBuffers(int offset, int length, ByteBuffer expected, ByteBuffer computed) { try { - Assert.assertEquals(expected, computed); + Assertions.assertEquals(expected, computed); } catch(AssertionError e) { LOG.error("Buffer mismatched at offset=" + offset + ", length=" + length + "\n expected = " + StringUtils.bytes2HexString(expected) diff --git a/ratis-examples/src/test/java/org/apache/ratis/server/impl/TestReadAfterWrite.java b/ratis-examples/src/test/java/org/apache/ratis/server/impl/TestReadAfterWrite.java index f515628c97..025c7ed0db 100644 --- a/ratis-examples/src/test/java/org/apache/ratis/server/impl/TestReadAfterWrite.java +++ b/ratis-examples/src/test/java/org/apache/ratis/server/impl/TestReadAfterWrite.java @@ -33,9 +33,9 @@ import org.apache.ratis.statemachine.StateMachine; import org.apache.ratis.util.CodeInjectionForTesting; import org.apache.ratis.util.Slf4jUtils; -import org.junit.Assert; -import org.junit.Before; -import org.junit.Test; +import org.junit.jupiter.api.Assertions; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.Test; import org.slf4j.event.Level; import java.util.concurrent.CompletableFuture; @@ -48,7 +48,7 @@ public class TestReadAfterWrite extends BaseTest implements MiniRaftClusterWithGrpc.FactoryGet { - @Before + @BeforeEach public void setup() { Slf4jUtils.setLogLevel(ArithmeticStateMachine.LOG, Level.DEBUG); Slf4jUtils.setLogLevel(CodeInjectionForTesting.LOG, Level.DEBUG); @@ -108,7 +108,7 @@ void runTestReadAfterWrite(RaftClient client) throws Exception { final AsyncApi async = client.async(); final int initialValue = 10; final RaftClientReply assign = async.send(a.assign(new DoubleValue(initialValue))).join(); - Assert.assertTrue(assign.isSuccess()); + Assertions.assertTrue(assign.isSuccess()); final Message query = Expression.Utils.toMessage(a); assertReply(async.sendReadOnly(query), initialValue); @@ -131,15 +131,15 @@ void runTestReadAfterWrite(RaftClient client) throws Exception { final RaftClientReply reply = readAfterWrite.get(100, TimeUnit.MILLISECONDS); final DoubleValue result = (DoubleValue) Expression.Utils.bytes2Expression( reply.getMessage().getContent().toByteArray(), 0); - Assert.fail("result=" + result + ", reply=" + reply); + Assertions.fail("result=" + result + ", reply=" + reply); } catch (TimeoutException e) { LOG.info("Good", e); } // plus2 should still be blocked. - Assert.assertFalse(plus2.isDone()); + Assertions.assertFalse(plus2.isDone()); // readAfterWrite should still be blocked. - Assert.assertFalse(readAfterWrite.isDone()); + Assertions.assertFalse(readAfterWrite.isDone()); // unblock plus2 blockingCode.complete(); @@ -151,10 +151,10 @@ void runTestReadAfterWrite(RaftClient client) throws Exception { void assertReply(CompletableFuture future, int expected) { LOG.info("assertReply, expected {}", expected); final RaftClientReply reply = future.join(); - Assert.assertTrue(reply.isSuccess()); + Assertions.assertTrue(reply.isSuccess()); LOG.info("reply {}", reply); final DoubleValue result = (DoubleValue) Expression.Utils.bytes2Expression( reply.getMessage().getContent().toByteArray(), 0); - Assert.assertEquals(expected, (int) (double) result.evaluate(null)); + Assertions.assertEquals(expected, (int) (double) result.evaluate(null)); } } diff --git a/ratis-netty/src/main/java/org/apache/ratis/netty/client/NettyClientStreamRpc.java b/ratis-netty/src/main/java/org/apache/ratis/netty/client/NettyClientStreamRpc.java index 534fcc5818..5e111daff8 100644 --- a/ratis-netty/src/main/java/org/apache/ratis/netty/client/NettyClientStreamRpc.java +++ b/ratis-netty/src/main/java/org/apache/ratis/netty/client/NettyClientStreamRpc.java @@ -339,15 +339,14 @@ public void channelRead(ChannelHandlerContext ctx, Object msg) { try { replyMap.receiveReply(reply); } catch (Throwable cause) { - LOG.warn(name + ": channelRead error:", cause); + LOG.warn("{} : channelRead error:", name, cause); replyMap.completeExceptionally(cause); } } @Override public void exceptionCaught(ChannelHandlerContext ctx, Throwable cause) { - LOG.warn(name + ": exceptionCaught", cause); - + LOG.warn("{} : exceptionCaught", name, cause); ctx.close(); } diff --git a/ratis-test/src/test/java/org/apache/ratis/datastream/DataStreamTestUtils.java b/ratis-test/src/test/java/org/apache/ratis/datastream/DataStreamTestUtils.java index a2c3162f55..989b6cd2b2 100644 --- a/ratis-test/src/test/java/org/apache/ratis/datastream/DataStreamTestUtils.java +++ b/ratis-test/src/test/java/org/apache/ratis/datastream/DataStreamTestUtils.java @@ -152,7 +152,7 @@ class MultiDataStreamStateMachine extends BaseStateMachine { @Override public CompletableFuture stream(RaftClientRequest request) { final SingleDataStream s = new SingleDataStream(request); - LOG.info("XXX {} put {}, {}", this, ClientInvocationId.valueOf(request), s); + LOG.info("{} put {}, {}", this, ClientInvocationId.valueOf(request), s); streams.put(ClientInvocationId.valueOf(request), s); return CompletableFuture.completedFuture(s); } From be0ebb06e79b63f5736e4825dbb9cc118b21c0ec Mon Sep 17 00:00:00 2001 From: slfan1989 <55643692+slfan1989@users.noreply.github.com> Date: Fri, 23 May 2025 16:03:05 +0800 Subject: [PATCH 187/397] RATIS-1977. Remove Junit 4 dependencies. (#1269) --- pom.xml | 19 +-- ratis-common/pom.xml | 11 -- .../test/java/org/apache/ratis/BaseTest.java | 9 -- ratis-examples/pom.xml | 5 - ratis-proto/pom.xml | 4 +- ratis-server/pom.xml | 10 -- .../apache/ratis/RaftExceptionBaseTest.java | 4 +- .../java/org/apache/ratis/RaftTestUtil.java | 11 +- .../server/impl/LeaderElectionTests.java | 147 +++++++++--------- .../impl/StateMachineShutdownTests.java | 25 +-- ratis-test/pom.xml | 10 -- 11 files changed, 103 insertions(+), 152 deletions(-) diff --git a/pom.xml b/pom.xml index 1a85348a2b..4d8dfe1e61 100644 --- a/pom.xml +++ b/pom.xml @@ -221,8 +221,10 @@ 4 2.0.7 - 5.11.2 + 5.12.2 + 4.11.0 0.8.12 + 1.3.5 flaky | org.apache.ratis.test.tag.FlakyTest @@ -417,12 +419,6 @@ test ${slf4j.version} - - - junit - junit - 4.13.2 - org.junit junit-bom @@ -433,13 +429,12 @@ org.mockito mockito-core - 4.3.1 + ${mockito.version} - org.apache.tomcat - annotations-api - 6.0.53 - provided + jakarta.annotation + jakarta.annotation-api + ${jakarta.annotation.version} diff --git a/ratis-common/pom.xml b/ratis-common/pom.xml index a09f446798..b040f87964 100644 --- a/ratis-common/pom.xml +++ b/ratis-common/pom.xml @@ -38,12 +38,6 @@ slf4j-api - - junit - junit - test - - org.junit.jupiter junit-jupiter-api @@ -54,11 +48,6 @@ junit-jupiter-engine test - - org.junit.vintage - junit-vintage-engine - test - org.junit.platform junit-platform-launcher diff --git a/ratis-common/src/test/java/org/apache/ratis/BaseTest.java b/ratis-common/src/test/java/org/apache/ratis/BaseTest.java index ff912879d7..52b986ab9b 100644 --- a/ratis-common/src/test/java/org/apache/ratis/BaseTest.java +++ b/ratis-common/src/test/java/org/apache/ratis/BaseTest.java @@ -25,8 +25,6 @@ import org.apache.ratis.util.StringUtils; import org.apache.ratis.util.TimeDuration; import org.apache.ratis.util.function.CheckedRunnable; -import org.junit.After; -import org.junit.Before; import org.junit.jupiter.api.AfterEach; import org.junit.jupiter.api.Assertions; import org.junit.jupiter.api.Assumptions; @@ -71,8 +69,6 @@ public void setFirstException(Throwable e) { } } - // TODO: Junit 4 reference should be removed once all the unit tests are migrated to Junit 5. - private String testCaseName; @BeforeEach @@ -84,8 +80,6 @@ public void setup(TestInfo testInfo) { + "." + (method == null? null : method.getName()); } - // @Before annotation is retained to support junit 4 tests. - @Before @BeforeEach public void checkAssumptions() { final Throwable first = firstException.get(); @@ -95,8 +89,6 @@ public void checkAssumptions() { Assumptions.assumeTrue(exited == null, () -> "Already exited with " + exited); } - // @After annotation is retained to support junit 4 tests. - @After @AfterEach public void assertNoFailures() { final Throwable e = firstException.get(); @@ -129,7 +121,6 @@ public File getClassTestDir() { } public File getTestDir() { - // This will work for both junit 4 and 5. return new File(getClassTestDir(), testCaseName); } diff --git a/ratis-examples/pom.xml b/ratis-examples/pom.xml index 6b734a9651..d13e68df5a 100644 --- a/ratis-examples/pom.xml +++ b/ratis-examples/pom.xml @@ -117,11 +117,6 @@ runtime - - junit - junit - test - org.junit.jupiter junit-jupiter-engine diff --git a/ratis-proto/pom.xml b/ratis-proto/pom.xml index 9e65096e44..9a548b4066 100644 --- a/ratis-proto/pom.xml +++ b/ratis-proto/pom.xml @@ -176,8 +176,8 @@ ratis-thirdparty-misc - org.apache.tomcat - annotations-api + jakarta.annotation + jakarta.annotation-api diff --git a/ratis-server/pom.xml b/ratis-server/pom.xml index 22041f0ee0..54ea8e1deb 100644 --- a/ratis-server/pom.xml +++ b/ratis-server/pom.xml @@ -65,11 +65,6 @@ slf4j-api - - junit - junit - test - org.junit.jupiter junit-jupiter-api @@ -80,11 +75,6 @@ junit-jupiter-engine test - - org.junit.vintage - junit-vintage-engine - test - org.junit.platform junit-platform-launcher diff --git a/ratis-server/src/test/java/org/apache/ratis/RaftExceptionBaseTest.java b/ratis-server/src/test/java/org/apache/ratis/RaftExceptionBaseTest.java index 3d2837cd6b..9f602e0815 100644 --- a/ratis-server/src/test/java/org/apache/ratis/RaftExceptionBaseTest.java +++ b/ratis-server/src/test/java/org/apache/ratis/RaftExceptionBaseTest.java @@ -34,7 +34,7 @@ import org.apache.ratis.util.Slf4jUtils; import org.apache.ratis.util.SizeInBytes; import org.junit.jupiter.api.Assertions; -import org.junit.Assume; +import org.junit.jupiter.api.Assumptions; import org.junit.jupiter.api.Test; import org.slf4j.event.Level; @@ -84,7 +84,7 @@ RaftClientReply assertNotLeaderException(RaftPeerId expectedSuggestedLeader, final SimpleMessage message = new SimpleMessage(messageId); final RaftClientReply reply = rpc.sendRequest(cluster.newRaftClientRequest(ClientId.randomId(), server, message)); Assertions.assertNotNull(reply); - Assume.assumeFalse(reply.isSuccess()); + Assumptions.assumeFalse(reply.isSuccess()); final NotLeaderException nle = reply.getNotLeaderException(); Objects.requireNonNull(nle); Assertions.assertEquals(expectedSuggestedLeader, nle.getSuggestedLeader().getId()); diff --git a/ratis-server/src/test/java/org/apache/ratis/RaftTestUtil.java b/ratis-server/src/test/java/org/apache/ratis/RaftTestUtil.java index f8d92cf08a..40a93df17f 100644 --- a/ratis-server/src/test/java/org/apache/ratis/RaftTestUtil.java +++ b/ratis-server/src/test/java/org/apache/ratis/RaftTestUtil.java @@ -43,7 +43,8 @@ import org.apache.ratis.util.Preconditions; import org.apache.ratis.util.ProtoUtils; import org.apache.ratis.util.TimeDuration; -import org.junit.AssumptionViolatedException; +import org.apache.ratis.util.function.CheckedConsumer; +import org.junit.jupiter.api.Assumptions; import org.junit.jupiter.api.Assertions; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -475,18 +476,18 @@ static List getPeersWithPriority(List peers, RaftPeer sugges static RaftPeerId changeLeader(MiniRaftCluster cluster, RaftPeerId oldLeader) throws Exception { - return changeLeader(cluster, oldLeader, AssumptionViolatedException::new); + return changeLeader(cluster, oldLeader, Assumptions::abort); } - static RaftPeerId changeLeader(MiniRaftCluster cluster, RaftPeerId oldLeader, Function constructor) - throws Exception { + static RaftPeerId changeLeader(MiniRaftCluster cluster, RaftPeerId oldLeader, + CheckedConsumer failToChangeLeaderHandler) throws Exception { final String name = JavaUtils.getCallerStackTraceElement().getMethodName() + "-changeLeader"; cluster.setBlockRequestsFrom(oldLeader.toString(), true); try { return JavaUtils.attemptRepeatedly(() -> { final RaftPeerId newLeader = waitForLeader(cluster).getId(); if (newLeader.equals(oldLeader)) { - throw constructor.apply("Failed to change leader: newLeader == oldLeader == " + oldLeader); + failToChangeLeaderHandler.accept("Failed to change leader: newLeader == oldLeader == " + oldLeader); } LOG.info("Changed leader from " + oldLeader + " to " + newLeader); return newLeader; diff --git a/ratis-server/src/test/java/org/apache/ratis/server/impl/LeaderElectionTests.java b/ratis-server/src/test/java/org/apache/ratis/server/impl/LeaderElectionTests.java index edb0a01bd3..8a17fd135b 100644 --- a/ratis-server/src/test/java/org/apache/ratis/server/impl/LeaderElectionTests.java +++ b/ratis-server/src/test/java/org/apache/ratis/server/impl/LeaderElectionTests.java @@ -38,7 +38,6 @@ import org.apache.ratis.server.metrics.LeaderElectionMetrics; import org.apache.ratis.server.protocol.TermIndex; import org.apache.ratis.server.raftlog.segmented.SegmentedRaftLogTestUtils; -import org.apache.ratis.util.ExitUtils; import org.apache.ratis.test.tag.Flaky; import org.apache.ratis.thirdparty.com.codahale.metrics.Timer; import org.apache.ratis.util.CodeInjectionForTesting; @@ -48,7 +47,6 @@ import org.apache.ratis.util.TimeDuration; import org.apache.ratis.util.Timestamp; import org.apache.ratis.util.function.CheckedBiConsumer; -import org.junit.Assert; import org.junit.jupiter.api.Assertions; import org.junit.jupiter.api.Test; import org.slf4j.Logger; @@ -71,10 +69,10 @@ import static org.apache.ratis.server.metrics.LeaderElectionMetrics.LEADER_ELECTION_COUNT_METRIC; import static org.apache.ratis.server.metrics.LeaderElectionMetrics.LEADER_ELECTION_TIME_TAKEN; import static org.apache.ratis.server.metrics.LeaderElectionMetrics.LEADER_ELECTION_TIMEOUT_COUNT_METRIC; -import static org.junit.Assert.assertEquals; -import static org.junit.Assert.assertTrue; -import static org.junit.Assert.fail; -import static org.junit.Assert.assertNotNull; +import static org.junit.jupiter.api.Assertions.assertEquals; +import static org.junit.jupiter.api.Assertions.assertNotNull; +import static org.junit.jupiter.api.Assertions.assertTrue; +import static org.junit.jupiter.api.Assertions.fail; import static org.mockito.Mockito.mock; import static org.mockito.Mockito.when; @@ -136,10 +134,10 @@ public void testWaitServerReady() throws Exception { LOG.info("{} isRunning at {}ms? {}", server.getId(), startTime.elapsedTimeMs(), isRunning); // Leader will be elected if the server is ready - Assertions.assertNotNull(waitForLeader(cluster), "No leader is elected."); + assertNotNull(waitForLeader(cluster), "No leader is elected."); final long elapsedMs = startTime.elapsedTimeMs(); // allow a small difference to tolerate system timer inaccuracy - Assertions.assertTrue(elapsedMs > sleepMs - 10, () -> "elapseMs = " + elapsedMs + " but sleepMs = " + sleepMs); + assertTrue(elapsedMs > sleepMs - 10, () -> "elapseMs = " + elapsedMs + " but sleepMs = " + sleepMs); cluster.shutdown(); CodeInjectionForTesting.remove(RaftServerImpl.START_COMPLETE); } @@ -154,7 +152,7 @@ public void testAddServerForWaitReady() throws IOException, InterruptedException try (RaftClient client = cluster.createClient()) { for (int i = 0; i < 10; ++i) { RaftClientReply reply = client.io().send(new RaftTestUtil.SimpleMessage("message_" + i)); - Assertions.assertTrue(reply.isSuccess()); + assertTrue(reply.isSuccess()); } // add 3 new servers and wait longer time CodeInjectionForTesting.put(RaftServerImpl.START_COMPLETE, new SleepCode(2000)); @@ -164,11 +162,11 @@ public void testAddServerForWaitReady() throws IOException, InterruptedException RaftClientReply reply = client.admin().setConfiguration(SetConfigurationRequest.Arguments.newBuilder() .setServersInNewConf(peerChanges.newPeers) .setMode(SetConfigurationRequest.Mode.ADD).build()); - Assert.assertTrue(reply.isSuccess()); + assertTrue(reply.isSuccess()); for (RaftServer server : cluster.getServers()) { RaftServerProxy proxy = (RaftServerProxy) server; proxy.getImpls().forEach(s -> { - Assertions.assertTrue(s.isRunning()); + assertTrue(s.isRunning()); }); } } @@ -185,8 +183,7 @@ public void testChangeLeader() throws Exception { RaftPeerId leader = RaftTestUtil.waitForLeader(cluster).getId(); for(int i = 0; i < 10; i++) { - leader = RaftTestUtil.changeLeader(cluster, leader, IllegalStateException::new); - ExitUtils.assertNotTerminated(); + leader = RaftTestUtil.changeLeader(cluster, leader, Assertions::fail); } SegmentedRaftLogTestUtils.setRaftLogWorkerLogLevel(Level.INFO); cluster.shutdown(); @@ -217,18 +214,18 @@ public void testLeaderNotCountListenerForMajority() throws Exception { void runTestLeaderNotCountListenerForMajority(CLUSTER cluster) throws Exception { final RaftServer.Division leader = waitForLeader(cluster); - Assertions.assertEquals(2, ((RaftConfigurationImpl)cluster.getLeader().getRaftConf()).getMajorityCount()); + assertEquals(2, ((RaftConfigurationImpl)cluster.getLeader().getRaftConf()).getMajorityCount()); try (RaftClient client = cluster.createClient(leader.getId())) { client.io().send(new RaftTestUtil.SimpleMessage("message")); List listeners = cluster.getListeners() .stream().map(RaftServer.Division::getPeer).collect(Collectors.toList()); - Assertions.assertEquals(2, listeners.size()); + assertEquals(2, listeners.size()); RaftClientReply reply = client.admin().setConfiguration(cluster.getPeers()); - Assertions.assertTrue(reply.isSuccess()); + assertTrue(reply.isSuccess()); Collection peer = leader.getRaftConf().getAllPeers(RaftProtos.RaftPeerRole.LISTENER); - Assertions.assertEquals(0, peer.size()); + assertEquals(0, peer.size()); } - Assertions.assertEquals(3, ((RaftConfigurationImpl)cluster.getLeader().getRaftConf()).getMajorityCount()); + assertEquals(3, ((RaftConfigurationImpl)cluster.getLeader().getRaftConf()).getMajorityCount()); } @Test @@ -246,7 +243,7 @@ void runTestListenerNotStartLeaderElection(CLUSTER cluster) throws Exception { RaftTestUtil.isolate(cluster, listenerId); maxTimeout.sleep(); maxTimeout.sleep(); - Assertions.assertEquals(RaftProtos.RaftPeerRole.LISTENER, listener.getInfo().getCurrentRole()); + assertEquals(RaftProtos.RaftPeerRole.LISTENER, listener.getInfo().getCurrentRole()); } finally { RaftTestUtil.deIsolate(cluster, listener.getId()); } @@ -262,18 +259,18 @@ public void testTransferLeader() throws Exception { client.io().send(new RaftTestUtil.SimpleMessage("message")); List followers = cluster.getFollowers(); - Assertions.assertEquals(2, followers.size()); + assertEquals(2, followers.size()); RaftServer.Division newLeader = followers.get(0); RaftClientReply reply = client.admin().transferLeadership(newLeader.getId(), 20000); - Assertions.assertTrue(reply.isSuccess()); + assertTrue(reply.isSuccess()); final RaftServer.Division currLeader = waitForLeader(cluster); - Assertions.assertEquals(newLeader.getId(), currLeader.getId()); + assertEquals(newLeader.getId(), currLeader.getId()); reply = client.io().send(new RaftTestUtil.SimpleMessage("message")); - Assertions.assertEquals(newLeader.getId().toString(), reply.getReplierId()); - Assertions.assertTrue(reply.isSuccess()); + assertEquals(newLeader.getId().toString(), reply.getReplierId()); + assertTrue(reply.isSuccess()); } cluster.shutdown(); @@ -290,24 +287,24 @@ public void testYieldLeaderToHigherPriority() throws Exception { client.io().send(new RaftTestUtil.SimpleMessage("message")); List followers = cluster.getFollowers(); - Assertions.assertEquals(2, followers.size()); + assertEquals(2, followers.size()); RaftServer.Division newLeader = followers.get(0); List peers = cluster.getPeers(); List peersWithNewPriority = getPeersWithPriority(peers, newLeader.getPeer()); RaftClientReply reply = client.admin().setConfiguration(peersWithNewPriority.toArray(new RaftPeer[0])); - Assertions.assertTrue(reply.isSuccess()); + assertTrue(reply.isSuccess()); // Wait the old leader to step down. // TODO: make it more deterministic. TimeDuration.valueOf(1, TimeUnit.SECONDS).sleep(); final RaftServer.Division currLeader = waitForLeader(cluster); - Assertions.assertEquals(newLeader.getId(), currLeader.getId()); + assertEquals(newLeader.getId(), currLeader.getId()); reply = client.io().send(new RaftTestUtil.SimpleMessage("message")); - Assertions.assertEquals(newLeader.getId().toString(), reply.getReplierId()); - Assertions.assertTrue(reply.isSuccess()); + assertEquals(newLeader.getId().toString(), reply.getReplierId()); + assertTrue(reply.isSuccess()); } cluster.shutdown(); @@ -322,7 +319,7 @@ public void testTransferLeaderTimeout() throws Exception { final RaftServer.Division leader = waitForLeader(cluster); try (RaftClient client = cluster.createClient(leader.getId())) { List followers = cluster.getFollowers(); - Assertions.assertEquals(followers.size(), 2); + assertEquals(followers.size(), 2); RaftServer.Division newLeader = followers.get(0); // isolate new leader, so that transfer leadership will timeout @@ -338,9 +335,9 @@ public void testTransferLeaderTimeout() throws Exception { client.admin().transferLeadership(newLeader.getId(), timeoutMs); } catch (TransferLeadershipException e) { long cost = System.currentTimeMillis() - start; - Assertions.assertTrue(cost > timeoutMs); - Assertions.assertTrue(e.getMessage().contains("Failed to transfer leadership to")); - Assertions.assertTrue(e.getMessage().contains(TransferLeadership.Result.Type.TIMED_OUT.toString())); + assertTrue(cost > timeoutMs); + assertTrue(e.getMessage().contains("Failed to transfer leadership to")); + assertTrue(e.getMessage().contains(TransferLeadership.Result.Type.TIMED_OUT.toString())); } return true; @@ -354,17 +351,17 @@ public void testTransferLeaderTimeout() throws Exception { try { client.io().send(new RaftTestUtil.SimpleMessage("message")); } catch (LeaderSteppingDownException e) { - Assertions.assertTrue(e.getMessage().contains("is stepping down")); + assertTrue(e.getMessage().contains("is stepping down")); } return null; }, 5, TimeDuration.ONE_SECOND, "check leader steppingDown", RaftServer.LOG); - Assertions.assertTrue(transferTimeoutFuture.get()); + assertTrue(transferTimeoutFuture.get()); // after transfer timeout, leader should accept request RaftClientReply reply = client.io().send(new RaftTestUtil.SimpleMessage("message")); - Assertions.assertEquals(leader.getId().toString(), reply.getReplierId()); - Assertions.assertTrue(reply.isSuccess()); + assertEquals(leader.getId().toString(), reply.getReplierId()); + assertTrue(reply.isSuccess()); RaftTestUtil.deIsolate(cluster, newLeader.getId()); } @@ -402,7 +399,7 @@ static void enforceLeader(MiniRaftCluster cluster, final String newLeader, Logge log.info(cluster.printServers()); final RaftServer.Division leader = cluster.getLeader(); - Assertions.assertEquals(newLeader, leader.getId().toString()); + assertEquals(newLeader, leader.getId().toString()); } @Test @@ -431,7 +428,7 @@ public void testLateServerStart() throws Exception { .orElseThrow(() -> new IllegalStateException("No leader yet")), 10, ONE_SECOND, "getLeaderId", LOG); LOG.info(cluster.printServers()); - Assertions.assertEquals(leader.getId(), lastServerLeaderId); + assertEquals(leader.getId(), lastServerLeaderId); cluster.shutdown(); } @@ -446,7 +443,7 @@ protected void testDisconnectLeader() throws Exception { RaftTestUtil.isolate(cluster, leader.getId()); RaftClientReply reply = client.io().send(new RaftTestUtil.SimpleMessage("message")); Assertions.assertNotEquals(reply.getReplierId(), leader.getId().toString()); - Assertions.assertTrue(reply.isSuccess()); + assertTrue(reply.isSuccess()); } finally { RaftTestUtil.deIsolate(cluster, leader.getId()); } @@ -463,15 +460,15 @@ public void testAddListener() throws Exception { try (RaftClient client = cluster.createClient(leader.getId())) { client.io().send(new RaftTestUtil.SimpleMessage("message")); List servers = cluster.getPeers(); - Assertions.assertEquals(servers.size(), 3); + assertEquals(servers.size(), 3); MiniRaftCluster.PeerChanges changes = cluster.addNewPeers(1, true, false, RaftProtos.RaftPeerRole.LISTENER); RaftClientReply reply = client.admin().setConfiguration(servers, Arrays.asList(changes.newPeers)); - Assertions.assertTrue(reply.isSuccess()); + assertTrue(reply.isSuccess()); Collection listener = leader.getRaftConf().getAllPeers(RaftProtos.RaftPeerRole.LISTENER); - Assertions.assertEquals(1, listener.size()); - Assertions.assertEquals(changes.newPeers[0].getId(), new ArrayList<>(listener).get(0).getId()); + assertEquals(1, listener.size()); + assertEquals(changes.newPeers[0].getId(), new ArrayList<>(listener).get(0).getId()); } cluster.shutdown(); } @@ -485,18 +482,18 @@ public void testAddFollowerWhenExistsListener() throws Exception { try (RaftClient client = cluster.createClient(leader.getId())) { client.io().send(new RaftTestUtil.SimpleMessage("message")); List servers = cluster.getPeers(); - Assertions.assertEquals(4, servers.size()); + assertEquals(4, servers.size()); List listener = new ArrayList<>( leader.getRaftConf().getAllPeers(RaftProtos.RaftPeerRole.LISTENER)); - Assertions.assertEquals(1, listener.size()); + assertEquals(1, listener.size()); MiniRaftCluster.PeerChanges changes = cluster.addNewPeers(1, true, false); ArrayList newPeers = new ArrayList<>(Arrays.asList(changes.newPeers)); newPeers.addAll(leader.getRaftConf().getAllPeers(RaftProtos.RaftPeerRole.FOLLOWER)); RaftClientReply reply = client.admin().setConfiguration(newPeers, listener); - Assertions.assertTrue(reply.isSuccess()); - Assertions.assertEquals(4, + assertTrue(reply.isSuccess()); + assertEquals(4, leader.getRaftConf().getAllPeers(RaftProtos.RaftPeerRole.FOLLOWER).size()); - Assertions.assertEquals(1, + assertEquals(1, leader.getRaftConf().getAllPeers(RaftProtos.RaftPeerRole.LISTENER).size()); } cluster.shutdown(); @@ -510,13 +507,13 @@ public void testRemoveListener() throws Exception { final RaftServer.Division leader = waitForLeader(cluster); try (RaftClient client = cluster.createClient(leader.getId())) { client.io().send(new RaftTestUtil.SimpleMessage("message")); - Assertions.assertEquals(1, cluster.getListeners().size()); + assertEquals(1, cluster.getListeners().size()); List servers = cluster.getFollowers().stream().map(RaftServer.Division::getPeer).collect( Collectors.toList()); servers.add(leader.getPeer()); RaftClientReply reply = client.admin().setConfiguration(servers); - Assertions.assertTrue(reply.isSuccess()); - Assertions.assertEquals(0, leader.getRaftConf().getAllPeers(RaftProtos.RaftPeerRole.LISTENER).size()); + assertTrue(reply.isSuccess()); + assertEquals(0, leader.getRaftConf().getAllPeers(RaftProtos.RaftPeerRole.LISTENER).size()); } cluster.shutdown(); } @@ -531,15 +528,15 @@ public void testChangeFollowerToListener() throws Exception { client.io().send(new RaftTestUtil.SimpleMessage("message")); List followers = cluster.getFollowers().stream().map( RaftServer.Division::getPeer).collect(Collectors.toList()); - Assertions.assertEquals(2, followers.size()); + assertEquals(2, followers.size()); List listeners = new ArrayList<>(); listeners.add(followers.get(1)); followers.remove(1); RaftClientReply reply = client.admin().setConfiguration(followers, listeners); - Assertions.assertTrue(reply.isSuccess()); + assertTrue(reply.isSuccess()); Collection peer = leader.getRaftConf().getAllPeers(RaftProtos.RaftPeerRole.LISTENER); - Assertions.assertEquals(1, peer.size()); - Assertions.assertEquals(listeners.get(0).getId(), new ArrayList<>(peer).get(0).getId()); + assertEquals(1, peer.size()); + assertEquals(listeners.get(0).getId(), new ArrayList<>(peer).get(0).getId()); } cluster.shutdown(); } @@ -554,11 +551,11 @@ public void testChangeListenerToFollower() throws Exception { client.io().send(new RaftTestUtil.SimpleMessage("message")); List listeners = cluster.getListeners() .stream().map(RaftServer.Division::getPeer).collect(Collectors.toList()); - Assertions.assertEquals(listeners.size(), 1); + assertEquals(listeners.size(), 1); RaftClientReply reply = client.admin().setConfiguration(cluster.getPeers()); - Assertions.assertTrue(reply.isSuccess()); + assertTrue(reply.isSuccess()); Collection peer = leader.getRaftConf().getAllPeers(RaftProtos.RaftPeerRole.LISTENER); - Assertions.assertEquals(0, peer.size()); + assertEquals(0, peer.size()); } cluster.shutdown(); } @@ -638,7 +635,7 @@ public void testPreVote() { RaftTestUtil.isolate(cluster, follower.getId()); // send message so that the isolated follower's log lag the others RaftClientReply reply = client.io().send(new RaftTestUtil.SimpleMessage("message")); - Assertions.assertTrue(reply.isSuccess()); + assertTrue(reply.isSuccess()); final long savedTerm = leader.getInfo().getCurrentTerm(); LOG.info("Wait follower {} timeout and trigger pre-vote", follower.getId()); @@ -653,7 +650,7 @@ public void testPreVote() { assertEquals(savedTerm, leader.getInfo().getCurrentTerm()); reply = client.io().send(new RaftTestUtil.SimpleMessage("message")); - Assertions.assertTrue(reply.isSuccess()); + assertTrue(reply.isSuccess()); } cluster.shutdown(); @@ -688,23 +685,23 @@ void runTestPauseResumeLeaderElection(CLUSTER cluster) throws IOException, Inter final RaftServer.Division leader = RaftTestUtil.waitForLeader(cluster); final RaftPeerId leaderId = leader.getId(); final List followers = cluster.getFollowers(); - Assertions.assertTrue(followers.size() >= 1); + assertTrue(followers.size() >= 1); final RaftServerImpl f1 = (RaftServerImpl)followers.get(0); try (final RaftClient client = cluster.createClient()) { pauseLeaderReply = client.getLeaderElectionManagementApi(f1.getId()).pause(); - Assertions.assertTrue(pauseLeaderReply.isSuccess()); + assertTrue(pauseLeaderReply.isSuccess()); client.io().send(new RaftTestUtil.SimpleMessage("message")); RaftServer.Division newLeader = followers.get(0); List peers = cluster.getPeers(); List peersWithNewPriority = getPeersWithPriority(peers, newLeader.getPeer()); RaftClientReply reply = client.admin().setConfiguration(peersWithNewPriority.toArray(new RaftPeer[0])); - Assertions.assertTrue(reply.isSuccess()); - JavaUtils.attempt(() -> Assertions.assertEquals(leaderId, leader.getId()), + assertTrue(reply.isSuccess()); + JavaUtils.attempt(() -> assertEquals(leaderId, leader.getId()), 20, HUNDRED_MILLIS, "check leader id", LOG); final RaftClientReply resumeLeaderReply = client.getLeaderElectionManagementApi(f1.getId()).resume(); - Assertions.assertTrue(resumeLeaderReply.isSuccess()); - JavaUtils.attempt(() -> Assertions.assertEquals(f1.getId(), cluster.getLeader().getId()), + assertTrue(resumeLeaderReply.isSuccess()); + JavaUtils.attempt(() -> assertEquals(f1.getId(), cluster.getLeader().getId()), 20, HUNDRED_MILLIS, "check new leader", LOG); } } @@ -730,15 +727,15 @@ void runTestLeaderLease(CLUSTER cluster, long leaseTimeoutMs) throws Exception { try (final RaftClient client = cluster.createClient(leader.getId())) { client.io().send(new RaftTestUtil.SimpleMessage("message")); - Assertions.assertTrue(leader.getInfo().isLeader()); - Assertions.assertTrue(leader.getInfo().isLeaderReady()); + assertTrue(leader.getInfo().isLeader()); + assertTrue(leader.getInfo().isLeaderReady()); RaftServerTestUtil.assertLeaderLease(leader, true); RaftTestUtil.isolate(cluster, leader.getId()); Thread.sleep(leaseTimeoutMs); - Assertions.assertTrue(leader.getInfo().isLeader()); - Assertions.assertTrue(leader.getInfo().isLeaderReady()); + assertTrue(leader.getInfo().isLeader()); + assertTrue(leader.getInfo().isLeaderReady()); RaftServerTestUtil.assertLeaderLease(leader, false); } finally { RaftTestUtil.deIsolate(cluster, leader.getId()); @@ -759,8 +756,8 @@ void runTestLeaderLeaseDuringReconfiguration(CLUSTER cluster, long leaseTimeoutM try (final RaftClient client = cluster.createClient(leader.getId())) { client.io().send(new RaftTestUtil.SimpleMessage("message")); - Assertions.assertTrue(leader.getInfo().isLeader()); - Assertions.assertTrue(leader.getInfo().isLeaderReady()); + assertTrue(leader.getInfo().isLeader()); + assertTrue(leader.getInfo().isLeaderReady()); RaftServerTestUtil.assertLeaderLease(leader, true); final List followers = cluster.getFollowers(); @@ -781,8 +778,8 @@ void runTestLeaderLeaseDuringReconfiguration(CLUSTER cluster, long leaseTimeoutM Thread.sleep(leaseTimeoutMs); - Assertions.assertTrue(leader.getInfo().isLeader()); - Assertions.assertTrue(leader.getInfo().isLeaderReady()); + assertTrue(leader.getInfo().isLeader()); + assertTrue(leader.getInfo().isLeaderReady()); RaftServerTestUtil.assertLeaderLease(leader, false); } finally { diff --git a/ratis-server/src/test/java/org/apache/ratis/server/impl/StateMachineShutdownTests.java b/ratis-server/src/test/java/org/apache/ratis/server/impl/StateMachineShutdownTests.java index 3333fdb852..d97c5c4d50 100644 --- a/ratis-server/src/test/java/org/apache/ratis/server/impl/StateMachineShutdownTests.java +++ b/ratis-server/src/test/java/org/apache/ratis/server/impl/StateMachineShutdownTests.java @@ -28,7 +28,10 @@ import org.apache.ratis.statemachine.impl.SimpleStateMachine4Testing; import org.apache.ratis.statemachine.StateMachine; import org.apache.ratis.statemachine.TransactionContext; -import org.junit.*; +import org.junit.jupiter.api.AfterEach; +import org.junit.jupiter.api.Assertions; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.Test; import org.mockito.MockedStatic; import org.mockito.Mockito; import org.slf4j.Logger; @@ -121,12 +124,12 @@ public void unblockAllTxns() { } } - @Before + @BeforeEach public void setup() { mocked = Mockito.mockStatic(CompletableFuture.class, Mockito.CALLS_REAL_METHODS); } - @After + @AfterEach public void tearDownClass() { if (mocked != null) { mocked.close(); @@ -162,10 +165,10 @@ public void testStateMachineShutdownWaitsForApplyTxn() throws Exception { RaftClientReply watchReply = client.io().watch( logIndex, RaftProtos.ReplicationLevel.ALL_COMMITTED); watchReply.getCommitInfos().forEach( - val -> Assert.assertTrue(val.getCommitIndex() >= logIndex)); + val -> Assertions.assertTrue(val.getCommitIndex() >= logIndex)); final RaftServer.Division secondFollower = cluster.getFollowers().get(1); // Second follower is blocked in apply transaction - Assert.assertTrue(secondFollower.getInfo().getLastAppliedIndex() < logIndex); + Assertions.assertTrue(secondFollower.getInfo().getLastAppliedIndex() < logIndex); // Now shutdown the follower in a separate thread final Thread t = new Thread(secondFollower::close); @@ -176,24 +179,24 @@ public void testStateMachineShutdownWaitsForApplyTxn() throws Exception { // Now unblock the second follower long minIndex = ((StateMachineWithConditionalWait) secondFollower.getStateMachine()).blockTxns.stream() .min(Comparator.naturalOrder()).get(); - Assert.assertEquals(2, StateMachineWithConditionalWait.numTxns.values().stream() + Assertions.assertEquals(2, StateMachineWithConditionalWait.numTxns.values().stream() .filter(val -> val.get() == 3).count()); // The second follower should still be blocked in apply transaction - Assert.assertTrue(secondFollower.getInfo().getLastAppliedIndex() < minIndex); + Assertions.assertTrue(secondFollower.getInfo().getLastAppliedIndex() < minIndex); for (long index : ((StateMachineWithConditionalWait) secondFollower.getStateMachine()).blockTxns) { if (minIndex != index) { ((StateMachineWithConditionalWait) secondFollower.getStateMachine()).unBlockApplyTxn(index); } } - Assert.assertEquals(2, StateMachineWithConditionalWait.numTxns.values().stream() + Assertions.assertEquals(2, StateMachineWithConditionalWait.numTxns.values().stream() .filter(val -> val.get() == 3).count()); - Assert.assertTrue(secondFollower.getInfo().getLastAppliedIndex() < minIndex); + Assertions.assertTrue(secondFollower.getInfo().getLastAppliedIndex() < minIndex); ((StateMachineWithConditionalWait) secondFollower.getStateMachine()).unBlockApplyTxn(minIndex); // Now wait for the thread t.join(5000); - Assert.assertEquals(logIndex, secondFollower.getInfo().getLastAppliedIndex()); - Assert.assertEquals(3, StateMachineWithConditionalWait.numTxns.values().stream() + Assertions.assertEquals(logIndex, secondFollower.getInfo().getLastAppliedIndex()); + Assertions.assertEquals(3, StateMachineWithConditionalWait.numTxns.values().stream() .filter(val -> val.get() == 3).count()); cluster.shutdown(); diff --git a/ratis-test/pom.xml b/ratis-test/pom.xml index a47e9bd359..efd87bedde 100644 --- a/ratis-test/pom.xml +++ b/ratis-test/pom.xml @@ -131,11 +131,6 @@ test - - junit - junit - test - org.junit.jupiter junit-jupiter-engine @@ -151,11 +146,6 @@ junit-jupiter-params test - - org.junit.vintage - junit-vintage-engine - test - org.junit.platform junit-platform-launcher From 240358d0ac0caa011871766b74fb7b456c408959 Mon Sep 17 00:00:00 2001 From: Tsz-Wo Nicholas Sze Date: Mon, 2 Jun 2025 12:59:37 -0700 Subject: [PATCH 188/397] RATIS-2305. Add a security doc. (#1270) --- ratis-docs/src/site/markdown/security.md | 46 ++++++++++++++++++++++++ 1 file changed, 46 insertions(+) create mode 100644 ratis-docs/src/site/markdown/security.md diff --git a/ratis-docs/src/site/markdown/security.md b/ratis-docs/src/site/markdown/security.md new file mode 100644 index 0000000000..eb233bf932 --- /dev/null +++ b/ratis-docs/src/site/markdown/security.md @@ -0,0 +1,46 @@ + + +# Security + +Raft is a consensus algorithm designed for fault tolerance. +A basic assumption of Raft is that + +- the servers can trust each other. + +In contrast, Raft is not designed to solve the Byzantine Generals Problem, +which assumes that some of the servers may be malicious. + +Apache Ratis is a Raft library +which supports a pluggable transport. +Applications using Ratis usually run a transport over a network. +In such case, +the applications must either provide secure communications between the servers +or provide a safe network environment such as running the servers in a private network. +Applications must not accept requests from any untrusted servers. + +Below are the TLS Configuration Parameters. +Applications may use them to build `RaftServer`/`RaftClient` objects for establishing secure connections. + +| **Property** | **Description** | +|:----------------------------------------|:----------------------------------| +| `raft.grpc.tls.conf` | gRPC default TLS conf | +| `raft.grpc.server.tls.conf` | gRPC server TLS conf | +| `raft.grpc.client.tls.conf` | gRPC client TLS conf | +| `raft.grpc.admin.tls.conf` | gRPC admin TLS conf | +| `raft.netty.dataStream.server.tls.conf` | Netty data stream server TLS conf | +| `raft.netty.dataStream.client.tls.conf` | Netty data stream client TLS conf | From 8dbf4131b1e1542d27fb220d1fa693e0c722d7c0 Mon Sep 17 00:00:00 2001 From: Tsz-Wo Nicholas Sze Date: Mon, 2 Jun 2025 13:00:16 -0700 Subject: [PATCH 189/397] RATIS-2308. Remove the unused gRPC TLS confs. (#1271) --- .../org/apache/ratis/grpc/GrpcConfigKeys.java | 41 +++++-------------- .../shell/cli/sh/TestSecureRatisShell.java | 2 - 2 files changed, 10 insertions(+), 33 deletions(-) diff --git a/ratis-grpc/src/main/java/org/apache/ratis/grpc/GrpcConfigKeys.java b/ratis-grpc/src/main/java/org/apache/ratis/grpc/GrpcConfigKeys.java index a603960f17..f21a9b99f1 100644 --- a/ratis-grpc/src/main/java/org/apache/ratis/grpc/GrpcConfigKeys.java +++ b/ratis-grpc/src/main/java/org/apache/ratis/grpc/GrpcConfigKeys.java @@ -54,50 +54,29 @@ static Consumer getDefaultLog() { interface TLS { String PREFIX = GrpcConfigKeys.PREFIX + ".tls"; - String ENABLED_KEY = PREFIX + ".enabled"; - boolean ENABLED_DEFAULT = false; - static boolean enabled(RaftProperties properties) { - return getBoolean(properties::getBoolean, ENABLED_KEY, ENABLED_DEFAULT, getDefaultLog()); - } + @Deprecated static void setEnabled(RaftProperties properties, boolean enabled) { - setBoolean(properties::setBoolean, ENABLED_KEY, enabled); + LOG.warn("This method has no effect. Use setConf(Parameters, GrpcTlsConfig) instead."); } - String MUTUAL_AUTHN_ENABLED_KEY = PREFIX + ".mutual_authn.enabled"; - boolean MUTUAL_AUTHN_ENABLED_DEFAULT = false; - static boolean mutualAuthnEnabled(RaftProperties properties) { - return getBoolean(properties::getBoolean, - MUTUAL_AUTHN_ENABLED_KEY, MUTUAL_AUTHN_ENABLED_DEFAULT, getDefaultLog()); - } + @Deprecated static void setMutualAuthnEnabled(RaftProperties properties, boolean mutualAuthnEnabled) { - setBoolean(properties::setBoolean, MUTUAL_AUTHN_ENABLED_KEY, mutualAuthnEnabled); + LOG.warn("This method has no effect. Use setConf(Parameters, GrpcTlsConfig) instead."); } - String PRIVATE_KEY_FILE_NAME_KEY = PREFIX + ".private.key.file.name"; - String PRIVATE_KEY_FILE_NAME_DEFAULT = "private.pem"; - static String privateKeyFileName(RaftProperties properties) { - return get(properties::get, PRIVATE_KEY_FILE_NAME_KEY, PRIVATE_KEY_FILE_NAME_DEFAULT, getDefaultLog()); - } + @Deprecated static void setPrivateKeyFileName(RaftProperties properties, String privateKeyFileName) { - set(properties::set, PRIVATE_KEY_FILE_NAME_KEY, privateKeyFileName); + LOG.warn("This method has no effect. Use setConf(Parameters, GrpcTlsConfig) instead."); } - String CERT_CHAIN_FILE_NAME_KEY = PREFIX + ".cert.chain.file.name"; - String CERT_CHAIN_FILE_NAME_DEFAULT = "certificate.crt"; - static String certChainFileName(RaftProperties properties) { - return get(properties::get, CERT_CHAIN_FILE_NAME_KEY, CERT_CHAIN_FILE_NAME_DEFAULT, getDefaultLog()); - } + @Deprecated static void setCertChainFileName(RaftProperties properties, String certChainFileName) { - set(properties::set, CERT_CHAIN_FILE_NAME_KEY, certChainFileName); + LOG.warn("This method has no effect. Use setConf(Parameters, GrpcTlsConfig) instead."); } - String TRUST_STORE_KEY = PREFIX + ".trust.store"; - String TRUST_STORE_DEFAULT = "ca.crt"; - static String trustStore(RaftProperties properties) { - return get(properties::get, TRUST_STORE_KEY, TRUST_STORE_DEFAULT, getDefaultLog()); - } + @Deprecated static void setTrustStore(RaftProperties properties, String trustStore) { - set(properties::set, TRUST_STORE_KEY, trustStore); + LOG.warn("This method has no effect. Use setConf(Parameters, GrpcTlsConfig) instead."); } String CONF_PARAMETER = PREFIX + ".conf"; diff --git a/ratis-test/src/test/java/org/apache/ratis/shell/cli/sh/TestSecureRatisShell.java b/ratis-test/src/test/java/org/apache/ratis/shell/cli/sh/TestSecureRatisShell.java index 785cd2e829..7f11490670 100644 --- a/ratis-test/src/test/java/org/apache/ratis/shell/cli/sh/TestSecureRatisShell.java +++ b/ratis-test/src/test/java/org/apache/ratis/shell/cli/sh/TestSecureRatisShell.java @@ -75,8 +75,6 @@ public void testRatisShell() throws Exception { final String[] ids = {"s0"}; final RaftProperties properties = new RaftProperties(); RaftClientConfigKeys.Rpc.setRequestTimeout(properties, TimeDuration.ONE_MINUTE); - GrpcConfigKeys.TLS.setEnabled(properties, true); - GrpcConfigKeys.TLS.setMutualAuthnEnabled(properties, true); try(MiniRaftClusterWithGrpc cluster = new MiniRaftClusterWithGrpc(ids, properties, SERVER_PARAMETERS)) { cluster.start(); From e7b7aa3bfef65ffc036d6ac63ce9486f546fa17e Mon Sep 17 00:00:00 2001 From: "Doroszlai, Attila" <6454655+adoroszlai@users.noreply.github.com> Date: Tue, 10 Jun 2025 18:59:57 +0200 Subject: [PATCH 190/397] RATIS-2221. Remove ratis-replicated-map module (#1260) --- pom.xml | 14 -- ratis-assembly/pom.xml | 10 - ratis-assembly/src/main/assembly/bin.xml | 1 - ratis-assembly/src/main/assembly/src.xml | 1 - ratis-proto/src/main/proto/RMap.proto | 189 ------------------ ratis-replicated-map/README.md | 36 ---- ratis-replicated-map/pom.xml | 98 --------- .../src/test/resources/log4j.properties | 18 -- 8 files changed, 367 deletions(-) delete mode 100644 ratis-proto/src/main/proto/RMap.proto delete mode 100644 ratis-replicated-map/README.md delete mode 100644 ratis-replicated-map/pom.xml delete mode 100644 ratis-replicated-map/src/test/resources/log4j.properties diff --git a/pom.xml b/pom.xml index 4d8dfe1e61..01b657e60c 100644 --- a/pom.xml +++ b/pom.xml @@ -85,7 +85,6 @@ ratis-test ratis-examples - ratis-replicated-map ratis-metrics-api ratis-metrics-default ratis-metrics-dropwizard3 @@ -351,19 +350,6 @@ ${project.version} - - ratis-replicated-map - org.apache.ratis - ${project.version} - - - ratis-replicated-map - org.apache.ratis - ${project.version} - test-jar - test - - ratis-server-api org.apache.ratis diff --git a/ratis-assembly/pom.xml b/ratis-assembly/pom.xml index 11058eb329..e7b0f3c23b 100644 --- a/ratis-assembly/pom.xml +++ b/ratis-assembly/pom.xml @@ -273,16 +273,6 @@ test-jar - - ratis-replicated-map - org.apache.ratis - - - ratis-replicated-map - org.apache.ratis - test-jar - - org.apache.ratis ratis-metrics-api diff --git a/ratis-assembly/src/main/assembly/bin.xml b/ratis-assembly/src/main/assembly/bin.xml index 7fd3cc0df5..12161b00a8 100644 --- a/ratis-assembly/src/main/assembly/bin.xml +++ b/ratis-assembly/src/main/assembly/bin.xml @@ -44,7 +44,6 @@ org.apache.ratis:ratis-grpc org.apache.ratis:ratis-netty org.apache.ratis:ratis-proto - org.apache.ratis:ratis-replicated-map org.apache.ratis:ratis-server-api org.apache.ratis:ratis-server org.apache.ratis:ratis-test diff --git a/ratis-assembly/src/main/assembly/src.xml b/ratis-assembly/src/main/assembly/src.xml index 6aac4ec062..fdc64ed2ad 100644 --- a/ratis-assembly/src/main/assembly/src.xml +++ b/ratis-assembly/src/main/assembly/src.xml @@ -38,7 +38,6 @@ org.apache.ratis:ratis-netty org.apache.ratis:ratis-proto org.apache.ratis:ratis-docs - org.apache.ratis:ratis-replicated-map org.apache.ratis:ratis-server-api org.apache.ratis:ratis-server org.apache.ratis:ratis-shell diff --git a/ratis-proto/src/main/proto/RMap.proto b/ratis-proto/src/main/proto/RMap.proto deleted file mode 100644 index 43c9377d24..0000000000 --- a/ratis-proto/src/main/proto/RMap.proto +++ /dev/null @@ -1,189 +0,0 @@ -/** - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ -syntax = "proto3"; -option java_package = "org.apache.ratis.proto.rmap"; -option java_outer_classname = "RMapProtos"; -option java_generate_equals_and_hash = true; -package ratis.rmap; - -// TODO: This .proto file should go to the ratis-replicated-map module, but we need it here -// due to shading. - -// Metadata about a replicated map -message RMapInfo { - int64 rmap_id = 1; - string name = 2; - string key_class = 3; - string value_class = 4; - string key_serde_class = 5; - string value_serde_class = 6; - string key_comparator_class = 7; -} - -// An entry in a replicated map.` -message Entry { - bytes key = 1; - bytes value = 2; -} - -// TODO: raft client should allow a Service decleration, and calling a method from the service -// similar to how coprocessor calls work in HBase. -message Request { - oneof RequestType { - MultiActionRequest multi_action_request = 1; - ScanRequest scan_request = 2; - CreateRMapRequest create_rmap_request = 3; - DeleteRMapRequest delete_rmap_request = 4; - ListRMapInfosRequest list_rmap_infos_request = 5; - } -} - -message Response { - ExceptionResponse exception = 1; - oneof ResponseType { - MultiActionResponse multi_action_response = 2; - ScanResponse scan_response = 3; - CreateRMapResponse create_rmap_response = 4; - DeleteRMapResponse delete_rmap_response = 5; - ListRMapInfosResponse list_rmap_infos_response = 6; - } -} - -message MultiActionRequest { - int64 rmap_id = 1; - repeated Action action = 2; -} - -message Action { - oneof ActionType { - GetRequest get_request = 1; - PutRequest put_request = 2; - } -} - -message ActionResponse { - oneof ActionType { - GetResponse get_response = 1; - PutResponse put_response = 2; - } -} - -message MultiActionResponse { - repeated ActionResponse action_response = 1; -} - -message CreateRMapRequest { - RMapInfo rmap_info = 1; -} - -message CreateRMapResponse { - RMapInfo rmap_info = 1; -} - -message DeleteRMapRequest { - int64 rmap_id = 1; -} - -message DeleteRMapResponse { -} - -message ListRMapInfosRequest { - oneof ListRMapInfosType { - // if set, we only care about a particular RMapInfo - int64 rmap_id = 1; - - // only return infos whose names match this pattern - string name_pattern = 2; - } -} - -message ListRMapInfosResponse { - repeated RMapInfo rmap_info = 1; -} - -message GetRequest { - bytes key = 1; -} - -message GetResponse { - bool found = 1; - bytes key = 2; - bytes value = 3; -} - -message PutRequest { - bytes key = 1; - bytes value = 2; -} - -message PutResponse { -} - -message Scan { - bytes start_Key = 1; - bytes end_key = 2; - bool start_key_inclusive = 3; - bool end_key_inclusive = 4; - bool keys_only = 5; - int32 limit = 6; -} - -message ScanRequest { - int64 rmap_id = 1; - Scan scan = 2; -} - -message ScanResponse { - repeated Entry entry = 1; -} - -message ExceptionResponse { - // Class name of the exception thrown from the server - string exception_class_name = 1; - // Exception stack trace from the server side - string stack_trace = 2; - // Optional hostname. Filled in for some exceptions such as region moved - // where exception gives clue on where the region may have moved. - string hostname = 3; - int32 port = 4; - // Set if we are NOT to retry on receipt of this exception - bool do_not_retry = 5; -} - -message Id { - int64 id = 1; -} - -message WALEntry { - int64 rmap_id = 1; // these are shared by all WALEntry types - repeated Entry entry = 2; - oneof WALEntryType { - // Multi is not here, because we do not want to create one more object unnecessarily - CreateRMapWALEntry create_rmap_entry = 3; - DeleteRMapWALEntry delete_rmap_entry = 4; - } -} - -message CreateRMapWALEntry { - RMapInfo rmap_info = 1; - Id id = 2; -} - -message DeleteRMapWALEntry { - int64 id = 1; -} diff --git a/ratis-replicated-map/README.md b/ratis-replicated-map/README.md deleted file mode 100644 index ec4d3c554a..0000000000 --- a/ratis-replicated-map/README.md +++ /dev/null @@ -1,36 +0,0 @@ - - -Overview -======== -Ratis replicated map is an implementation of a sorted map (think TreeMap) as -a replicated state machine. This is not under examples because it is intended -to be used in production where a simple in-memory map is sufficient to hold the -data. The data is fully cached in memory, but it is still durable since raft -log is used as a replicated log, and data is snapshotted periodically. - - -The replicated map (RMap) is not only the state machine implementation, but -all of the remaining code, including the client and querying capabilities which -is built on top of the other modules. In that sense, it is dog-fooding the ratis -library to implement an end-to-end solution for a replicated in-memory data store. - -Replicated maps are conceptually similar to ZooKeeper/Etcd/LogCabin where the data -is hosted in a known cluster configuration and is not sharded. All the servers -in the cluster participate in a single RAFT ring. - -The data model is that users can create independent RMap instances in the cluster -and read / write or scan the data as key value pairs in those replicated maps. A -replicated map named the meta map contains information about all of the other maps -in the cluster. diff --git a/ratis-replicated-map/pom.xml b/ratis-replicated-map/pom.xml deleted file mode 100644 index 486b9cf9c5..0000000000 --- a/ratis-replicated-map/pom.xml +++ /dev/null @@ -1,98 +0,0 @@ - - - - 4.0.0 - - ratis - org.apache.ratis - 3.1.3 - - - ratis-replicated-map - Apache Ratis Replicated Map - - - - true - - - - - org.apache.ratis - ratis-thirdparty-misc - - - ratis-proto - org.apache.ratis - - - ratis-common - org.apache.ratis - - - ratis-common - org.apache.ratis - test - test-jar - - - ratis-client - org.apache.ratis - - - ratis-client - org.apache.ratis - test - test-jar - - - ratis-server - org.apache.ratis - - - ratis-server - org.apache.ratis - test - test-jar - - - ratis-grpc - org.apache.ratis - provided - - - ratis-grpc - org.apache.ratis - test - test-jar - - - ratis-netty - org.apache.ratis - provided - - - ratis-netty - org.apache.ratis - test - test-jar - - - - org.slf4j - slf4j-api - - - diff --git a/ratis-replicated-map/src/test/resources/log4j.properties b/ratis-replicated-map/src/test/resources/log4j.properties deleted file mode 100644 index ced0687caa..0000000000 --- a/ratis-replicated-map/src/test/resources/log4j.properties +++ /dev/null @@ -1,18 +0,0 @@ -# Licensed 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. -# log4j configuration used during build and unit tests - -log4j.rootLogger=info,stdout -log4j.threshold=ALL -log4j.appender.stdout=org.apache.log4j.ConsoleAppender -log4j.appender.stdout.layout=org.apache.log4j.PatternLayout -log4j.appender.stdout.layout.ConversionPattern=%d{ISO8601} %-5p %c{2} (%F:%M(%L)) - %m%n From c38b8a07654aa074433a09204adb63d84c306897 Mon Sep 17 00:00:00 2001 From: Nandakumar Vadivelu Date: Mon, 16 Jun 2025 00:13:57 +0530 Subject: [PATCH 191/397] RATIS-2310. Implement hashCode and equals method for SizeInBytes class. (#1272) --- .../main/java/org/apache/ratis/util/SizeInBytes.java | 10 ++++++++++ 1 file changed, 10 insertions(+) diff --git a/ratis-common/src/main/java/org/apache/ratis/util/SizeInBytes.java b/ratis-common/src/main/java/org/apache/ratis/util/SizeInBytes.java index 0756f4feee..a2a80697f8 100644 --- a/ratis-common/src/main/java/org/apache/ratis/util/SizeInBytes.java +++ b/ratis-common/src/main/java/org/apache/ratis/util/SizeInBytes.java @@ -83,4 +83,14 @@ public String getInput() { public String toString() { return description; } + + @Override + public boolean equals(Object obj) { + return obj instanceof SizeInBytes && size == ((SizeInBytes)obj).size; + } + + @Override + public int hashCode() { + return Long.hashCode(size); + } } From 9f1b197c0538a036dad0d924c02358c71497ccc6 Mon Sep 17 00:00:00 2001 From: Jiang Tian Date: Thu, 3 Jul 2025 05:33:47 +0800 Subject: [PATCH 192/397] RATIS-2314. Fix that SegmentedRaftLogWorker may append entry by itself. (#1274) --- .../java/org/apache/ratis/server/impl/RaftServerImpl.java | 5 +++-- 1 file changed, 3 insertions(+), 2 deletions(-) diff --git a/ratis-server/src/main/java/org/apache/ratis/server/impl/RaftServerImpl.java b/ratis-server/src/main/java/org/apache/ratis/server/impl/RaftServerImpl.java index 7f3ed98313..e5d9bfeca1 100644 --- a/ratis-server/src/main/java/org/apache/ratis/server/impl/RaftServerImpl.java +++ b/ratis-server/src/main/java/org/apache/ratis/server/impl/RaftServerImpl.java @@ -1639,8 +1639,9 @@ private CompletableFuture appendLog(List entries) { return appendLogFuture.get(); } - return appendLogFuture.updateAndGet(f -> f.thenCompose( - ignored -> JavaUtils.allOf(state.getLog().append(entries)))) + + return appendLogFuture.updateAndGet(f -> f.thenComposeAsync( + ignored -> JavaUtils.allOf(state.getLog().append(entries)), serverExecutor)) .whenComplete((v, e) -> appendLogTermIndices.removeExisting(entriesTermIndices)); } From 4933d72b4d7e25d26e1debe208f31af20db351a8 Mon Sep 17 00:00:00 2001 From: Nandakumar Vadivelu Date: Wed, 9 Jul 2025 15:32:36 +0530 Subject: [PATCH 193/397] RATIS-2317. Move acquire PendingRequestPermit out of synchronized block in RaftServerImpl#appendTransaction. (#1275) --- .../ratis/server/impl/RaftServerImpl.java | 50 ++++++++++++------- .../ratis/server/impl/RetryCacheImpl.java | 13 +---- 2 files changed, 33 insertions(+), 30 deletions(-) diff --git a/ratis-server/src/main/java/org/apache/ratis/server/impl/RaftServerImpl.java b/ratis-server/src/main/java/org/apache/ratis/server/impl/RaftServerImpl.java index e5d9bfeca1..3c10e103b6 100644 --- a/ratis-server/src/main/java/org/apache/ratis/server/impl/RaftServerImpl.java +++ b/ratis-server/src/main/java/org/apache/ratis/server/impl/RaftServerImpl.java @@ -171,7 +171,7 @@ public RaftPeerRole getCurrentRole() { @Override public boolean isLeaderReady() { - return isLeader() && getRole().isLeaderReady(); + return getRole().isLeaderReady(); } @Override @@ -752,6 +752,11 @@ RaftClientReply newExceptionReply(RaftClientRequest request, RaftException excep } private CompletableFuture checkLeaderState(RaftClientRequest request) { + try { + assertGroup(getMemberId(), request); + } catch (GroupMismatchException e) { + return JavaUtils.completeExceptionally(e); + } return checkLeaderState(request, null); } @@ -759,12 +764,6 @@ private CompletableFuture checkLeaderState(RaftClientRequest re * @return null if the server is in leader state. */ private CompletableFuture checkLeaderState(RaftClientRequest request, CacheEntry entry) { - try { - assertGroup(getMemberId(), request); - } catch (GroupMismatchException e) { - return RetryCacheImpl.failWithException(e, entry); - } - if (!getInfo().isLeader()) { NotLeaderException exception = generateNotLeaderException(); final RaftClientReply reply = newExceptionReply(request, exception); @@ -809,6 +808,11 @@ void assertLifeCycleState(Set expected) throws ServerNotReadyEx getMemberId() + " is not in " + expected + ": current state is " + c), expected); } + private CompletableFuture getResourceUnavailableReply(RaftClientRequest request, CacheEntry entry) { + return entry.failWithException(new ResourceUnavailableException( + getMemberId() + ": Failed to acquire a pending write request for " + request)); + } + /** * Handle a normal update request from client. */ @@ -819,6 +823,17 @@ private CompletableFuture appendTransaction( assertLifeCycleState(LifeCycle.States.RUNNING); + final LeaderStateImpl unsyncedLeaderState = role.getLeaderState().orElse(null); + if (unsyncedLeaderState == null) { + final RaftClientReply reply = newExceptionReply(request, generateNotLeaderException()); + return RetryCacheImpl.failWithReply(reply, cacheEntry); + } + final PendingRequests.Permit unsyncedPermit = unsyncedLeaderState.tryAcquirePendingRequest(request.getMessage()); + if (unsyncedPermit == null) { + return getResourceUnavailableReply(request, cacheEntry); + } + + final LeaderStateImpl leaderState; final PendingRequest pending; synchronized (this) { final CompletableFuture reply = checkLeaderState(request, cacheEntry); @@ -826,16 +841,15 @@ private CompletableFuture appendTransaction( return reply; } - // append the message to its local log - final LeaderStateImpl leaderState = role.getLeaderStateNonNull(); - writeIndexCache.add(request.getClientId(), context.getLogIndexFuture()); - - final PendingRequests.Permit permit = leaderState.tryAcquirePendingRequest(request.getMessage()); + leaderState = role.getLeaderStateNonNull(); + final PendingRequests.Permit permit = leaderState == unsyncedLeaderState? unsyncedPermit + : leaderState.tryAcquirePendingRequest(request.getMessage()); if (permit == null) { - cacheEntry.failWithException(new ResourceUnavailableException( - getMemberId() + ": Failed to acquire a pending write request for " + request)); - return cacheEntry.getReplyFuture(); + return getResourceUnavailableReply(request, cacheEntry); } + + // append the message to its local log + writeIndexCache.add(request.getClientId(), context.getLogIndexFuture()); try { state.appendLog(context); } catch (StateMachineException e) { @@ -853,13 +867,11 @@ private CompletableFuture appendTransaction( // put the request into the pending queue pending = leaderState.addPendingRequest(permit, request, context); if (pending == null) { - cacheEntry.failWithException(new ResourceUnavailableException( + return cacheEntry.failWithException(new ResourceUnavailableException( getMemberId() + ": Failed to add a pending write request for " + request)); - return cacheEntry.getReplyFuture(); } - leaderState.notifySenders(); } - + leaderState.notifySenders(); return pending.getFuture(); } diff --git a/ratis-server/src/main/java/org/apache/ratis/server/impl/RetryCacheImpl.java b/ratis-server/src/main/java/org/apache/ratis/server/impl/RetryCacheImpl.java index 50d238b07a..4da459ae9e 100644 --- a/ratis-server/src/main/java/org/apache/ratis/server/impl/RetryCacheImpl.java +++ b/ratis-server/src/main/java/org/apache/ratis/server/impl/RetryCacheImpl.java @@ -84,9 +84,10 @@ void failWithReply(RaftClientReply reply) { replyFuture.complete(reply); } - void failWithException(Throwable t) { + CompletableFuture failWithException(Throwable t) { failed = true; replyFuture.completeExceptionally(t); + return replyFuture; } @Override @@ -266,14 +267,4 @@ static CompletableFuture failWithReply( return CompletableFuture.completedFuture(reply); } } - - static CompletableFuture failWithException( - Throwable t, CacheEntry entry) { - if (entry != null) { - entry.failWithException(t); - return entry.getReplyFuture(); - } else { - return JavaUtils.completeExceptionally(t); - } - } } From caa54f325ab695ce15e775b841d0d5b88a7f35e9 Mon Sep 17 00:00:00 2001 From: Jiang Tian Date: Thu, 14 Aug 2025 00:17:22 +0800 Subject: [PATCH 194/397] RATIS-2321. Fix NPE in ServerState after continuous leader changes (#1280) --- .../src/main/java/org/apache/ratis/server/impl/ServerState.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/ratis-server/src/main/java/org/apache/ratis/server/impl/ServerState.java b/ratis-server/src/main/java/org/apache/ratis/server/impl/ServerState.java index d183930b9d..725c499d11 100644 --- a/ratis-server/src/main/java/org/apache/ratis/server/impl/ServerState.java +++ b/ratis-server/src/main/java/org/apache/ratis/server/impl/ServerState.java @@ -265,7 +265,7 @@ void setLeader(RaftPeerId newLeaderId, Object op) { suffix = ""; } else { final Timestamp previous = lastNoLeaderTime.getAndSet(null); - suffix = ", leader elected after " + previous.elapsedTimeMs() + "ms"; + suffix = ", leader elected after " + (previous != null ? previous.elapsedTimeMs() : 0) + "ms"; server.setFirstElection(op); server.getStateMachine().event().notifyLeaderChanged(getMemberId(), newLeaderId); } From add315851d6b17186917dfac78171900a14684de Mon Sep 17 00:00:00 2001 From: gaoyajun02 <81629032+gaoyajun02@users.noreply.github.com> Date: Tue, 26 Aug 2025 23:59:18 +0800 Subject: [PATCH 195/397] RATIS-2323. Extend ratis-shell add command (#1282) --- .../ratis/shell/cli/sh/peer/AddCommand.java | 72 ++++++++++++++++--- 1 file changed, 64 insertions(+), 8 deletions(-) diff --git a/ratis-shell/src/main/java/org/apache/ratis/shell/cli/sh/peer/AddCommand.java b/ratis-shell/src/main/java/org/apache/ratis/shell/cli/sh/peer/AddCommand.java index be8f789954..c0d9f8dfe6 100644 --- a/ratis-shell/src/main/java/org/apache/ratis/shell/cli/sh/peer/AddCommand.java +++ b/ratis-shell/src/main/java/org/apache/ratis/shell/cli/sh/peer/AddCommand.java @@ -46,6 +46,9 @@ public class AddCommand extends AbstractRatisCommand { public static final String ADDRESS_OPTION_NAME = "address"; public static final String PEER_ID_OPTION_NAME = "peerId"; + public static final String CLIENT_ADDRESS_OPTION_NAME = "clientAddress"; + public static final String ADMIN_ADDRESS_OPTION_NAME = "adminAddress"; + /** * @param context command context */ @@ -62,6 +65,8 @@ public String getCommandName() { public int run(CommandLine cl) throws IOException { super.run(cl); final Map peersInfo = new HashMap<>(); + final Map clientAddressInfo = new HashMap<>(); + final Map adminAddressInfo = new HashMap<>(); List ids; if (cl.hasOption(ADDRESS_OPTION_NAME) && cl.hasOption(PEER_ID_OPTION_NAME)) { @@ -74,8 +79,34 @@ public int run(CommandLine cl) throws IOException { for (int i = 0; i < ids.size(); i++) { peersInfo.put(ids.get(i), addresses.get(i)); } + + if (cl.hasOption(CLIENT_ADDRESS_OPTION_NAME)) { + final List clientAddresses = + Arrays.stream(cl.getOptionValue(CLIENT_ADDRESS_OPTION_NAME).split(",")) + .map(CliUtils::parseInetSocketAddress) + .collect(Collectors.toList()); + Preconditions.assertSame(ids.size(), clientAddresses.size(), "clientAddress size"); + for (int i = 0; i < ids.size(); i++) { + clientAddressInfo.put(ids.get(i), clientAddresses.get(i)); + } + } + + if (cl.hasOption(ADMIN_ADDRESS_OPTION_NAME)) { + final List adminAddresses = + Arrays.stream(cl.getOptionValue(ADMIN_ADDRESS_OPTION_NAME).split(",")) + .map(CliUtils::parseInetSocketAddress) + .collect(Collectors.toList()); + Preconditions.assertSame(ids.size(), adminAddresses.size(), "adminAddress size"); + for (int i = 0; i < ids.size(); i++) { + adminAddressInfo.put(ids.get(i), adminAddresses.get(i)); + } + } } else if (cl.hasOption(ADDRESS_OPTION_NAME)) { ids = getIds(cl.getOptionValue(ADDRESS_OPTION_NAME).split(","), peersInfo::put); + if (cl.hasOption(CLIENT_ADDRESS_OPTION_NAME) || cl.hasOption(ADMIN_ADDRESS_OPTION_NAME)) { + throw new IllegalArgumentException( + "When using auto-generated peer IDs, clientAddress and adminAddress are not supported."); + } } else { throw new IllegalArgumentException( "Both " + PEER_ID_OPTION_NAME + " and " + ADDRESS_OPTION_NAME + " options are missing."); @@ -83,11 +114,22 @@ public int run(CommandLine cl) throws IOException { try (RaftClient client = newRaftClient()) { final Stream remaining = getPeerStream(RaftPeerRole.FOLLOWER); - final Stream adding = ids.stream().map(raftPeerId -> RaftPeer.newBuilder() - .setId(raftPeerId) - .setAddress(peersInfo.get(raftPeerId)) - .setPriority(0) - .build()); + final Stream adding = ids.stream().map(raftPeerId -> { + RaftPeer.Builder builder = RaftPeer.newBuilder() + .setId(raftPeerId) + .setAddress(peersInfo.get(raftPeerId)) + .setPriority(0); + + if (clientAddressInfo.containsKey(raftPeerId)) { + builder.setClientAddress(clientAddressInfo.get(raftPeerId)); + } + + if (adminAddressInfo.containsKey(raftPeerId)) { + builder.setAdminAddress(adminAddressInfo.get(raftPeerId)); + } + + return builder.build(); + }); final List peers = Stream.concat(remaining, adding).collect(Collectors.toList()); final List listeners = getPeerStream(RaftPeerRole.LISTENER) .collect(Collectors.toList()); @@ -104,9 +146,12 @@ public String getUsage() { return String.format("%s" + " -%s " + " [-%s ]" - + " <[-%s ]|[-%s ]>", + + " <[-%s ]|[-%s ]>" + + " [-%s ]" + + " [-%s ]", getCommandName(), PEER_OPTION_NAME, GROUPID_OPTION_NAME, - ADDRESS_OPTION_NAME, PEER_ID_OPTION_NAME); + ADDRESS_OPTION_NAME, PEER_ID_OPTION_NAME, + CLIENT_ADDRESS_OPTION_NAME, ADMIN_ADDRESS_OPTION_NAME); } @Override @@ -123,8 +168,19 @@ public Options getOptions() { .desc("The address information of ratis peers") .build()) .addOption(Option.builder() - .option(PEER_ID_OPTION_NAME).hasArg() + .option(PEER_ID_OPTION_NAME) + .hasArg() .desc("The peer id of ratis peers") + .build()) + .addOption(Option.builder() + .option(CLIENT_ADDRESS_OPTION_NAME) + .hasArg() + .desc("The client address information of ratis peers") + .build()) + .addOption(Option.builder() + .option(ADMIN_ADDRESS_OPTION_NAME) + .hasArg() + .desc("The admin address information of ratis peers") .build()); } From c94e8a005e4b032146cea6a9b992d6ace218f12a Mon Sep 17 00:00:00 2001 From: Tsz-Wo Nicholas Sze Date: Mon, 8 Sep 2025 13:14:36 -0700 Subject: [PATCH 196/397] RATIS-2318. Add a test to show how to manually restore a snapshot. (#1279) --- .../java/org/apache/ratis/util/FileUtils.java | 26 ++++ .../counter/server/CounterStateMachine.java | 32 ++-- .../server/TestManualRestoreSnapshot.java | 146 ++++++++++++++++++ .../apache/ratis/server/impl/ServerState.java | 17 +- .../ratis/server/raftlog/RaftLogBase.java | 3 +- .../statemachine/impl/BaseStateMachine.java | 6 +- .../impl/SimpleStateMachineStorage.java | 8 +- .../ratis/server/impl/MiniRaftCluster.java | 10 +- 8 files changed, 225 insertions(+), 23 deletions(-) create mode 100644 ratis-examples/src/test/java/org/apache/ratis/examples/counter/server/TestManualRestoreSnapshot.java diff --git a/ratis-common/src/main/java/org/apache/ratis/util/FileUtils.java b/ratis-common/src/main/java/org/apache/ratis/util/FileUtils.java index 315010c626..79c00b5a35 100644 --- a/ratis-common/src/main/java/org/apache/ratis/util/FileUtils.java +++ b/ratis-common/src/main/java/org/apache/ratis/util/FileUtils.java @@ -33,6 +33,7 @@ import java.nio.file.FileVisitResult; import java.nio.file.Files; import java.nio.file.LinkOption; +import java.nio.file.NotDirectoryException; import java.nio.file.OpenOption; import java.nio.file.Path; import java.nio.file.Paths; @@ -43,7 +44,10 @@ import java.util.Arrays; import java.util.List; import java.util.Objects; +import java.util.function.BiConsumer; +import java.util.function.Consumer; import java.util.function.Supplier; +import java.util.stream.Stream; public interface FileUtils { Logger LOG = LoggerFactory.getLogger(FileUtils.class); @@ -382,4 +386,26 @@ public FileVisitResult postVisitDirectory(Path dir, IOException e) throws IOExce } }); } + + static void listDir(File dir, Consumer out, BiConsumer err) { + listDir(dir.toPath(), out, err); + } + + static void listDir(Path dir, Consumer out, BiConsumer err) { + try { + listDir(dir, out); + } catch (IOException e) { + err.accept("Failed to listDir: " + dir, e); + } + } + + static void listDir(Path dir, Consumer out) throws IOException { + if (!Files.isDirectory(dir, LinkOption.NOFOLLOW_LINKS)) { + throw new NotDirectoryException( "Failed to listDir: " + dir + " is not a directory."); + } + + try(Stream s = Files.list(dir)) { + s.forEach(out); + } + } } diff --git a/ratis-examples/src/main/java/org/apache/ratis/examples/counter/server/CounterStateMachine.java b/ratis-examples/src/main/java/org/apache/ratis/examples/counter/server/CounterStateMachine.java index b5352f0adc..914180feb8 100644 --- a/ratis-examples/src/main/java/org/apache/ratis/examples/counter/server/CounterStateMachine.java +++ b/ratis-examples/src/main/java/org/apache/ratis/examples/counter/server/CounterStateMachine.java @@ -78,6 +78,11 @@ TermIndex getApplied() { int getCounter() { return counter; } + + @Override + public String toString() { + return counter + "@" + applied; + } } private final SimpleStateMachineStorage storage = new SimpleStateMachineStorage(); @@ -94,11 +99,11 @@ public CounterStateMachine() { } /** @return the current state. */ - private synchronized CounterState getState() { + synchronized CounterState getState() { return new CounterState(getLastAppliedTermIndex(), counter.get()); } - private synchronized void updateState(TermIndex applied, int counterValue) { + synchronized void updateState(TermIndex applied, int counterValue) { updateLastAppliedTermIndex(applied); counter.set(counterValue); } @@ -141,36 +146,45 @@ public void reinitialize() throws IOException { load(storage.loadLatestSnapshot()); } + @Override + public SimpleStateMachineStorage getStateMachineStorage() { + return storage; + } + /** * Store the current state as a snapshot file in the {@link #storage}. * * @return the index of the snapshot */ @Override - public long takeSnapshot() { + public long takeSnapshot() throws IOException { //get the current state final CounterState state = getState(); final long index = state.getApplied().getIndex(); //create a file with a proper name to store the snapshot final File snapshotFile = storage.getSnapshotFile(state.getApplied().getTerm(), index); + try { + saveSnapshot(state, snapshotFile); + } catch (Exception e) { + throw new IOException("Failed to save snapshot (" + state + ") to file " + snapshotFile, e); + } + //return the index of the stored snapshot (which is the last applied one) + return index; + } + + void saveSnapshot(CounterState state, File snapshotFile) throws IOException { //write the counter value into the snapshot file try (ObjectOutputStream out = new ObjectOutputStream(new BufferedOutputStream( Files.newOutputStream(snapshotFile.toPath())))) { out.writeInt(state.getCounter()); - } catch (IOException ioe) { - LOG.warn("Failed to write snapshot file \"" + snapshotFile - + "\", last applied index=" + state.getApplied()); } // update storage final MD5Hash md5 = MD5FileUtil.computeAndSaveMd5ForFile(snapshotFile); final FileInfo info = new FileInfo(snapshotFile.toPath(), md5); storage.updateLatestSnapshot(new SingleFileSnapshotInfo(info, state.getApplied())); - - //return the index of the stored snapshot (which is the last applied one) - return index; } /** diff --git a/ratis-examples/src/test/java/org/apache/ratis/examples/counter/server/TestManualRestoreSnapshot.java b/ratis-examples/src/test/java/org/apache/ratis/examples/counter/server/TestManualRestoreSnapshot.java new file mode 100644 index 0000000000..0480a5364a --- /dev/null +++ b/ratis-examples/src/test/java/org/apache/ratis/examples/counter/server/TestManualRestoreSnapshot.java @@ -0,0 +1,146 @@ +/* + * 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.ratis.examples.counter.server; + +import org.apache.ratis.BaseTest; +import org.apache.ratis.RaftTestUtil; +import org.apache.ratis.client.RaftClient; +import org.apache.ratis.examples.counter.CounterCommand; +import org.apache.ratis.grpc.MiniRaftClusterWithGrpc; +import org.apache.ratis.protocol.Message; +import org.apache.ratis.protocol.RaftClientReply; +import org.apache.ratis.protocol.RaftGroup; +import org.apache.ratis.protocol.RaftPeerId; +import org.apache.ratis.server.RaftServer; +import org.apache.ratis.server.impl.MiniRaftCluster; +import org.apache.ratis.server.protocol.TermIndex; +import org.apache.ratis.statemachine.SnapshotInfo; +import org.apache.ratis.statemachine.StateMachine; +import org.apache.ratis.statemachine.impl.SimpleStateMachineStorage; +import org.apache.ratis.util.FileUtils; +import org.apache.ratis.util.JavaUtils; +import org.apache.ratis.util.TimeDuration; +import org.junit.jupiter.api.Test; + +import java.io.File; +import java.util.ArrayList; +import java.util.List; + +import static org.junit.jupiter.api.Assertions.assertNotNull; +import static org.junit.jupiter.api.Assertions.assertTrue; + +/** + * Test manually restoring a snapshot. + * Due to hardware failures or software bugs, + * the state of a state machine can become corrupted. + * In such case, we may manually copy a snapshot from the leader + * and then install it to the corrupted state machine. + */ +public class TestManualRestoreSnapshot extends BaseTest implements MiniRaftClusterWithGrpc.FactoryGet { + public static final int NUM_SERVERS = 3; + + { + getProperties().setClass(MiniRaftCluster.STATEMACHINE_CLASS_KEY, CounterStateMachine.class, StateMachine.class); + } + + @Test + public void testManualRestoreSnapshot() throws Exception { + runWithNewCluster(NUM_SERVERS, this::run); + } + + void run(MiniRaftCluster cluster) throws Exception { + final RaftGroup group = cluster.getGroup(); + + // send some messages + final RaftServer.Division leader = RaftTestUtil.waitForLeader(cluster); + LOG.info("Leader: {}", leader); + sendMessages(cluster, 5); + + // kill a follower + final RaftServer.Division toBeKilled = cluster.getFollowers().get(0); + LOG.info("Follower to be killed: {}", toBeKilled.getId()); + final SimpleStateMachineStorage smStorage = ((CounterStateMachine) toBeKilled.getStateMachine()) + .getStateMachineStorage(); + final File raftLogCurrentDir = toBeKilled.getRaftStorage().getStorageDir().getCurrentDir(); + cluster.killServer(toBeKilled.getId()); + + // send more messages + sendMessages(cluster, 3); + + // get a snapshot from the leader + final CounterStateMachine leaderStateMachine = (CounterStateMachine) leader.getStateMachine(); + final CounterStateMachine.CounterState snapshot = leaderStateMachine.getState(); + LOG.info("{}: Leader {}", leader.getId(), snapshot); + + // remove raft log from the killed follower + FileUtils.listDir(raftLogCurrentDir, s -> LOG.info("{}", s), LOG::error); + final String[] logFiles = raftLogCurrentDir.list((dir, name) -> name.startsWith("log")); + assertNotNull(logFiles); + for (String logFile : logFiles) { + FileUtils.deleteFile(new File(raftLogCurrentDir, logFile)); + } + + // remove the killed follower + final RaftPeerId followerId = toBeKilled.getId(); + cluster.removeServer(followerId); + + // save the leader snapshot to the killed follower + final TermIndex applied = snapshot.getApplied(); + final File snapshotFile = smStorage.getSnapshotFile(applied.getTerm(), applied.getIndex()); + final RaftServer toSaveSnapshot = cluster.putNewServer(followerId, group, false); + ((CounterStateMachine) toSaveSnapshot.getDivision(group.getGroupId()).getStateMachine()) + .saveSnapshot(snapshot, snapshotFile); + + // start follower and verify last applied + LOG.info("Restarting {}", followerId); + final RaftServer.Division restartedFollower = cluster.restartServer(followerId, group, false); + final StateMachine stateMachine = restartedFollower.getStateMachine(); + final SnapshotInfo info = stateMachine.getLatestSnapshot(); + LOG.info("{} restarted snapshot info {} from {}", followerId, info, stateMachine); + + JavaUtils.attemptUntilTrue(() -> { + System.out.println(cluster.printServers()); + final TermIndex leaderLastApplied = leaderStateMachine.getLastAppliedTermIndex(); + LOG.info("Leader {} last applied {}", leader.getId(), leaderLastApplied); + final TermIndex followerLastApplied = stateMachine.getLastAppliedTermIndex(); + LOG.info("Follower {} last applied {}", followerId, followerLastApplied); + return followerLastApplied.equals(leaderLastApplied); + }, 10, TimeDuration.ONE_SECOND, "followerLastApplied", LOG); + + sendMessages(cluster, 7); + } + + static void sendMessages(MiniRaftCluster cluster, int numMessages) throws Exception { + final List messages = getUpdateRequests(numMessages); + try(final RaftClient client = cluster.createClient()) { + for (Message message : messages) { + final RaftClientReply reply = client.io().send(message); + assertTrue(reply.isSuccess()); + } + } + } + + static List getUpdateRequests(int numMessages) { + final List messages = new ArrayList<>(); + for(int i = 0; i < numMessages; i++) { + messages.add(CounterCommand.INCREMENT.getMessage()); + } + return messages; + } + +} diff --git a/ratis-server/src/main/java/org/apache/ratis/server/impl/ServerState.java b/ratis-server/src/main/java/org/apache/ratis/server/impl/ServerState.java index 725c499d11..dc05c63e3e 100644 --- a/ratis-server/src/main/java/org/apache/ratis/server/impl/ServerState.java +++ b/ratis-server/src/main/java/org/apache/ratis/server/impl/ServerState.java @@ -123,12 +123,7 @@ class ServerState { // On start the leader is null, start the clock now this.lastNoLeaderTime = new AtomicReference<>(Timestamp.currentTime()); this.noLeaderTimeout = RaftServerConfigKeys.Notification.noLeaderTimeout(prop); - - final LongSupplier getSnapshotIndexFromStateMachine = () -> Optional.ofNullable(stateMachine.getLatestSnapshot()) - .map(SnapshotInfo::getIndex) - .filter(i -> i >= 0) - .orElse(RaftLog.INVALID_LOG_INDEX); - this.log = JavaUtils.memoize(() -> initRaftLog(getSnapshotIndexFromStateMachine, prop)); + this.log = JavaUtils.memoize(() -> initRaftLog(() -> getSnapshotIndexFromStateMachine(stateMachine), prop)); this.readRequests = new ReadRequests(prop, stateMachine); this.stateMachineUpdater = JavaUtils.memoize(() -> new StateMachineUpdater( stateMachine, server, this, getLog().getSnapshotIndex(), prop, @@ -154,6 +149,16 @@ RaftGroupMemberId getMemberId() { return memberId; } + private long getSnapshotIndexFromStateMachine(StateMachine stateMachine) { + final SnapshotInfo latest = stateMachine.getLatestSnapshot(); + LOG.info("{}: getLatestSnapshot({}) returns {}", getMemberId(), stateMachine, latest); + if (latest == null) { + return RaftLog.INVALID_LOG_INDEX; + } + final long index = latest.getIndex(); + return index >= 0 ? index : RaftLog.INVALID_LOG_INDEX; + } + void writeRaftConfiguration(LogEntryProto conf) { getStorage().writeRaftConfiguration(conf); } diff --git a/ratis-server/src/main/java/org/apache/ratis/server/raftlog/RaftLogBase.java b/ratis-server/src/main/java/org/apache/ratis/server/raftlog/RaftLogBase.java index 710dede480..8c2b66f969 100644 --- a/ratis-server/src/main/java/org/apache/ratis/server/raftlog/RaftLogBase.java +++ b/ratis-server/src/main/java/org/apache/ratis/server/raftlog/RaftLogBase.java @@ -86,7 +86,8 @@ protected RaftLogBase(RaftGroupMemberId memberId, RaftProperties properties) { this.name = memberId + "-" + JavaUtils.getClassSimpleName(getClass()); this.memberId = memberId; - long index = getSnapshotIndexFromStateMachine.getAsLong(); + final long index = getSnapshotIndexFromStateMachine.getAsLong(); + LOG.info("{}: snapshotIndexFromStateMachine = {}", name, index); this.commitIndex = new RaftLogIndex("commitIndex", index); this.snapshotIndex = new RaftLogIndex("snapshotIndex", index); this.purgeIndex = new RaftLogIndex("purgeIndex", LEAST_VALID_LOG_INDEX - 1); diff --git a/ratis-server/src/main/java/org/apache/ratis/statemachine/impl/BaseStateMachine.java b/ratis-server/src/main/java/org/apache/ratis/statemachine/impl/BaseStateMachine.java index bb7e9856bc..3f18ee538b 100644 --- a/ratis-server/src/main/java/org/apache/ratis/statemachine/impl/BaseStateMachine.java +++ b/ratis-server/src/main/java/org/apache/ratis/statemachine/impl/BaseStateMachine.java @@ -42,6 +42,7 @@ import java.util.SortedMap; import java.util.TreeMap; import java.util.concurrent.CompletableFuture; +import java.util.concurrent.atomic.AtomicInteger; import java.util.concurrent.atomic.AtomicReference; /** @@ -49,6 +50,9 @@ */ public class BaseStateMachine implements StateMachine, StateMachine.DataApi, StateMachine.EventApi, StateMachine.LeaderEventApi, StateMachine.FollowerEventApi { + private static final AtomicInteger ID_GENERATOR = new AtomicInteger(); + + private final int id = ID_GENERATOR.incrementAndGet(); private final CompletableFuture server = new CompletableFuture<>(); @SuppressWarnings({"squid:S3077"}) // Suppress volatile for generic type private volatile RaftGroupId groupId; @@ -226,7 +230,7 @@ public void close() throws IOException { @Override public String toString() { - return JavaUtils.getClassSimpleName(getClass()) + ":" + return JavaUtils.getClassSimpleName(getClass()) + "-" + id + ":" + (!server.isDone()? "uninitialized": getId() + ":" + groupId); } } diff --git a/ratis-server/src/main/java/org/apache/ratis/statemachine/impl/SimpleStateMachineStorage.java b/ratis-server/src/main/java/org/apache/ratis/statemachine/impl/SimpleStateMachineStorage.java index 7e8afbaa85..2552c33c25 100644 --- a/ratis-server/src/main/java/org/apache/ratis/statemachine/impl/SimpleStateMachineStorage.java +++ b/ratis-server/src/main/java/org/apache/ratis/statemachine/impl/SimpleStateMachineStorage.java @@ -226,8 +226,12 @@ public SingleFileSnapshotInfo loadLatestSnapshot() { return null; } try { - return updateLatestSnapshot(findLatestSnapshot(dir.toPath())); - } catch (IOException ignored) { + final SingleFileSnapshotInfo latest = updateLatestSnapshot(findLatestSnapshot(dir.toPath())); + LOG.info("Latest snapshot is {} in {}", latest, dir); + return latest; + } catch (IOException e) { + LOG.warn("Failed to updateLatestSnapshot from {}", dir, e); + FileUtils.listDir(dir, s -> LOG.warn(" {}", s), LOG::error); return null; } } diff --git a/ratis-server/src/test/java/org/apache/ratis/server/impl/MiniRaftCluster.java b/ratis-server/src/test/java/org/apache/ratis/server/impl/MiniRaftCluster.java index f4785ce906..b18440c835 100644 --- a/ratis-server/src/test/java/org/apache/ratis/server/impl/MiniRaftCluster.java +++ b/ratis-server/src/test/java/org/apache/ratis/server/impl/MiniRaftCluster.java @@ -351,9 +351,11 @@ public void start() throws IOException { : JavaUtils.runRepeatedly(() -> LOG.info("TIMED-PRINT: {}.", printServers()), 10, 10, TimeUnit.SECONDS)); } - /** - * start a stopped server again. - */ + public void removeServer(RaftPeerId serverId) { + servers.remove(serverId); + } + + /** Restart the server with the given id. */ public RaftServer.Division restartServer(RaftPeerId serverId, boolean format) throws IOException { return restartServer(serverId, group, format); } @@ -361,7 +363,7 @@ public RaftServer.Division restartServer(RaftPeerId serverId, boolean format) th public RaftServer.Division restartServer(RaftPeerId serverId, RaftGroup raftGroup, boolean format) throws IOException { killServer(serverId); - servers.remove(serverId); + removeServer(serverId); final RaftServer proxy = putNewServer(serverId, raftGroup, format); proxy.start(); From 79b4505029ee1c9babe37a36602e0aebd6fdd822 Mon Sep 17 00:00:00 2001 From: Tsz-Wo Nicholas Sze Date: Thu, 18 Sep 2025 08:58:26 -0700 Subject: [PATCH 197/397] RATIS-2330. Intermittent failure in TestRaftServerWithGrpc.testRaftServerMetrics. (#1286) --- .../apache/ratis/server/impl/PendingRequests.java | 12 +++--------- 1 file changed, 3 insertions(+), 9 deletions(-) diff --git a/ratis-server/src/main/java/org/apache/ratis/server/impl/PendingRequests.java b/ratis-server/src/main/java/org/apache/ratis/server/impl/PendingRequests.java index 259695d5ed..c6a9dd2794 100644 --- a/ratis-server/src/main/java/org/apache/ratis/server/impl/PendingRequests.java +++ b/ratis-server/src/main/java/org/apache/ratis/server/impl/PendingRequests.java @@ -117,7 +117,7 @@ private static class RequestMap { raftServerMetrics.addNumPendingRequestsMegaByteSize(resource::getMegaByteSize); } - Permit tryAcquire(Message message) { + synchronized Permit tryAcquire(Message message) { final int messageSize = Message.getSize(message); final int messageSizeMb = roundUpMb(messageSize ); final Acquired acquired = resource.tryAcquire(messageSizeMb); @@ -139,13 +139,7 @@ Permit tryAcquire(Message message) { if (messageSizeMb > diffMb) { resource.releaseExtraMb(messageSizeMb - diffMb); } - return putPermit(); - } - private synchronized Permit putPermit() { - if (resource.isClosed()) { - return null; - } final Permit permit = new Permit(); permits.put(permit, permit); return permit; @@ -157,9 +151,9 @@ synchronized PendingRequest put(Permit permit, PendingRequest p) { if (removed == null) { return null; } - Preconditions.assertTrue(removed == permit); + Preconditions.assertSame(permit, removed, "permit"); final PendingRequest previous = map.put(p.getTermIndex(), p); - Preconditions.assertTrue(previous == null); + Preconditions.assertNull(previous, "previous"); return p; } From 5b2a94db3fb99da2340f516a2f2efdf8915ff65b Mon Sep 17 00:00:00 2001 From: Potato Date: Fri, 19 Sep 2025 09:59:10 +0800 Subject: [PATCH 198/397] RATIS-2328. Bump ratis-thirdparty from 1.0.9 to 1.0.10 (#1284) Signed-off-by: OneSizeFitQuorum --- pom.xml | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/pom.xml b/pom.xml index 01b657e60c..b93500d3c1 100644 --- a/pom.xml +++ b/pom.xml @@ -207,11 +207,11 @@ 3.3.9 - 1.0.9 + 1.0.10 - 3.25.5 - 1.71.0 + 3.25.8 + 1.75.0 true From 5547980c922cbe30a7619784d165eaeacbd49a6f Mon Sep 17 00:00:00 2001 From: Symious <14933944+symious@users.noreply.github.com> Date: Sat, 20 Sep 2025 09:38:36 +0800 Subject: [PATCH 199/397] RATIS-2325. Create GrpcStubPool for GrpcServerProtocolClient (#1283) --- .../org/apache/ratis/grpc/GrpcConfigKeys.java | 9 ++ .../grpc/server/GrpcServerProtocolClient.java | 38 ++++- .../ratis/grpc/server/GrpcServicesImpl.java | 4 +- .../ratis/grpc/server/GrpcStubPool.java | 132 ++++++++++++++++++ 4 files changed, 179 insertions(+), 4 deletions(-) create mode 100644 ratis-grpc/src/main/java/org/apache/ratis/grpc/server/GrpcStubPool.java diff --git a/ratis-grpc/src/main/java/org/apache/ratis/grpc/GrpcConfigKeys.java b/ratis-grpc/src/main/java/org/apache/ratis/grpc/GrpcConfigKeys.java index f21a9b99f1..cef62779dd 100644 --- a/ratis-grpc/src/main/java/org/apache/ratis/grpc/GrpcConfigKeys.java +++ b/ratis-grpc/src/main/java/org/apache/ratis/grpc/GrpcConfigKeys.java @@ -282,6 +282,15 @@ static GrpcTlsConfig tlsConf(Parameters parameters) { static void setTlsConf(Parameters parameters, GrpcTlsConfig conf) { parameters.put(TLS_CONF_PARAMETER, conf, TLS_CONF_CLASS); } + + String STUB_POOL_SIZE_KEY = PREFIX + ".stub.pool.size"; + int STUB_POOL_SIZE_DEFAULT = 10; + static int stubPoolSize(RaftProperties properties) { + return get(properties::getInt, STUB_POOL_SIZE_KEY, STUB_POOL_SIZE_DEFAULT, getDefaultLog()); + } + static void setStubPoolSize(RaftProperties properties, int size) { + setInt(properties::setInt, STUB_POOL_SIZE_KEY, size); + } } String MESSAGE_SIZE_MAX_KEY = PREFIX + ".message.size.max"; diff --git a/ratis-grpc/src/main/java/org/apache/ratis/grpc/server/GrpcServerProtocolClient.java b/ratis-grpc/src/main/java/org/apache/ratis/grpc/server/GrpcServerProtocolClient.java index 4a280ab335..2e936bb0b4 100644 --- a/ratis-grpc/src/main/java/org/apache/ratis/grpc/server/GrpcServerProtocolClient.java +++ b/ratis-grpc/src/main/java/org/apache/ratis/grpc/server/GrpcServerProtocolClient.java @@ -47,6 +47,7 @@ public class GrpcServerProtocolClient implements Closeable { // Common channel private final ManagedChannel channel; + private final GrpcStubPool pool; // Channel and stub for heartbeat private ManagedChannel hbChannel; private RaftServerProtocolServiceStub hbAsyncStub; @@ -59,7 +60,7 @@ public class GrpcServerProtocolClient implements Closeable { //visible for using in log / error messages AND to use in instrumented tests private final RaftPeerId raftPeerId; - public GrpcServerProtocolClient(RaftPeer target, int flowControlWindow, + public GrpcServerProtocolClient(RaftPeer target, int connections, int flowControlWindow, TimeDuration requestTimeout, GrpcTlsConfig tlsConfig, boolean separateHBChannel) { raftPeerId = target.getId(); LOG.info("Build channel for {}", target); @@ -72,6 +73,8 @@ public GrpcServerProtocolClient(RaftPeer target, int flowControlWindow, hbAsyncStub = RaftServerProtocolServiceGrpc.newStub(hbChannel); } requestTimeoutDuration = requestTimeout; + this.pool = new GrpcStubPool(target, connections, + ch -> RaftServerProtocolServiceGrpc.newStub(ch), tlsConfig); } private ManagedChannel buildChannel(RaftPeer target, int flowControlWindow, @@ -107,6 +110,7 @@ public void close() { GrpcUtil.shutdownManagedChannel(hbChannel); } GrpcUtil.shutdownManagedChannel(channel); + pool.close(); } public RequestVoteReplyProto requestVote(RequestVoteRequestProto request) { @@ -125,8 +129,36 @@ public StartLeaderElectionReplyProto startLeaderElection(StartLeaderElectionRequ } void readIndex(ReadIndexRequestProto request, StreamObserver s) { - asyncStub.withDeadlineAfter(requestTimeoutDuration.getDuration(), requestTimeoutDuration.getUnit()) - .readIndex(request, s); + GrpcStubPool.PooledStub p; + try { + p = pool.acquire(); + } catch (InterruptedException e) { + Thread.currentThread().interrupt(); + s.onError(e); return; + } + p.getStub().withDeadlineAfter(requestTimeoutDuration.getDuration(), requestTimeoutDuration.getUnit()) + .readIndex(request, new StreamObserver() { + @Override + public void onNext(ReadIndexReplyProto v) { + s.onNext(v); + } + @Override + public void onError(Throwable t) { + try { + s.onError(t); + } finally { + p.release(); + } + } + @Override + public void onCompleted() { + try { + s.onCompleted(); + } finally { + p.release(); + } + } + }); } CallStreamObserver appendEntries( diff --git a/ratis-grpc/src/main/java/org/apache/ratis/grpc/server/GrpcServicesImpl.java b/ratis-grpc/src/main/java/org/apache/ratis/grpc/server/GrpcServicesImpl.java index 853a420d1c..b686be0a29 100644 --- a/ratis-grpc/src/main/java/org/apache/ratis/grpc/server/GrpcServicesImpl.java +++ b/ratis-grpc/src/main/java/org/apache/ratis/grpc/server/GrpcServicesImpl.java @@ -113,6 +113,7 @@ public static final class Builder { private String serverHost; private int serverPort; private GrpcTlsConfig serverTlsConfig; + private int serverStubPoolSize; private SizeInBytes messageSizeMax; private SizeInBytes flowControlWindow; @@ -135,6 +136,7 @@ public Builder setServer(RaftServer raftServer) { this.flowControlWindow = GrpcConfigKeys.flowControlWindow(properties, LOG::info); this.requestTimeoutDuration = RaftServerConfigKeys.Rpc.requestTimeout(properties); this.separateHeartbeatChannel = GrpcConfigKeys.Server.heartbeatChannel(properties); + this.serverStubPoolSize = GrpcConfigKeys.Server.stubPoolSize(properties); final SizeInBytes appenderBufferSize = RaftServerConfigKeys.Log.Appender.bufferByteLimit(properties); final SizeInBytes gap = SizeInBytes.ONE_MB; @@ -155,7 +157,7 @@ public Builder setCustomizer(Customizer customizer) { } private GrpcServerProtocolClient newGrpcServerProtocolClient(RaftPeer target) { - return new GrpcServerProtocolClient(target, flowControlWindow.getSizeInt(), + return new GrpcServerProtocolClient(target, serverStubPoolSize, flowControlWindow.getSizeInt(), requestTimeoutDuration, serverTlsConfig, separateHeartbeatChannel); } diff --git a/ratis-grpc/src/main/java/org/apache/ratis/grpc/server/GrpcStubPool.java b/ratis-grpc/src/main/java/org/apache/ratis/grpc/server/GrpcStubPool.java new file mode 100644 index 0000000000..fcfb0f1b89 --- /dev/null +++ b/ratis-grpc/src/main/java/org/apache/ratis/grpc/server/GrpcStubPool.java @@ -0,0 +1,132 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.ratis.grpc.server; + +import org.apache.ratis.grpc.GrpcTlsConfig; +import org.apache.ratis.grpc.GrpcUtil; +import org.apache.ratis.protocol.RaftPeer; +import org.apache.ratis.thirdparty.io.grpc.ManagedChannel; +import org.apache.ratis.thirdparty.io.grpc.netty.GrpcSslContexts; +import org.apache.ratis.thirdparty.io.grpc.netty.NegotiationType; +import org.apache.ratis.thirdparty.io.grpc.netty.NettyChannelBuilder; +import org.apache.ratis.thirdparty.io.grpc.stub.AbstractStub; +import org.apache.ratis.thirdparty.io.netty.channel.ChannelOption; +import org.apache.ratis.thirdparty.io.netty.channel.WriteBufferWaterMark; +import org.apache.ratis.thirdparty.io.netty.channel.nio.NioEventLoopGroup; +import org.apache.ratis.thirdparty.io.netty.channel.socket.nio.NioSocketChannel; +import org.apache.ratis.thirdparty.io.netty.handler.ssl.SslContextBuilder; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.util.ArrayList; +import java.util.Collections; +import java.util.List; +import java.util.concurrent.Semaphore; +import java.util.concurrent.ThreadLocalRandom; +import java.util.concurrent.TimeUnit; +import java.util.concurrent.atomic.AtomicInteger; +import java.util.function.Function; + +final class GrpcStubPool> { + public static final Logger LOG = LoggerFactory.getLogger(GrpcStubPool.class); + + static final class PooledStub> { + private final ManagedChannel ch; + private final S stub; + private final Semaphore permits; + + PooledStub(ManagedChannel ch, S stub, int maxInflight) { + this.ch = ch; + this.stub = stub; + this.permits = new Semaphore(maxInflight); + } + + S getStub() { + return stub; + } + + void release() { + permits.release(); + } + } + + private final List> pool; + private final AtomicInteger rr = new AtomicInteger(); + private final NioEventLoopGroup elg; + private final int size; + + GrpcStubPool(RaftPeer target, int n, Function stubFactory, GrpcTlsConfig tlsConfig) { + this(target, n, stubFactory, tlsConfig, Math.max(2, Runtime.getRuntime().availableProcessors() / 2), 16); + } + + GrpcStubPool(RaftPeer target, int n, + Function stubFactory, GrpcTlsConfig tlsConf, + int elgThreads, int maxInflightPerConn) { + this.elg = new NioEventLoopGroup(elgThreads); + ArrayList> tmp = new ArrayList<>(n); + for (int i = 0; i < n; i++) { + NettyChannelBuilder channelBuilder = NettyChannelBuilder.forTarget(target.getAddress()) + .eventLoopGroup(elg) + .channelType(NioSocketChannel.class) + .keepAliveTime(30, TimeUnit.SECONDS) + .keepAliveWithoutCalls(true) + .idleTimeout(24, TimeUnit.HOURS) + .withOption(ChannelOption.WRITE_BUFFER_WATER_MARK, new WriteBufferWaterMark(64 << 10, 128 << 10)); + if (tlsConf != null) { + LOG.debug("Setting TLS for {}", target.getAddress()); + SslContextBuilder sslContextBuilder = GrpcSslContexts.forClient(); + GrpcUtil.setTrustManager(sslContextBuilder, tlsConf.getTrustManager()); + if (tlsConf.getMtlsEnabled()) { + GrpcUtil.setKeyManager(sslContextBuilder, tlsConf.getKeyManager()); + } + try { + channelBuilder.useTransportSecurity().sslContext(sslContextBuilder.build()); + } catch (Exception ex) { + throw new RuntimeException(ex); + } + } else { + channelBuilder.negotiationType(NegotiationType.PLAINTEXT); + } + ManagedChannel ch = channelBuilder.build(); + tmp.add(new PooledStub<>(ch, stubFactory.apply(ch), maxInflightPerConn)); + ch.getState(true); + } + this.pool = Collections.unmodifiableList(tmp); + this.size = n; + } + + PooledStub acquire() throws InterruptedException { + final int start = ThreadLocalRandom.current().nextInt(size); + for (int k = 0; k < size; k++) { + PooledStub p = pool.get((start + k) % size); + if (p.permits.tryAcquire()) { + return p; + } + } + final PooledStub p = pool.get(start); + p.permits.acquire(); + return p; + } + + public void close() { + for (PooledStub p : pool) { + p.ch.shutdown(); + } + elg.shutdownGracefully(); + } +} From 1f6a24d8b2e582624d168fcee2b9533ac26c7874 Mon Sep 17 00:00:00 2001 From: Tsz-Wo Nicholas Sze Date: Mon, 22 Sep 2025 10:12:59 -0700 Subject: [PATCH 200/397] RATIS-2261. Intermittent failure in TestRaftSnapshotWithGrpc.testInstallSnapshotDuringBootstrap. (#1287) --- .../statemachine/RaftSnapshotBaseTest.java | 81 +++++++++---------- .../impl/SimpleStateMachine4Testing.java | 11 ++- .../ratis/grpc/TestRaftSnapshotWithGrpc.java | 12 +-- .../netty/TestRaftSnapshotWithNetty.java | 11 +-- .../TestRaftSnapshotWithSimulatedRpc.java | 11 +-- 5 files changed, 54 insertions(+), 72 deletions(-) diff --git a/ratis-server/src/test/java/org/apache/ratis/statemachine/RaftSnapshotBaseTest.java b/ratis-server/src/test/java/org/apache/ratis/statemachine/RaftSnapshotBaseTest.java index 2c4ac2eeeb..44ae74c4c1 100644 --- a/ratis-server/src/test/java/org/apache/ratis/statemachine/RaftSnapshotBaseTest.java +++ b/ratis-server/src/test/java/org/apache/ratis/statemachine/RaftSnapshotBaseTest.java @@ -49,15 +49,12 @@ import org.apache.ratis.util.JavaUtils; import org.apache.ratis.util.LifeCycle; import org.apache.ratis.util.Slf4jUtils; -import org.junit.jupiter.api.AfterEach; import org.junit.jupiter.api.Assertions; -import org.junit.jupiter.api.BeforeEach; import org.junit.jupiter.api.Test; import org.slf4j.Logger; import org.slf4j.LoggerFactory; import java.io.File; -import java.io.IOException; import java.util.Arrays; import java.util.List; import java.util.Optional; @@ -67,11 +64,18 @@ import org.apache.ratis.thirdparty.com.codahale.metrics.Timer; import org.slf4j.event.Level; -public abstract class RaftSnapshotBaseTest extends BaseTest { +public abstract class RaftSnapshotBaseTest + extends BaseTest + implements MiniRaftCluster.Factory.Get { { Slf4jUtils.setLogLevel(RaftServer.Division.LOG, Level.DEBUG); Slf4jUtils.setLogLevel(RaftLog.LOG, Level.DEBUG); - Slf4jUtils.setLogLevel(RaftClient.LOG, Level.DEBUG); + + final RaftProperties p = getProperties(); + p.setClass(MiniRaftCluster.STATEMACHINE_CLASS_KEY, SimpleStateMachine4Testing.class, StateMachine.class); + RaftServerConfigKeys.Snapshot.setAutoTriggerThreshold(p, SNAPSHOT_TRIGGER_THRESHOLD); + RaftServerConfigKeys.Snapshot.setAutoTriggerEnabled(p, true); + RaftServerConfigKeys.LeaderElection.setMemberMajorityAdd(p, true); } static final Logger LOG = LoggerFactory.getLogger(RaftSnapshotBaseTest.class); @@ -119,29 +123,6 @@ public static void assertLogContent(RaftServer.Division server, boolean isLeader } } - private MiniRaftCluster cluster; - - public abstract MiniRaftCluster.Factory getFactory(); - - @BeforeEach - public void setup() throws IOException { - final RaftProperties prop = new RaftProperties(); - prop.setClass(MiniRaftCluster.STATEMACHINE_CLASS_KEY, - SimpleStateMachine4Testing.class, StateMachine.class); - RaftServerConfigKeys.Snapshot.setAutoTriggerThreshold( - prop, SNAPSHOT_TRIGGER_THRESHOLD); - RaftServerConfigKeys.Snapshot.setAutoTriggerEnabled(prop, true); - this.cluster = getFactory().newCluster(1, prop); - cluster.start(); - } - - @AfterEach - public void tearDown() { - if (cluster != null) { - cluster.shutdown(); - } - } - /** * Keep generating writing traffic and make sure snapshots are taken. * We then restart the whole raft peer and check if it can correctly load @@ -149,8 +130,13 @@ public void tearDown() { */ @Test public void testRestartPeer() throws Exception { - RaftTestUtil.waitForLeader(cluster); - final RaftPeerId leaderId = cluster.getLeader().getId(); + runWithNewCluster(1, this::runTestRestartPeer); + + } + + void runTestRestartPeer(CLUSTER cluster) throws Exception { + LOG.info("runTestRestartPeer"); + final RaftPeerId leaderId = RaftTestUtil.waitForLeader(cluster).getId(); int i = 0; try(final RaftClient client = cluster.createClient(leaderId)) { for (; i < SNAPSHOT_TRIGGER_THRESHOLD * 2 - 1; i++) { @@ -180,7 +166,7 @@ public void testRestartPeer() throws Exception { public static boolean exists(File f) { if (f.exists()) { - LOG.info("File exists: " + f); + LOG.info("File exists: {}", f); return true; } return false; @@ -193,11 +179,15 @@ public static boolean exists(File f) { */ @Test public void testBasicInstallSnapshot() throws Exception { + runWithNewCluster(1, this::runTestBasicInstallSnapshot); + } + + void runTestBasicInstallSnapshot(CLUSTER cluster) throws Exception { + LOG.info("runTestBasicInstallSnapshot"); final List logs; int i = 0; try { - RaftTestUtil.waitForLeader(cluster); - final RaftPeerId leaderId = cluster.getLeader().getId(); + final RaftPeerId leaderId = RaftTestUtil.waitForLeader(cluster).getId(); try(final RaftClient client = cluster.createClient(leaderId)) { for (; i < SNAPSHOT_TRIGGER_THRESHOLD * 2 - 1; i++) { @@ -236,16 +226,14 @@ public void testBasicInstallSnapshot() throws Exception { Assertions.assertTrue(client.io().send(new SimpleMessage("m" + i)).isSuccess()); } - // add two more peers - String[] newPeers = new String[]{"s3", "s4"}; - MiniRaftCluster.PeerChanges change = cluster.addNewPeers( - newPeers, true, false); + // add a new peer + final MiniRaftCluster.PeerChanges change = cluster.addNewPeers(1, true, true); // trigger setConfiguration RaftServerTestUtil.runWithMinorityPeers(cluster, Arrays.asList(change.allPeersInNewConf), peers -> cluster.setConfiguration(peers.toArray(RaftPeer.emptyArray()))); - for (String newPeer : newPeers) { - final RaftServer.Division s = cluster.getDivision(RaftPeerId.valueOf(newPeer)); + for (RaftPeer newPeer : change.newPeers) { + final RaftServer.Division s = cluster.getDivision(newPeer.getId()); SimpleStateMachine4Testing simpleStateMachine = SimpleStateMachine4Testing.get(s); Assertions.assertSame(LifeCycle.State.RUNNING, simpleStateMachine.getLifeCycleState()); } @@ -275,6 +263,11 @@ public void testBasicInstallSnapshot() throws Exception { */ @Test public void testInstallSnapshotDuringBootstrap() throws Exception { + runWithNewCluster(1, this::runTestInstallSnapshotDuringBootstrap); + } + + void runTestInstallSnapshotDuringBootstrap(CLUSTER cluster) throws Exception { + LOG.info("runTestInstallSnapshotDuringBootstrap"); int i = 0; try { RaftTestUtil.waitForLeader(cluster); @@ -299,16 +292,14 @@ public void testInstallSnapshotDuringBootstrap() throws Exception { assertLeaderContent(cluster); - // add two more peers - String[] newPeers = new String[]{"s3", "s4"}; - MiniRaftCluster.PeerChanges change = cluster.addNewPeers( - newPeers, true, false); + // add a new peer + final MiniRaftCluster.PeerChanges change = cluster.addNewPeers(1, true, true); // trigger setConfiguration RaftServerTestUtil.runWithMinorityPeers(cluster, Arrays.asList(change.allPeersInNewConf), peers -> cluster.setConfiguration(peers.toArray(RaftPeer.emptyArray()))); - for (String newPeer : newPeers) { - final RaftServer.Division s = cluster.getDivision(RaftPeerId.valueOf(newPeer)); + for (RaftPeer newPeer : change.newPeers) { + final RaftServer.Division s = cluster.getDivision(newPeer.getId()); SimpleStateMachine4Testing simpleStateMachine = SimpleStateMachine4Testing.get(s); Assertions.assertSame(LifeCycle.State.RUNNING, simpleStateMachine.getLifeCycleState()); } diff --git a/ratis-server/src/test/java/org/apache/ratis/statemachine/impl/SimpleStateMachine4Testing.java b/ratis-server/src/test/java/org/apache/ratis/statemachine/impl/SimpleStateMachine4Testing.java index afab27680f..1ffbdbcb99 100644 --- a/ratis-server/src/test/java/org/apache/ratis/statemachine/impl/SimpleStateMachine4Testing.java +++ b/ratis-server/src/test/java/org/apache/ratis/statemachine/impl/SimpleStateMachine4Testing.java @@ -210,7 +210,7 @@ private void put(LogEntryProto entry) { @Override public synchronized void initialize(RaftServer server, RaftGroupId raftGroupId, RaftStorage raftStorage) throws IOException { - LOG.info("Initializing " + this); + LOG.info("Initializing {}", this); this.groupId = raftGroupId; getLifeCycle().startAndTransition(() -> { super.initialize(server, raftGroupId, raftStorage); @@ -233,7 +233,10 @@ public synchronized void pause() { @Override public synchronized void reinitialize() throws IOException { - LOG.info("Reinitializing " + this); + LOG.info("Reinitializing {}", this); + indexMap.clear(); + dataMap.clear(); + loadSnapshot(storage.getLatestSnapshot()); if (getLifeCycleState() == LifeCycle.State.PAUSED) { getLifeCycle().transition(LifeCycle.State.STARTING); @@ -328,14 +331,14 @@ public CompletableFuture query(Message request) { final String string = request.getContent().toStringUtf8(); Exception exception; try { - LOG.info("query " + string); + LOG.info("query {}", string); final LogEntryProto entry = dataMap.get(string); if (entry != null) { return CompletableFuture.completedFuture(Message.valueOf(entry.toByteString())); } exception = new IndexOutOfBoundsException(getId() + ": LogEntry not found for query " + string); } catch (Exception e) { - LOG.warn("Failed request " + request, e); + LOG.warn("Failed request {}", request, e); exception = e; } return JavaUtils.completeExceptionally(new StateMachineException( diff --git a/ratis-test/src/test/java/org/apache/ratis/grpc/TestRaftSnapshotWithGrpc.java b/ratis-test/src/test/java/org/apache/ratis/grpc/TestRaftSnapshotWithGrpc.java index e6c2f6613b..7c94fb3bf5 100644 --- a/ratis-test/src/test/java/org/apache/ratis/grpc/TestRaftSnapshotWithGrpc.java +++ b/ratis-test/src/test/java/org/apache/ratis/grpc/TestRaftSnapshotWithGrpc.java @@ -20,22 +20,16 @@ import java.util.Optional; import org.apache.ratis.metrics.LongCounter; -import org.apache.ratis.server.impl.MiniRaftCluster; import org.apache.ratis.metrics.MetricRegistries; import org.apache.ratis.metrics.MetricRegistryInfo; import org.apache.ratis.metrics.RatisMetricRegistry; import org.apache.ratis.server.RaftServer; import org.apache.ratis.statemachine.RaftSnapshotBaseTest; -import org.apache.ratis.test.tag.Flaky; import org.junit.jupiter.api.Assertions; -@Flaky("RATIS-2261") -public class TestRaftSnapshotWithGrpc extends RaftSnapshotBaseTest { - @Override - public MiniRaftCluster.Factory getFactory() { - return MiniRaftClusterWithGrpc.FACTORY; - } - +public class TestRaftSnapshotWithGrpc + extends RaftSnapshotBaseTest + implements MiniRaftClusterWithGrpc.FactoryGet { @Override protected void verifyInstallSnapshotMetric(RaftServer.Division leader) { MetricRegistryInfo info = new MetricRegistryInfo(leader.getMemberId().toString(), diff --git a/ratis-test/src/test/java/org/apache/ratis/netty/TestRaftSnapshotWithNetty.java b/ratis-test/src/test/java/org/apache/ratis/netty/TestRaftSnapshotWithNetty.java index f1340efc74..ae16f41ed8 100644 --- a/ratis-test/src/test/java/org/apache/ratis/netty/TestRaftSnapshotWithNetty.java +++ b/ratis-test/src/test/java/org/apache/ratis/netty/TestRaftSnapshotWithNetty.java @@ -1,4 +1,4 @@ -/** +/* * 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 @@ -17,12 +17,9 @@ */ package org.apache.ratis.netty; -import org.apache.ratis.server.impl.MiniRaftCluster; import org.apache.ratis.statemachine.RaftSnapshotBaseTest; -public class TestRaftSnapshotWithNetty extends RaftSnapshotBaseTest { - @Override - public MiniRaftCluster.Factory getFactory() { - return MiniRaftClusterWithNetty.FACTORY; - } +public class TestRaftSnapshotWithNetty + extends RaftSnapshotBaseTest + implements MiniRaftClusterWithNetty.FactoryGet { } diff --git a/ratis-test/src/test/java/org/apache/ratis/server/simulation/TestRaftSnapshotWithSimulatedRpc.java b/ratis-test/src/test/java/org/apache/ratis/server/simulation/TestRaftSnapshotWithSimulatedRpc.java index 1c76f7b005..62ee387dec 100644 --- a/ratis-test/src/test/java/org/apache/ratis/server/simulation/TestRaftSnapshotWithSimulatedRpc.java +++ b/ratis-test/src/test/java/org/apache/ratis/server/simulation/TestRaftSnapshotWithSimulatedRpc.java @@ -1,4 +1,4 @@ -/** +/* * 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 @@ -17,12 +17,9 @@ */ package org.apache.ratis.server.simulation; -import org.apache.ratis.server.impl.MiniRaftCluster; import org.apache.ratis.statemachine.RaftSnapshotBaseTest; -public class TestRaftSnapshotWithSimulatedRpc extends RaftSnapshotBaseTest { - @Override - public MiniRaftCluster.Factory getFactory() { - return MiniRaftClusterWithSimulatedRpc.FACTORY; - } +public class TestRaftSnapshotWithSimulatedRpc + extends RaftSnapshotBaseTest + implements MiniRaftClusterWithSimulatedRpc.FactoryGet { } From 744ea5aac677381820eb8a27e84f1e422314c80b Mon Sep 17 00:00:00 2001 From: Tsz-Wo Nicholas Sze Date: Mon, 22 Sep 2025 11:47:08 -0700 Subject: [PATCH 201/397] RATIS-2333. Fix TestInstallSnapshotNotificationWithGrpc failure. (#1289) --- .../java/org/apache/ratis/grpc/GrpcUtil.java | 18 ++++++++------- .../server/GrpcServerProtocolService.java | 8 +++++-- .../impl/SnapshotInstallationHandler.java | 3 ++- .../InstallSnapshotNotificationTests.java | 22 +++++++++---------- .../statemachine/RaftSnapshotBaseTest.java | 10 ++++----- 5 files changed, 34 insertions(+), 27 deletions(-) diff --git a/ratis-grpc/src/main/java/org/apache/ratis/grpc/GrpcUtil.java b/ratis-grpc/src/main/java/org/apache/ratis/grpc/GrpcUtil.java index 3645980abf..2f9ee01ecb 100644 --- a/ratis-grpc/src/main/java/org/apache/ratis/grpc/GrpcUtil.java +++ b/ratis-grpc/src/main/java/org/apache/ratis/grpc/GrpcUtil.java @@ -138,8 +138,10 @@ static Throwable tryUnwrapThrowable(StatusRuntimeException se) { static long getCallId(Throwable t) { if (t instanceof StatusRuntimeException) { final Metadata trailers = ((StatusRuntimeException)t).getTrailers(); - String callId = trailers.get(CALL_ID); - return callId != null ? Integer.parseInt(callId) : -1; + if (trailers != null) { + final String callId = trailers.get(CALL_ID); + return callId != null ? Integer.parseInt(callId) : -1; + } } return -1; } @@ -147,8 +149,8 @@ static long getCallId(Throwable t) { static boolean isHeartbeat(Throwable t) { if (t instanceof StatusRuntimeException) { final Metadata trailers = ((StatusRuntimeException)t).getTrailers(); - String isHeartbeat = trailers != null ? trailers.get(HEARTBEAT) : null; - return isHeartbeat != null && Boolean.valueOf(isHeartbeat); + final String isHeartbeat = trailers != null ? trailers.get(HEARTBEAT) : null; + return Boolean.parseBoolean(isHeartbeat); } return false; } @@ -156,7 +158,7 @@ static boolean isHeartbeat(Throwable t) { static IOException unwrapIOException(Throwable t) { final IOException e; if (t instanceof StatusRuntimeException) { - e = GrpcUtil.unwrapException((StatusRuntimeException) t); + e = unwrapException((StatusRuntimeException) t); } else { e = IOUtils.asIOException(t); } @@ -172,7 +174,7 @@ static void asyncCall( supplier.get().whenComplete((reply, exception) -> { if (exception != null) { warning.accept(exception); - responseObserver.onError(GrpcUtil.wrapException(exception)); + responseObserver.onError(wrapException(exception)); } else { responseObserver.onNext(toProto.apply(reply)); responseObserver.onCompleted(); @@ -180,7 +182,7 @@ static void asyncCall( }); } catch (Exception e) { warning.accept(e); - responseObserver.onError(GrpcUtil.wrapException(e)); + responseObserver.onError(wrapException(e)); } } @@ -189,7 +191,7 @@ static void warn(Logger log, Supplier message, Throwable t) { } class StatusRuntimeExceptionMetadataBuilder { - private Metadata trailers = new Metadata(); + private final Metadata trailers = new Metadata(); StatusRuntimeExceptionMetadataBuilder(Throwable t) { trailers.put(EXCEPTION_TYPE_KEY, t.getClass().getCanonicalName()); diff --git a/ratis-grpc/src/main/java/org/apache/ratis/grpc/server/GrpcServerProtocolService.java b/ratis-grpc/src/main/java/org/apache/ratis/grpc/server/GrpcServerProtocolService.java index 75362fcf88..b123c44a77 100644 --- a/ratis-grpc/src/main/java/org/apache/ratis/grpc/server/GrpcServerProtocolService.java +++ b/ratis-grpc/src/main/java/org/apache/ratis/grpc/server/GrpcServerProtocolService.java @@ -24,6 +24,8 @@ import org.apache.ratis.server.RaftServer; import org.apache.ratis.server.protocol.RaftServerProtocol; import org.apache.ratis.server.util.ServerStringUtils; +import org.apache.ratis.thirdparty.com.google.protobuf.MessageOrBuilder; +import org.apache.ratis.thirdparty.com.google.protobuf.TextFormat; import org.apache.ratis.thirdparty.io.grpc.Status; import org.apache.ratis.thirdparty.io.grpc.StatusRuntimeException; import org.apache.ratis.thirdparty.io.grpc.stub.StreamObserver; @@ -67,7 +69,8 @@ CompletableFuture getFuture() { } } - abstract class ServerRequestStreamObserver implements StreamObserver { + abstract class ServerRequestStreamObserver + implements StreamObserver { private final RaftServer.Op op; private final Supplier nameSupplier; private final StreamObserver responseObserver; @@ -172,7 +175,8 @@ public void onCompleted() { getId(), op, getPreviousRequestString(), suffix)); requestFuture.get().thenAccept(reply -> { BatchLogger.print(BatchLogKey.COMPLETED_REPLY, getName(), - suffix -> LOG.info("{}: Completed {}, lastReply: {} {}", getId(), op, reply, suffix)); + suffix -> LOG.info("{}: Completed {}, lastReply: {} {}", + getId(), op, TextFormat.shortDebugString(reply), suffix)); responseObserver.onCompleted(); }); } diff --git a/ratis-server/src/main/java/org/apache/ratis/server/impl/SnapshotInstallationHandler.java b/ratis-server/src/main/java/org/apache/ratis/server/impl/SnapshotInstallationHandler.java index 4f1ac4177f..2a2e9e2b58 100644 --- a/ratis-server/src/main/java/org/apache/ratis/server/impl/SnapshotInstallationHandler.java +++ b/ratis-server/src/main/java/org/apache/ratis/server/impl/SnapshotInstallationHandler.java @@ -36,6 +36,7 @@ import org.apache.ratis.server.protocol.TermIndex; import org.apache.ratis.server.raftlog.LogProtoUtils; import org.apache.ratis.server.util.ServerStringUtils; +import org.apache.ratis.thirdparty.com.google.protobuf.TextFormat; import org.apache.ratis.util.BatchLogger; import org.apache.ratis.util.CodeInjectionForTesting; import org.apache.ratis.util.LifeCycle; @@ -144,7 +145,7 @@ private InstallSnapshotReplyProto installSnapshotImpl(InstallSnapshotRequestProt final LogEntryProto proto = request.getLastRaftConfigurationLogEntryProto(); state.truncate(proto.getIndex()); if (!state.getRaftConf().equals(LogProtoUtils.toRaftConfiguration(proto))) { - LOG.info("{}: set new configuration {} from snapshot", getMemberId(), proto); + LOG.info("{}: set new configuration {} from snapshot", getMemberId(), TextFormat.shortDebugString(proto)); state.setRaftConf(proto); state.writeRaftConfiguration(proto); server.getStateMachine().event().notifyConfigurationChanged( diff --git a/ratis-server/src/test/java/org/apache/ratis/InstallSnapshotNotificationTests.java b/ratis-server/src/test/java/org/apache/ratis/InstallSnapshotNotificationTests.java index 411c93120f..75c62127ba 100644 --- a/ratis-server/src/test/java/org/apache/ratis/InstallSnapshotNotificationTests.java +++ b/ratis-server/src/test/java/org/apache/ratis/InstallSnapshotNotificationTests.java @@ -79,6 +79,7 @@ public abstract class InstallSnapshotNotificationTests cluster.setConfiguration(peers.toArray(RaftPeer.emptyArray()))); @@ -389,9 +389,9 @@ private void testInstallSnapshotNotificationCount(CLUSTER cluster) throws Except follower.getRaftLog().getNextIndex()); } - // Add two more peers who will need snapshots from the leader. - final MiniRaftCluster.PeerChanges change = cluster.addNewPeers(2, true, - true); + // Add new peer(s) who will need snapshots from the leader. + final int numNewPeers = 1; + final MiniRaftCluster.PeerChanges change = cluster.addNewPeers(numNewPeers, true, true); // trigger setConfiguration RaftServerTestUtil.runWithMinorityPeers(cluster, Arrays.asList(change.allPeersInNewConf), peers -> cluster.setConfiguration(peers.toArray(RaftPeer.emptyArray()))); @@ -412,7 +412,7 @@ private void testInstallSnapshotNotificationCount(CLUSTER cluster) throws Except } // Make sure each new peer got one snapshot notification. - Assertions.assertEquals(2, numSnapshotRequests.get()); + Assertions.assertEquals(numNewPeers, numSnapshotRequests.get()); } finally { cluster.shutdown(); @@ -556,9 +556,9 @@ private void testInstallSnapshotDuringBootstrap(CLUSTER cluster) throws Exceptio final boolean set = LEADER_SNAPSHOT_INFO_REF.compareAndSet(null, leaderSnapshotInfo); Assertions.assertTrue(set); - // add two more peers - final MiniRaftCluster.PeerChanges change = cluster.addNewPeers(2, true, - true); + // Add new peer(s) + final int numNewPeers = 1; + final MiniRaftCluster.PeerChanges change = cluster.addNewPeers(numNewPeers, true, true); // trigger setConfiguration RaftServerTestUtil.runWithMinorityPeers(cluster, Arrays.asList(change.allPeersInNewConf), peers -> cluster.setConfiguration(peers.toArray(RaftPeer.emptyArray()))); @@ -573,7 +573,7 @@ private void testInstallSnapshotDuringBootstrap(CLUSTER cluster) throws Exceptio } // Make sure each new peer got at least one snapshot notification. - Assertions.assertTrue(2 <= numSnapshotRequests.get()); + Assertions.assertTrue(numNewPeers <= numSnapshotRequests.get()); } finally { cluster.shutdown(); } diff --git a/ratis-server/src/test/java/org/apache/ratis/statemachine/RaftSnapshotBaseTest.java b/ratis-server/src/test/java/org/apache/ratis/statemachine/RaftSnapshotBaseTest.java index 44ae74c4c1..cd1a2eb55d 100644 --- a/ratis-server/src/test/java/org/apache/ratis/statemachine/RaftSnapshotBaseTest.java +++ b/ratis-server/src/test/java/org/apache/ratis/statemachine/RaftSnapshotBaseTest.java @@ -96,16 +96,16 @@ public static void assertLeaderContent(MiniRaftCluster cluster) throws Exception assertLogContent(leader, true); } - public static void assertLogContent(RaftServer.Division server, boolean isLeader) throws Exception { + public static void checkMetadataEntry(RaftServer.Division server) throws Exception { final RaftLog log = server.getRaftLog(); final long lastIndex = log.getLastEntryTermIndex().getIndex(); final LogEntryProto e = log.get(lastIndex); Assertions.assertTrue(e.hasMetadataEntry()); + Assertions.assertEquals(log.getLastCommittedIndex() - 1, e.getMetadataEntry().getCommitIndex()); + } - JavaUtils.attemptRepeatedly(() -> { - Assertions.assertEquals(log.getLastCommittedIndex() - 1, e.getMetadataEntry().getCommitIndex()); - return null; - }, 50, BaseTest.HUNDRED_MILLIS, "CheckMetadataEntry", LOG); + public static void assertLogContent(RaftServer.Division server, boolean isLeader) throws Exception { + JavaUtils.attempt(() -> checkMetadataEntry(server), 50, HUNDRED_MILLIS, "checkMetadataEntry", LOG); SimpleStateMachine4Testing simpleStateMachine = SimpleStateMachine4Testing.get(server); if (isLeader) { From 58f14362f33137be775a4c3df8b84972dd847bee Mon Sep 17 00:00:00 2001 From: slfan1989 <55643692+slfan1989@users.noreply.github.com> Date: Thu, 25 Sep 2025 00:07:20 +0800 Subject: [PATCH 202/397] RATIS-2322. Upgrade JUnit Jupiter to 5.13.3 and Maven Surefire Plugin to 3.5.3. (#1281) --- pom.xml | 4 ++-- ratis-client/pom.xml | 11 +++++++++++ ratis-docs/pom.xml | 13 +++++++++++++ ratis-metrics-api/pom.xml | 11 +++++++++++ ratis-netty/pom.xml | 11 +++++++++++ ratis-proto/pom.xml | 10 ++++++++++ ratis-resource-bundle/pom.xml | 13 ++++++++++++- ratis-server-api/pom.xml | 11 +++++++++++ ratis-shell/pom.xml | 11 +++++++++++ ratis-tools/pom.xml | 10 ++++++++++ 10 files changed, 102 insertions(+), 3 deletions(-) diff --git a/pom.xml b/pom.xml index b93500d3c1..b32c7ac1db 100644 --- a/pom.xml +++ b/pom.xml @@ -173,7 +173,7 @@ 1.6.1 3.3.0 3.6.0 - 3.0.0-M4 + 3.5.3 3.5.3 3.4.0 5.0.0.4389 @@ -220,7 +220,7 @@ 4 2.0.7 - 5.12.2 + 5.13.3 4.11.0 0.8.12 1.3.5 diff --git a/ratis-client/pom.xml b/ratis-client/pom.xml index 8a5008eafb..b73bbebbc7 100644 --- a/ratis-client/pom.xml +++ b/ratis-client/pom.xml @@ -47,5 +47,16 @@ org.slf4j slf4j-api + + + org.junit.jupiter + junit-jupiter-engine + test + + + org.junit.platform + junit-platform-launcher + test + diff --git a/ratis-docs/pom.xml b/ratis-docs/pom.xml index 08a86fb033..91b92f2f86 100644 --- a/ratis-docs/pom.xml +++ b/ratis-docs/pom.xml @@ -32,4 +32,17 @@ https://maven.apache.org/xsd/maven-4.0.0.xsd"> true + + + org.junit.jupiter + junit-jupiter-engine + test + + + org.junit.platform + junit-platform-launcher + test + + + diff --git a/ratis-metrics-api/pom.xml b/ratis-metrics-api/pom.xml index ba3b36b320..b101e13f23 100644 --- a/ratis-metrics-api/pom.xml +++ b/ratis-metrics-api/pom.xml @@ -39,5 +39,16 @@ org.slf4j slf4j-api + + + org.junit.jupiter + junit-jupiter-engine + test + + + org.junit.platform + junit-platform-launcher + test + diff --git a/ratis-netty/pom.xml b/ratis-netty/pom.xml index b5714a943e..29480d9987 100644 --- a/ratis-netty/pom.xml +++ b/ratis-netty/pom.xml @@ -78,5 +78,16 @@ slf4j-api + + org.junit.jupiter + junit-jupiter-engine + test + + + org.junit.platform + junit-platform-launcher + test + + diff --git a/ratis-proto/pom.xml b/ratis-proto/pom.xml index 9a548b4066..771187b583 100644 --- a/ratis-proto/pom.xml +++ b/ratis-proto/pom.xml @@ -179,5 +179,15 @@ jakarta.annotation jakarta.annotation-api + + org.junit.jupiter + junit-jupiter-engine + test + + + org.junit.platform + junit-platform-launcher + test + diff --git a/ratis-resource-bundle/pom.xml b/ratis-resource-bundle/pom.xml index 3403fab8f8..5a3d1ff876 100644 --- a/ratis-resource-bundle/pom.xml +++ b/ratis-resource-bundle/pom.xml @@ -36,7 +36,18 @@ true - + + + org.junit.jupiter + junit-jupiter-engine + test + + + org.junit.platform + junit-platform-launcher + test + + diff --git a/ratis-server-api/pom.xml b/ratis-server-api/pom.xml index 812da9f023..889277c146 100644 --- a/ratis-server-api/pom.xml +++ b/ratis-server-api/pom.xml @@ -52,5 +52,16 @@ org.slf4j slf4j-api + + + org.junit.jupiter + junit-jupiter-engine + test + + + org.junit.platform + junit-platform-launcher + test + diff --git a/ratis-shell/pom.xml b/ratis-shell/pom.xml index 7b4534a60c..ab2a3da2ad 100644 --- a/ratis-shell/pom.xml +++ b/ratis-shell/pom.xml @@ -53,6 +53,17 @@ org.slf4j slf4j-simple + + + org.junit.jupiter + junit-jupiter-engine + test + + + org.junit.platform + junit-platform-launcher + test + diff --git a/ratis-tools/pom.xml b/ratis-tools/pom.xml index 0fe13f4023..f1d0c33d9b 100644 --- a/ratis-tools/pom.xml +++ b/ratis-tools/pom.xml @@ -41,5 +41,15 @@ org.apache.ratis ratis-common + + org.junit.jupiter + junit-jupiter-engine + test + + + org.junit.platform + junit-platform-launcher + test + From bdcba19a707e51e9aafa7851f69cf47356cc216d Mon Sep 17 00:00:00 2001 From: Tsz-Wo Nicholas Sze Date: Wed, 24 Sep 2025 09:38:15 -0700 Subject: [PATCH 203/397] RATIS-2336. PeerProxyMap.getProxy() could return null. (#1290) --- .../src/main/java/org/apache/ratis/util/PeerProxyMap.java | 7 +++++-- 1 file changed, 5 insertions(+), 2 deletions(-) diff --git a/ratis-common/src/main/java/org/apache/ratis/util/PeerProxyMap.java b/ratis-common/src/main/java/org/apache/ratis/util/PeerProxyMap.java index eda41a0096..868b65cc37 100644 --- a/ratis-common/src/main/java/org/apache/ratis/util/PeerProxyMap.java +++ b/ratis-common/src/main/java/org/apache/ratis/util/PeerProxyMap.java @@ -59,7 +59,10 @@ RaftPeer getPeer() { } PROXY getProxy() throws IOException { - if (proxy == null) { + final PROXY p = proxy; + if (p != null) { + return p; + } else { synchronized (this) { if (proxy == null) { final LifeCycle.State current = lifeCycle.getCurrentState(); @@ -69,9 +72,9 @@ PROXY getProxy() throws IOException { lifeCycle.startAndTransition( () -> proxy = createProxyImpl(peer), IOException.class); } + return Objects.requireNonNull(proxy, "proxy"); } } - return proxy; } Optional setNullProxyAndClose() { From 3b6350e6be8b1f4f960e925eb96f8ed874f2a512 Mon Sep 17 00:00:00 2001 From: Tsz-Wo Nicholas Sze Date: Fri, 26 Sep 2025 10:59:34 -0700 Subject: [PATCH 204/397] RATIS-2337. Refactor MiniRaftCluster.PeerChanges. (#1291) --- .../ratis/InstallSnapshotFromLeaderTests.java | 16 ++-- .../InstallSnapshotNotificationTests.java | 34 +++---- .../apache/ratis/RaftExceptionBaseTest.java | 7 +- .../org/apache/ratis/RetryCacheTests.java | 13 ++- .../server/impl/LeaderElectionTests.java | 22 ++--- .../ratis/server/impl/MiniRaftCluster.java | 38 +++----- .../apache/ratis/server/impl/PeerChanges.java | 71 +++++++++++++++ .../impl/RaftReconfigurationBaseTest.java | 90 +++++++++---------- .../ratis/server/impl/RaftServerTestUtil.java | 20 ++--- .../statemachine/RaftSnapshotBaseTest.java | 20 ++--- .../cli/sh/PeerCommandIntegrationTest.java | 15 ++-- 11 files changed, 193 insertions(+), 153 deletions(-) create mode 100644 ratis-server/src/test/java/org/apache/ratis/server/impl/PeerChanges.java diff --git a/ratis-server/src/test/java/org/apache/ratis/InstallSnapshotFromLeaderTests.java b/ratis-server/src/test/java/org/apache/ratis/InstallSnapshotFromLeaderTests.java index ba6bca5b5d..687e1c1a51 100644 --- a/ratis-server/src/test/java/org/apache/ratis/InstallSnapshotFromLeaderTests.java +++ b/ratis-server/src/test/java/org/apache/ratis/InstallSnapshotFromLeaderTests.java @@ -30,6 +30,7 @@ import org.apache.ratis.server.RaftServer; import org.apache.ratis.server.RaftServerConfigKeys; import org.apache.ratis.server.impl.MiniRaftCluster; +import org.apache.ratis.server.impl.PeerChanges; import org.apache.ratis.server.impl.RaftServerTestUtil; import org.apache.ratis.server.protocol.TermIndex; import org.apache.ratis.server.raftlog.RaftLog; @@ -115,13 +116,12 @@ private void testMultiFileInstallSnapshot(CLUSTER cluster) throws Exception { Assertions.assertEquals(3, snapshot.getFiles().size()); // add two more peers - final MiniRaftCluster.PeerChanges change = cluster.addNewPeers(2, true, + final PeerChanges change = cluster.addNewPeers(2, true, true); // trigger setConfiguration - cluster.setConfiguration(change.allPeersInNewConf); + cluster.setConfiguration(change.getPeersInNewConf()); - RaftServerTestUtil - .waitAndCheckNewConf(cluster, change.allPeersInNewConf, 0, null); + RaftServerTestUtil.waitAndCheckNewConf(cluster, change.getPeersInNewConf(), 0, null); // Check the installed snapshot file number on each Follower matches with the // leader snapshot. @@ -161,17 +161,17 @@ private void testInstallSnapshotDuringLeaderSwitch(CLUSTER cluster) throws Excep } // add two more peers and install snapshot from leaders - final MiniRaftCluster.PeerChanges change = cluster.addNewPeers(2, true, + final PeerChanges change = cluster.addNewPeers(2, true, true); try (final RaftClient client = cluster.createClient(leaderId, RetryPolicies.noRetry())) { final RaftException e = Assertions.assertThrows(RaftException.class, - () -> client.admin().setConfiguration(change.allPeersInNewConf)); + () -> client.admin().setConfiguration(change.getPeersInNewConf())); Assertions.assertTrue( e instanceof RaftRetryFailureException || e instanceof ReconfigurationTimeoutException, () -> "Unexpected exception: " + e); } - final SnapshotInfo snapshotInfo = cluster.getDivision(change.newPeers[0].getId()) + final SnapshotInfo snapshotInfo = cluster.getDivision(change.getAddedPeers().get(0).getId()) .getStateMachine().getLatestSnapshot(); Assertions.assertNotNull(snapshotInfo); @@ -184,7 +184,7 @@ private void testInstallSnapshotDuringLeaderSwitch(CLUSTER cluster) throws Excep try (final RaftClient client = cluster.createClient(cluster.getLeader().getId())) { // successfully setConfiguration during leader switch - final RaftClientReply setConf = client.admin().setConfiguration(change.allPeersInNewConf); + final RaftClientReply setConf = client.admin().setConfiguration(change.getPeersInNewConf()); Assertions.assertTrue(setConf.isSuccess()); RaftTestUtil.deIsolate(cluster, leaderId); diff --git a/ratis-server/src/test/java/org/apache/ratis/InstallSnapshotNotificationTests.java b/ratis-server/src/test/java/org/apache/ratis/InstallSnapshotNotificationTests.java index 75c62127ba..e03171d66b 100644 --- a/ratis-server/src/test/java/org/apache/ratis/InstallSnapshotNotificationTests.java +++ b/ratis-server/src/test/java/org/apache/ratis/InstallSnapshotNotificationTests.java @@ -26,6 +26,7 @@ import org.apache.ratis.server.RaftServer; import org.apache.ratis.server.RaftServerConfigKeys; import org.apache.ratis.server.impl.MiniRaftCluster; +import org.apache.ratis.server.impl.PeerChanges; import org.apache.ratis.server.impl.RaftServerTestUtil; import org.apache.ratis.server.protocol.TermIndex; import org.apache.ratis.server.raftlog.RaftLog; @@ -49,7 +50,6 @@ import java.io.IOException; import java.nio.file.Files; import java.nio.file.Path; -import java.util.Arrays; import java.util.Collections; import java.util.List; import java.util.concurrent.CompletableFuture; @@ -241,13 +241,11 @@ private void testAddNewFollowers(CLUSTER cluster, int numRequests) throws Except Assertions.assertTrue(set); // Add new peer(s) - final MiniRaftCluster.PeerChanges change = cluster.addNewPeers(1, true, true); + final PeerChanges change = cluster.addNewPeers(1, true, true); // trigger setConfiguration - RaftServerTestUtil.runWithMinorityPeers(cluster, Arrays.asList(change.allPeersInNewConf), - peers -> cluster.setConfiguration(peers.toArray(RaftPeer.emptyArray()))); + RaftServerTestUtil.runWithMinorityPeers(cluster, change.getPeersInNewConf(), cluster::setConfiguration); - RaftServerTestUtil - .waitAndCheckNewConf(cluster, change.allPeersInNewConf, 0, null); + RaftServerTestUtil.waitAndCheckNewConf(cluster, change.getPeersInNewConf(), 0, null); // Check the installed snapshot index on each Follower matches with the // leader snapshot. @@ -391,12 +389,10 @@ private void testInstallSnapshotNotificationCount(CLUSTER cluster) throws Except // Add new peer(s) who will need snapshots from the leader. final int numNewPeers = 1; - final MiniRaftCluster.PeerChanges change = cluster.addNewPeers(numNewPeers, true, true); + final PeerChanges change = cluster.addNewPeers(numNewPeers, true, true); // trigger setConfiguration - RaftServerTestUtil.runWithMinorityPeers(cluster, Arrays.asList(change.allPeersInNewConf), - peers -> cluster.setConfiguration(peers.toArray(RaftPeer.emptyArray()))); - RaftServerTestUtil - .waitAndCheckNewConf(cluster, change.allPeersInNewConf, 0, null); + RaftServerTestUtil.runWithMinorityPeers(cluster, change.getPeersInNewConf(), cluster::setConfiguration); + RaftServerTestUtil.waitAndCheckNewConf(cluster, change.getPeersInNewConf(), 0, null); // Generate more data. try (final RaftClient client = cluster.createClient(leader.getId())) { @@ -479,13 +475,10 @@ private void testInstallSnapshotInstalledEvent(CLUSTER cluster) throws Exception Assertions.assertTrue(set); // add one new peer - final MiniRaftCluster.PeerChanges change = cluster.addNewPeers(1, true, true); + final PeerChanges change = cluster.addNewPeers(1, true, true); // trigger setConfiguration - RaftServerTestUtil.runWithMinorityPeers(cluster, Arrays.asList(change.allPeersInNewConf), - peers -> cluster.setConfiguration(peers.toArray(RaftPeer.emptyArray()))); - - RaftServerTestUtil - .waitAndCheckNewConf(cluster, change.allPeersInNewConf, 0, null); + RaftServerTestUtil.runWithMinorityPeers(cluster, change.getPeersInNewConf(), cluster::setConfiguration); + RaftServerTestUtil.waitAndCheckNewConf(cluster, change.getPeersInNewConf(), 0, null); // Check the installed snapshot index on each Follower matches with the // leader snapshot. @@ -558,12 +551,11 @@ private void testInstallSnapshotDuringBootstrap(CLUSTER cluster) throws Exceptio // Add new peer(s) final int numNewPeers = 1; - final MiniRaftCluster.PeerChanges change = cluster.addNewPeers(numNewPeers, true, true); + final PeerChanges change = cluster.addNewPeers(numNewPeers, true, true); // trigger setConfiguration - RaftServerTestUtil.runWithMinorityPeers(cluster, Arrays.asList(change.allPeersInNewConf), - peers -> cluster.setConfiguration(peers.toArray(RaftPeer.emptyArray()))); + RaftServerTestUtil.runWithMinorityPeers(cluster, change.getPeersInNewConf(), cluster::setConfiguration); - RaftServerTestUtil.waitAndCheckNewConf(cluster, change.allPeersInNewConf, 0, null); + RaftServerTestUtil.waitAndCheckNewConf(cluster, change.getPeersInNewConf(), 0, null); // Check the installed snapshot index on each Follower matches with the // leader snapshot. diff --git a/ratis-server/src/test/java/org/apache/ratis/RaftExceptionBaseTest.java b/ratis-server/src/test/java/org/apache/ratis/RaftExceptionBaseTest.java index 9f602e0815..a08731c580 100644 --- a/ratis-server/src/test/java/org/apache/ratis/RaftExceptionBaseTest.java +++ b/ratis-server/src/test/java/org/apache/ratis/RaftExceptionBaseTest.java @@ -28,6 +28,7 @@ import org.apache.ratis.server.RaftServer; import org.apache.ratis.server.RaftServerConfigKeys; import org.apache.ratis.server.impl.MiniRaftCluster; +import org.apache.ratis.server.impl.PeerChanges; import org.apache.ratis.server.raftlog.RaftLog; import org.apache.ratis.server.raftlog.RaftLogIOException; import org.apache.ratis.util.JavaUtils; @@ -109,12 +110,12 @@ void runTestNotLeaderExceptionWithReconf(CLUSTER cluster) throws Exception { final RaftPeerId newLeader = RaftTestUtil.changeLeader(cluster, oldLeader); // add two more peers - MiniRaftCluster.PeerChanges change = cluster.addNewPeers(new String[]{ + PeerChanges change = cluster.addNewPeers(new String[]{ "ss1", "ss2"}, true, false); // trigger setConfiguration - LOG.info("Start changing the configuration: {}", Arrays.asList(change.allPeersInNewConf)); + LOG.info("Start changing the configuration: {}", change.getPeersInNewConf()); try (final RaftClient c2 = cluster.createClient(newLeader)) { - RaftClientReply reply = c2.admin().setConfiguration(change.allPeersInNewConf); + RaftClientReply reply = c2.admin().setConfiguration(change.getPeersInNewConf()); Assertions.assertTrue(reply.isSuccess()); } LOG.info(cluster.printServers()); diff --git a/ratis-server/src/test/java/org/apache/ratis/RetryCacheTests.java b/ratis-server/src/test/java/org/apache/ratis/RetryCacheTests.java index 5a363bf081..0674c3ed94 100644 --- a/ratis-server/src/test/java/org/apache/ratis/RetryCacheTests.java +++ b/ratis-server/src/test/java/org/apache/ratis/RetryCacheTests.java @@ -18,7 +18,7 @@ package org.apache.ratis; import org.apache.ratis.server.impl.MiniRaftCluster; -import org.apache.ratis.server.impl.MiniRaftCluster.PeerChanges; +import org.apache.ratis.server.impl.PeerChanges; import org.apache.ratis.RaftTestUtil.SimpleMessage; import org.apache.ratis.client.RaftClient; import org.apache.ratis.client.RaftClientRpc; @@ -39,12 +39,10 @@ import org.junit.jupiter.api.Test; import org.slf4j.event.Level; -import java.util.Arrays; import java.util.Collections; +import java.util.List; import java.util.concurrent.TimeUnit; -import static java.util.Arrays.asList; - public abstract class RetryCacheTests extends BaseTest implements MiniRaftCluster.Factory.Get { @@ -139,10 +137,9 @@ void runTestRetryOnNewLeader(CLUSTER cluster) throws Exception { // trigger the reconfiguration, make sure the original leader is kicked out final PeerChanges change = cluster.removePeers(2, true, Collections.emptyList()); - final RaftPeer[] allPeers = change.allPeersInNewConf; + final List allPeers = change.getPeersInNewConf(); // trigger setConfiguration - RaftServerTestUtil.runWithMinorityPeers(cluster, Arrays.asList(allPeers), - peers -> cluster.setConfiguration(peers.toArray(RaftPeer.emptyArray()))); + RaftServerTestUtil.runWithMinorityPeers(cluster, allPeers, cluster::setConfiguration); final RaftPeerId newLeaderId = JavaUtils.attemptRepeatedly(() -> { final RaftPeerId id = RaftTestUtil.waitForLeader(cluster).getId(); @@ -153,7 +150,7 @@ void runTestRetryOnNewLeader(CLUSTER cluster) throws Exception { // same clientId and callId in the request r = cluster.newRaftClientRequest(client.getId(), newLeaderId, callId, new SimpleMessage("message")); - rpc.addRaftPeers(Arrays.asList(change.newPeers)); + rpc.addRaftPeers(change.getAddedPeers()); for (int i = 0; i < 10; i++) { try { assertReply(rpc.sendRequest(r), client, callId); diff --git a/ratis-server/src/test/java/org/apache/ratis/server/impl/LeaderElectionTests.java b/ratis-server/src/test/java/org/apache/ratis/server/impl/LeaderElectionTests.java index 8a17fd135b..d4023a23d2 100644 --- a/ratis-server/src/test/java/org/apache/ratis/server/impl/LeaderElectionTests.java +++ b/ratis-server/src/test/java/org/apache/ratis/server/impl/LeaderElectionTests.java @@ -53,7 +53,6 @@ import java.io.IOException; import java.util.ArrayList; -import java.util.Arrays; import java.util.Collection; import java.util.Iterator; import java.util.List; @@ -65,6 +64,7 @@ import static org.apache.ratis.RaftTestUtil.getPeersWithPriority; import static org.apache.ratis.RaftTestUtil.waitForLeader; +import static org.apache.ratis.proto.RaftProtos.RaftPeerRole.LISTENER; import static org.apache.ratis.server.metrics.LeaderElectionMetrics.LAST_LEADER_ELECTION_ELAPSED_TIME; import static org.apache.ratis.server.metrics.LeaderElectionMetrics.LEADER_ELECTION_COUNT_METRIC; import static org.apache.ratis.server.metrics.LeaderElectionMetrics.LEADER_ELECTION_TIME_TAKEN; @@ -156,11 +156,11 @@ public void testAddServerForWaitReady() throws IOException, InterruptedException } // add 3 new servers and wait longer time CodeInjectionForTesting.put(RaftServerImpl.START_COMPLETE, new SleepCode(2000)); - MiniRaftCluster.PeerChanges peerChanges = cluster.addNewPeers(2, true, false); + final PeerChanges peerChanges = cluster.addNewPeers(2, true, false); LOG.info("add new 3 servers"); LOG.info(cluster.printServers()); RaftClientReply reply = client.admin().setConfiguration(SetConfigurationRequest.Arguments.newBuilder() - .setServersInNewConf(peerChanges.newPeers) + .setServersInNewConf(peerChanges.getAddedPeers()) .setMode(SetConfigurationRequest.Mode.ADD).build()); assertTrue(reply.isSuccess()); for (RaftServer server : cluster.getServers()) { @@ -461,14 +461,14 @@ public void testAddListener() throws Exception { client.io().send(new RaftTestUtil.SimpleMessage("message")); List servers = cluster.getPeers(); assertEquals(servers.size(), 3); - MiniRaftCluster.PeerChanges changes = cluster.addNewPeers(1, - true, false, RaftProtos.RaftPeerRole.LISTENER); - RaftClientReply reply = client.admin().setConfiguration(servers, Arrays.asList(changes.newPeers)); + final PeerChanges changes = cluster.addNewPeers(1, true, false, LISTENER); + final List added = changes.getAddedPeers(); + final RaftClientReply reply = client.admin().setConfiguration(servers, added); assertTrue(reply.isSuccess()); Collection listener = leader.getRaftConf().getAllPeers(RaftProtos.RaftPeerRole.LISTENER); assertEquals(1, listener.size()); - assertEquals(changes.newPeers[0].getId(), new ArrayList<>(listener).get(0).getId()); + assertEquals(added.get(0).getId(), listener.iterator().next().getId()); } cluster.shutdown(); } @@ -486,8 +486,8 @@ public void testAddFollowerWhenExistsListener() throws Exception { List listener = new ArrayList<>( leader.getRaftConf().getAllPeers(RaftProtos.RaftPeerRole.LISTENER)); assertEquals(1, listener.size()); - MiniRaftCluster.PeerChanges changes = cluster.addNewPeers(1, true, false); - ArrayList newPeers = new ArrayList<>(Arrays.asList(changes.newPeers)); + final PeerChanges changes = cluster.addNewPeers(1, true, false); + final List newPeers = new ArrayList<>(changes.getAddedPeers()); newPeers.addAll(leader.getRaftConf().getAllPeers(RaftProtos.RaftPeerRole.FOLLOWER)); RaftClientReply reply = client.admin().setConfiguration(newPeers, listener); assertTrue(reply.isSuccess()); @@ -761,7 +761,7 @@ void runTestLeaderLeaseDuringReconfiguration(CLUSTER cluster, long leaseTimeoutM RaftServerTestUtil.assertLeaderLease(leader, true); final List followers = cluster.getFollowers(); - final MiniRaftCluster.PeerChanges changes = cluster.addNewPeers(2, true); + final PeerChanges changes = cluster.addNewPeers(2, true); // blocking the original 2 followers BlockRequestHandlingInjection.getInstance().blockReplier(followers.get(0).getId().toString()); @@ -770,7 +770,7 @@ void runTestLeaderLeaseDuringReconfiguration(CLUSTER cluster, long leaseTimeoutM // start reconfiguration in another thread, shall fail eventually new Thread(() -> { try { - client.admin().setConfiguration(changes.allPeersInNewConf); + client.admin().setConfiguration(changes.getPeersInNewConf()); } catch (IOException e) { System.out.println("as expected: " + e.getMessage()); } diff --git a/ratis-server/src/test/java/org/apache/ratis/server/impl/MiniRaftCluster.java b/ratis-server/src/test/java/org/apache/ratis/server/impl/MiniRaftCluster.java index b18440c835..6dd5a12768 100644 --- a/ratis-server/src/test/java/org/apache/ratis/server/impl/MiniRaftCluster.java +++ b/ratis-server/src/test/java/org/apache/ratis/server/impl/MiniRaftCluster.java @@ -238,18 +238,6 @@ private int getPort(String address) { } } - public static class PeerChanges { - public final RaftPeer[] allPeersInNewConf; - public final RaftPeer[] newPeers; - public final RaftPeer[] removedPeers; - - public PeerChanges(RaftPeer[] all, RaftPeer[] newPeers, RaftPeer[] removed) { - this.allPeersInNewConf = all; - this.newPeers = newPeers; - this.removedPeers = removed; - } - } - public static RaftGroup initRaftGroup(Collection ids, Collection listenerIds) { Stream peer = ids.stream() .map(id -> RaftPeer.newBuilder().setId(id)) @@ -493,12 +481,12 @@ public PeerChanges addNewPeers(String[] ids, boolean startNewPeer, } } - final Collection newPeers = toRaftPeers(newServers); - final RaftPeer[] np = newPeers.toArray(RaftPeer.emptyArray()); - newPeers.addAll(group.getPeers()); - RaftPeer[] p = newPeers.toArray(RaftPeer.emptyArray()); - group = RaftGroup.valueOf(group.getGroupId(), p); - return new PeerChanges(p, np, RaftPeer.emptyArray()); + final List newPeers = toRaftPeers(newServers); + final List allPeers = new ArrayList<>(newPeers.size() + group.getPeers().size()); + allPeers.addAll(newPeers); + allPeers.addAll(group.getPeers()); + group = RaftGroup.valueOf(group.getGroupId(), allPeers); + return new PeerChanges(allPeers, newPeers, Collections.emptyList()); } void startServers(Iterable raftServers) throws IOException { @@ -513,7 +501,7 @@ void startServers(Iterable raftServers) throws IOException */ public PeerChanges removePeers(int number, boolean removeLeader, Collection excluded) throws InterruptedException { - Collection raftPeers = new ArrayList<>(group.getPeers()); + final List raftPeers = new ArrayList<>(group.getPeers()); List removedPeers = new ArrayList<>(number); if (removeLeader) { final RaftPeer leader = RaftTestUtil.waitForLeader(this).getPeer(); @@ -531,9 +519,8 @@ public PeerChanges removePeers(int number, boolean removeLeader, removed++; } } - final RaftPeer[] p = raftPeers.toArray(RaftPeer.emptyArray()); - group = RaftGroup.valueOf(group.getGroupId(), p); - return new PeerChanges(p, RaftPeer.emptyArray(), removedPeers.toArray(RaftPeer.emptyArray())); + group = RaftGroup.valueOf(group.getGroupId(), raftPeers); + return new PeerChanges(raftPeers, Collections.emptyList(), removedPeers); } public void killServer(RaftPeerId id) { @@ -815,15 +802,14 @@ public RaftClientRequest newRaftClientRequest( } public SetConfigurationRequest newSetConfigurationRequest( - ClientId clientId, RaftPeerId leaderId, - RaftPeer... raftPeers) { + ClientId clientId, RaftPeerId leaderId, List raftPeers) { return new SetConfigurationRequest(clientId, leaderId, getGroupId(), CallId.getDefault(), SetConfigurationRequest.Arguments.newBuilder().setServersInNewConf(raftPeers).build()); } - public void setConfiguration(RaftPeer... raftPeers) throws IOException { + public void setConfiguration(List raftPeers) throws IOException { try(RaftClient client = createClient()) { - LOG.info("Start changing the configuration: {}", Arrays.asList(raftPeers)); + LOG.info("Start changing the configuration: {}", raftPeers); final RaftClientReply reply = client.admin().setConfiguration(raftPeers); Preconditions.assertTrue(reply.isSuccess()); } diff --git a/ratis-server/src/test/java/org/apache/ratis/server/impl/PeerChanges.java b/ratis-server/src/test/java/org/apache/ratis/server/impl/PeerChanges.java new file mode 100644 index 0000000000..edb8e1acca --- /dev/null +++ b/ratis-server/src/test/java/org/apache/ratis/server/impl/PeerChanges.java @@ -0,0 +1,71 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.ratis.server.impl; + +import org.apache.ratis.protocol.RaftPeer; + +import java.util.Collections; +import java.util.List; +import java.util.Objects; + +public class PeerChanges { + private final List peersInNewConf; + private final List addedPeers; + private final List removedPeers; + + PeerChanges(List all, List addedPeers, List removed) { + this.peersInNewConf = Collections.unmodifiableList(all); + this.addedPeers = Collections.unmodifiableList(addedPeers); + this.removedPeers = Collections.unmodifiableList(removed); + } + + public List getPeersInNewConf() { + return peersInNewConf; + } + + public List getAddedPeers() { + return addedPeers; + } + + @Override + public boolean equals(Object obj) { + if (obj == this) { + return true; + } else if (!(obj instanceof PeerChanges)) { + return false; + } + final PeerChanges that = (PeerChanges) obj; + return Objects.equals(this.peersInNewConf, that.peersInNewConf) + && Objects.equals(this.addedPeers, that.addedPeers) + && Objects.equals(this.removedPeers, that.removedPeers); + } + + @Override + public int hashCode() { + return Objects.hashCode(peersInNewConf); + } + + @Override + public String toString() { + return "PeerChanges{" + + "\n newConf=" + peersInNewConf + + "\n added=" + addedPeers + + "\n removed=" + removedPeers + + "\n}"; + } +} diff --git a/ratis-server/src/test/java/org/apache/ratis/server/impl/RaftReconfigurationBaseTest.java b/ratis-server/src/test/java/org/apache/ratis/server/impl/RaftReconfigurationBaseTest.java index 68ad4ebd24..036718ea3b 100644 --- a/ratis-server/src/test/java/org/apache/ratis/server/impl/RaftReconfigurationBaseTest.java +++ b/ratis-server/src/test/java/org/apache/ratis/server/impl/RaftReconfigurationBaseTest.java @@ -37,7 +37,6 @@ import org.apache.ratis.server.RaftConfiguration; import org.apache.ratis.server.RaftServer; import org.apache.ratis.server.RaftServerConfigKeys; -import org.apache.ratis.server.impl.MiniRaftCluster.PeerChanges; import org.apache.ratis.server.raftlog.LogProtoUtils; import org.apache.ratis.server.raftlog.RaftLog; import org.apache.ratis.server.raftlog.RaftLogBase; @@ -64,7 +63,6 @@ import java.util.concurrent.atomic.AtomicBoolean; import java.util.concurrent.atomic.AtomicReference; -import static java.util.Arrays.asList; import static org.apache.ratis.server.impl.RaftServerTestUtil.waitAndCheckNewConf; import static org.junit.jupiter.api.Assertions.assertThrows; @@ -141,7 +139,7 @@ public void testAddPeers() throws Exception { RaftTestUtil.waitForLeader(cluster); // add new peers - RaftPeer[] allPeers = cluster.addNewPeers(2, true).allPeersInNewConf; + final List allPeers = cluster.addNewPeers(2, true).getPeersInNewConf(); // trigger setConfiguration cluster.setConfiguration(allPeers); @@ -161,13 +159,13 @@ public void testLeaderElectionWhenChangeFromSingleToHA() throws Exception { RaftGroupId groupId = cluster.getGroup().getGroupId(); RaftPeer curPeer = cluster.getGroup().getPeers().iterator().next(); - RaftPeer newPeer = cluster.addNewPeers(1, true, true).newPeers[0]; + RaftPeer newPeer = cluster.addNewPeers(1, true, true).getAddedPeers().get(0); RaftServerProxy leaderServer = cluster.getServer(curPeer.getId()); // Update leader conf to transitional single mode. RaftConfigurationImpl oldNewConf = RaftConfigurationImpl.newBuilder() - .setOldConf(new PeerConfiguration(Arrays.asList(curPeer))) + .setOldConf(new PeerConfiguration(Collections.singletonList(curPeer))) .setConf(new PeerConfiguration(Arrays.asList(curPeer, newPeer))) .setLogEntryIndex(Long.MAX_VALUE / 2) .build(); @@ -194,7 +192,7 @@ public void testChangeMajority() throws Exception { SetConfigurationRequest.Arguments arguments = SetConfigurationRequest.Arguments.newBuilder() .setServersInCurrentConf(cluster.getPeers()) - .setServersInNewConf(c1.allPeersInNewConf) + .setServersInNewConf(c1.getPeersInNewConf()) .setMode(SetConfigurationRequest.Mode.COMPARE_AND_SET) .build(); assertThrows(SetConfigurationException.class, () -> client.admin().setConfiguration(arguments), @@ -212,8 +210,8 @@ public void testRemovePeers() throws Exception { RaftTestUtil.waitForLeader(cluster); // remove peers, leader still included in the new conf - RaftPeer[] allPeers = cluster - .removePeers(2, false, Collections.emptyList()).allPeersInNewConf; + final List allPeers = cluster.removePeers(2, false, Collections.emptyList()) + .getPeersInNewConf(); // trigger setConfiguration cluster.setConfiguration(allPeers); @@ -240,8 +238,8 @@ private void runTestAddRemovePeers(boolean leaderStepdown, CLUSTER cluster) thro RaftTestUtil.waitForLeader(cluster); PeerChanges change = cluster.addNewPeers(2, true); - RaftPeer[] allPeers = cluster.removePeers(2, leaderStepdown, - asList(change.newPeers)).allPeersInNewConf; + final List allPeers = cluster.removePeers(2, leaderStepdown, change.getAddedPeers()) + .getPeersInNewConf(); // trigger setConfiguration cluster.setConfiguration(allPeers); @@ -259,7 +257,7 @@ private void runTestSetConfigurationInAddMode(CLUSTER cluster) throws Exception final RaftServer.Division leader = RaftTestUtil.waitForLeader(cluster); PeerChanges change = cluster.addNewPeers(1, true); - List peers = Arrays.asList(change.newPeers); + List peers = change.getAddedPeers(); try (final RaftClient client = cluster.createClient(leader.getId())) { for (int i = 0; i < 10; i++) { @@ -271,7 +269,7 @@ private void runTestSetConfigurationInAddMode(CLUSTER cluster) throws Exception .setServersInNewConf(peers) .setMode(SetConfigurationRequest.Mode.ADD).build()); Assertions.assertTrue(reply.isSuccess()); - waitAndCheckNewConf(cluster, change.allPeersInNewConf, 0, null); + waitAndCheckNewConf(cluster, change.getPeersInNewConf(), 0, null); } cluster.close(); } @@ -286,7 +284,7 @@ private void runTestSetConfigurationInCasMode(CLUSTER cluster) throws Exception List oldPeers = cluster.getPeers(); PeerChanges change = cluster.addNewPeers(1, true); - List peers = Arrays.asList(change.allPeersInNewConf); + List peers = change.getPeersInNewConf(); try (final RaftClient client = cluster.createClient(leader.getId())) { for (int i = 0; i < 10; i++) { @@ -309,7 +307,7 @@ private void runTestSetConfigurationInCasMode(CLUSTER cluster) throws Exception .setMode(SetConfigurationRequest.Mode.COMPARE_AND_SET) .build()); Assertions.assertTrue(reply.isSuccess()); - waitAndCheckNewConf(cluster, change.allPeersInNewConf, 0, null); + waitAndCheckNewConf(cluster, change.getPeersInNewConf(), 0, null); } cluster.close(); } @@ -334,25 +332,21 @@ void runTestReconfTwice(CLUSTER cluster) throws Exception { final AtomicBoolean reconf1 = new AtomicBoolean(false); final AtomicBoolean reconf2 = new AtomicBoolean(false); - final AtomicReference finalPeers = new AtomicReference<>(null); - final AtomicReference deadPeers = new AtomicReference<>(null); + final AtomicReference> finalPeers = new AtomicReference<>(null); CountDownLatch latch = new CountDownLatch(1); Thread clientThread = new Thread(() -> { try { PeerChanges c1 = cluster.addNewPeers(2, true, true); - LOG.info("Start changing the configuration: {}", - asList(c1.allPeersInNewConf)); + LOG.info("Start changing the configuration: {}", c1.getPeersInNewConf()); - RaftClientReply reply = client.admin().setConfiguration(c1.allPeersInNewConf); + RaftClientReply reply = client.admin().setConfiguration(c1.getPeersInNewConf()); reconf1.set(reply.isSuccess()); - PeerChanges c2 = cluster.removePeers(2, true, asList(c1.newPeers)); - finalPeers.set(c2.allPeersInNewConf); - deadPeers.set(c2.removedPeers); + PeerChanges c2 = cluster.removePeers(2, true, c1.getAddedPeers()); + finalPeers.set(c2.getPeersInNewConf()); - LOG.info("Start changing the configuration again: {}", - asList(c2.allPeersInNewConf)); - reply = client.admin().setConfiguration(c2.allPeersInNewConf); + LOG.info("Start changing the configuration again: {}", c2.getPeersInNewConf()); + reply = client.admin().setConfiguration(c2.getPeersInNewConf()); reconf2.set(reply.isSuccess()); latch.countDown(); @@ -393,13 +387,12 @@ void runTestReconfTimeout(CLUSTER cluster) throws Exception { try (final RaftClient client = cluster.createClient(leaderId)) { PeerChanges c1 = cluster.addNewPeers(2, false); - LOG.info("Start changing the configuration: {}", - asList(c1.allPeersInNewConf)); + LOG.info("Start changing the configuration: {}", c1.getPeersInNewConf()); Assertions.assertFalse(((RaftConfigurationImpl)cluster.getLeader().getRaftConf()).isTransitional()); final RaftClientRpc sender = client.getClientRpc(); final SetConfigurationRequest request = cluster.newSetConfigurationRequest( - client.getId(), leaderId, c1.allPeersInNewConf); + client.getId(), leaderId, c1.getPeersInNewConf()); try { RaftClientReply reply = sender.sendRequest(request); Assertions.fail("did not get expected exception " + reply.toString()); @@ -423,10 +416,10 @@ void runTestReconfTimeout(CLUSTER cluster) throws Exception { // start the two new peers LOG.info("Start new peers"); - for (RaftPeer np : c1.newPeers) { + for (RaftPeer np : c1.getAddedPeers()) { cluster.restartServer(np.getId(), false); } - Assertions.assertTrue(client.admin().setConfiguration(c1.allPeersInNewConf).isSuccess()); + Assertions.assertTrue(client.admin().setConfiguration(c1.getPeersInNewConf()).isSuccess()); } } @@ -446,7 +439,7 @@ public void testBootstrapReconfWithSingleNodeAddTwo() throws Exception { try (final RaftClient client = cluster.createClient(leaderId)) { final PeerChanges c1 = cluster.addNewPeers(2, true); - assertThrows(SetConfigurationException.class, () -> client.admin().setConfiguration(c1.allPeersInNewConf), + assertThrows(SetConfigurationException.class, () -> client.admin().setConfiguration(c1.getPeersInNewConf()), "Expect change majority error."); } }); @@ -472,12 +465,11 @@ void runTestBootstrapReconf(int numNewPeer, boolean startNewPeer, CLUSTER cluste } final PeerChanges c1 = cluster.addNewPeers(numNewPeer, startNewPeer); - LOG.info("Start changing the configuration: {}", - asList(c1.allPeersInNewConf)); + LOG.info("Start changing the configuration: {}", c1.getPeersInNewConf()); final AtomicReference success = new AtomicReference<>(); Thread clientThread = new Thread(() -> { try { - RaftClientReply reply = client.admin().setConfiguration(c1.allPeersInNewConf); + RaftClientReply reply = client.admin().setConfiguration(c1.getPeersInNewConf()); success.set(reply.isSuccess()); } catch (IOException ioe) { LOG.error("FAILED", ioe); @@ -489,8 +481,8 @@ void runTestBootstrapReconf(int numNewPeer, boolean startNewPeer, CLUSTER cluste // Make sure that set configuration is run inside the thread RaftTestUtil.waitFor(() -> clientThread.isAlive(), 300, 5000); ONE_SECOND.sleep(); - LOG.info("start new peer(s): {}", c1.newPeers); - for(RaftPeer p : c1.newPeers) { + LOG.info("start new peer(s): {}", c1.getAddedPeers()); + for(RaftPeer p : c1.getAddedPeers()) { cluster.restartServer(p.getId(), false); } } @@ -500,7 +492,7 @@ void runTestBootstrapReconf(int numNewPeer, boolean startNewPeer, CLUSTER cluste RaftTestUtil.waitFor(() -> cluster.getLeader() != null, 300, 5000); final RaftLog leaderLog = cluster.getLeader().getRaftLog(); - for (RaftPeer newPeer : c1.newPeers) { + for (RaftPeer newPeer : c1.getAddedPeers()) { final RaftServer.Division d = cluster.getDivision(newPeer.getId()); RaftTestUtil.waitFor(() -> leaderLog.getEntries(0, Long.MAX_VALUE).length == d.getRaftLog().getEntries(0, Long.MAX_VALUE).length, 300, 15000); @@ -528,16 +520,16 @@ void runTestKillLeaderDuringReconf(CLUSTER cluster) throws Exception { final RaftPeerId leaderId = RaftTestUtil.waitForLeader(cluster).getId(); PeerChanges c1 = cluster.addNewPeers(1, false); - PeerChanges c2 = cluster.removePeers(1, false, asList(c1.newPeers)); + PeerChanges c2 = cluster.removePeers(1, false, c1.getAddedPeers()); - LOG.info("Start setConf: {}", asList(c2.allPeersInNewConf)); + LOG.info("Start setConf: {}", c2.getPeersInNewConf()); LOG.info(cluster.printServers()); final CompletableFuture setConf = new CompletableFuture<>(); clientThread = new Thread(() -> { try(final RaftClient client = cluster.createClient(leaderId)) { for(int i = 0; clientRunning.get() && !setConf.isDone(); i++) { - final RaftClientReply reply = client.admin().setConfiguration(c2.allPeersInNewConf); + final RaftClientReply reply = client.admin().setConfiguration(c2.getPeersInNewConf()); if (reply.isSuccess()) { setConf.complete(null); return; @@ -572,8 +564,8 @@ void runTestKillLeaderDuringReconf(CLUSTER cluster) throws Exception { LOG.info("newLeaderId: {}", newLeaderId); TimeDuration.valueOf(1500, TimeUnit.MILLISECONDS).sleep(); - LOG.info("start new peers: {}", Arrays.asList(c1.newPeers)); - for (RaftPeer np : c1.newPeers) { + LOG.info("start new peers: {}", c1.getAddedPeers()); + for (RaftPeer np : c1.getAddedPeers()) { cluster.restartServer(np.getId(), false); } @@ -582,12 +574,12 @@ void runTestKillLeaderDuringReconf(CLUSTER cluster) throws Exception { } catch(TimeoutException ignored) { } - RaftServerProxy newServer = cluster.getServer(c1.newPeers[0].getId()); + RaftServerProxy newServer = cluster.getServer(c1.getAddedPeers().get(0).getId()); if (newServer.getLifeCycleState() == LifeCycle.State.CLOSED) { - LOG.info("New peer {} is shutdown. Skip the check", c1.newPeers[0].getId()); + LOG.info("New peer {} is shutdown. Skip the check", c1.getAddedPeers().get(0).getId()); } else { // the client fails with the first leader, and then retry the same setConfiguration request - waitAndCheckNewConf(cluster, c2.allPeersInNewConf, 1, Collections.singletonList(leaderId)); + waitAndCheckNewConf(cluster, c2.getPeersInNewConf(), 1, Collections.singletonList(leaderId)); setConf.get(2, TimeUnit.SECONDS); } } finally { @@ -657,14 +649,14 @@ void runTestOverlappedSetConfRequests(CLUSTER cluster) throws Exception { final RaftPeerId leaderId = cluster.getLeader().getId(); - RaftPeer[] newPeers = cluster.addNewPeers(2, true).allPeersInNewConf; + final List newPeers = cluster.addNewPeers(2, true).getPeersInNewConf(); // delay every peer's logSync so that the setConf request is delayed cluster.getPeers() .forEach(peer -> LOG_SYNC_DELAY.setDelayMs(peer.getId().toString(), 1000)); final CountDownLatch latch = new CountDownLatch(1); - final RaftPeer[] peersInRequest2 = cluster.getPeers().toArray(new RaftPeer[0]); + final List peersInRequest2 = cluster.getPeers(); AtomicBoolean caughtException = new AtomicBoolean(false); new Thread(() -> { try(final RaftClient client2 = cluster.createClient(leaderId)) { @@ -730,7 +722,7 @@ void runTestRevertConfigurationChange(CLUSTER cluster) throws Exception { BlockRequestHandlingInjection.getInstance().blockReplier(leaderId.toString()); cluster.setBlockRequestsFrom(leaderId.toString(), true); - PeerChanges change = cluster.removePeers(1, false, new ArrayList<>()); + PeerChanges change = cluster.removePeers(1, false, Collections.emptyList()); AtomicBoolean gotNotLeader = new AtomicBoolean(false); final Thread clientThread = new Thread(() -> { @@ -738,7 +730,7 @@ void runTestRevertConfigurationChange(CLUSTER cluster) throws Exception { LOG.info("client starts to change conf"); final RaftClientRpc sender = client.getClientRpc(); RaftClientReply reply = sender.sendRequest(cluster.newSetConfigurationRequest( - client.getId(), leaderId, change.allPeersInNewConf)); + client.getId(), leaderId, change.getPeersInNewConf())); if (reply.getNotLeaderException() != null) { gotNotLeader.set(true); } diff --git a/ratis-server/src/test/java/org/apache/ratis/server/impl/RaftServerTestUtil.java b/ratis-server/src/test/java/org/apache/ratis/server/impl/RaftServerTestUtil.java index db22c4854a..c225730d37 100644 --- a/ratis-server/src/test/java/org/apache/ratis/server/impl/RaftServerTestUtil.java +++ b/ratis-server/src/test/java/org/apache/ratis/server/impl/RaftServerTestUtil.java @@ -47,7 +47,7 @@ import org.slf4j.event.Level; import java.io.IOException; -import java.util.Arrays; +import java.util.ArrayList; import java.util.Collection; import java.util.Collections; import java.util.HashSet; @@ -82,18 +82,18 @@ public static void setPendingRequestsLogLevel(Level level) { } public static void waitAndCheckNewConf(MiniRaftCluster cluster, - RaftPeer[] peers, int numOfNewPeers, int numOfRemovedPeers, Collection deadPeers) + List peers, int numOfNewPeers, int numOfRemovedPeers, Collection deadPeers) throws Exception { final TimeDuration sleepTime = cluster.getTimeoutMax().apply(n -> n * (numOfRemovedPeers + numOfNewPeers + 2)); - JavaUtils.attempt(() -> waitAndCheckNewConf(cluster, Arrays.asList(peers), deadPeers), + JavaUtils.attempt(() -> waitAndCheckNewConf(cluster, peers, deadPeers), 10, sleepTime, "waitAndCheckNewConf", LOG); } public static void waitAndCheckNewConf(MiniRaftCluster cluster, - RaftPeer[] peers, int numOfRemovedPeers, Collection deadPeers) + List peers, int numOfRemovedPeers, Collection deadPeers) throws Exception { final TimeDuration sleepTime = cluster.getTimeoutMax().apply(n -> n * (numOfRemovedPeers + 2)); - JavaUtils.attempt(() -> waitAndCheckNewConf(cluster, Arrays.asList(peers), deadPeers), + JavaUtils.attempt(() -> waitAndCheckNewConf(cluster, peers, deadPeers), 10, sleepTime, "waitAndCheckNewConf", LOG); } private static void waitAndCheckNewConf(MiniRaftCluster cluster, @@ -224,16 +224,16 @@ public static boolean isHighestPriority(RaftConfiguration config, RaftPeerId pee return ((RaftConfigurationImpl)config).isHighestPriority(peerId); } - public static void runWithMinorityPeers(MiniRaftCluster cluster, Collection peersInNewConf, - CheckedConsumer, IOException> consumer) throws IOException { - Collection peers = parseMinorityPeers(cluster, peersInNewConf); + public static void runWithMinorityPeers(MiniRaftCluster cluster, List peersInNewConf, + CheckedConsumer, IOException> consumer) throws IOException { + List peers = parseMinorityPeers(cluster, peersInNewConf); while (peers != null) { consumer.accept(peers); peers = parseMinorityPeers(cluster, peersInNewConf); } } - private static Collection parseMinorityPeers(MiniRaftCluster cluster, Collection peersInNewConf) { + private static List parseMinorityPeers(MiniRaftCluster cluster, List peersInNewConf) { RaftConfigurationImpl conf = (RaftConfigurationImpl) cluster.getLeader().getRaftConf(); Set peers = new HashSet<>(conf.getCurrentPeers()); @@ -247,7 +247,7 @@ private static Collection parseMinorityPeers(MiniRaftCluster cluster, break; } } - return peers; + return new ArrayList<>(peers); } // All new peers has been added. Handle the removed peers. diff --git a/ratis-server/src/test/java/org/apache/ratis/statemachine/RaftSnapshotBaseTest.java b/ratis-server/src/test/java/org/apache/ratis/statemachine/RaftSnapshotBaseTest.java index cd1a2eb55d..91f974dbfa 100644 --- a/ratis-server/src/test/java/org/apache/ratis/statemachine/RaftSnapshotBaseTest.java +++ b/ratis-server/src/test/java/org/apache/ratis/statemachine/RaftSnapshotBaseTest.java @@ -38,6 +38,7 @@ import org.apache.ratis.protocol.RaftPeerId; import org.apache.ratis.server.RaftServer; import org.apache.ratis.server.RaftServerConfigKeys; +import org.apache.ratis.server.impl.PeerChanges; import org.apache.ratis.server.impl.RaftServerTestUtil; import org.apache.ratis.server.metrics.RaftServerMetricsImpl; import org.apache.ratis.server.raftlog.RaftLog; @@ -55,7 +56,6 @@ import org.slf4j.LoggerFactory; import java.io.File; -import java.util.Arrays; import java.util.List; import java.util.Optional; import java.util.stream.Collectors; @@ -227,12 +227,11 @@ void runTestBasicInstallSnapshot(CLUSTER cluster) throws Exception { } // add a new peer - final MiniRaftCluster.PeerChanges change = cluster.addNewPeers(1, true, true); + final PeerChanges change = cluster.addNewPeers(1, true, true); // trigger setConfiguration - RaftServerTestUtil.runWithMinorityPeers(cluster, Arrays.asList(change.allPeersInNewConf), - peers -> cluster.setConfiguration(peers.toArray(RaftPeer.emptyArray()))); + RaftServerTestUtil.runWithMinorityPeers(cluster, change.getPeersInNewConf(), cluster::setConfiguration); - for (RaftPeer newPeer : change.newPeers) { + for (RaftPeer newPeer : change.getAddedPeers()) { final RaftServer.Division s = cluster.getDivision(newPeer.getId()); SimpleStateMachine4Testing simpleStateMachine = SimpleStateMachine4Testing.get(s); Assertions.assertSame(LifeCycle.State.RUNNING, simpleStateMachine.getLifeCycleState()); @@ -240,7 +239,7 @@ void runTestBasicInstallSnapshot(CLUSTER cluster) throws Exception { // Verify installSnapshot counter on leader before restart. verifyInstallSnapshotMetric(cluster.getLeader()); - RaftServerTestUtil.waitAndCheckNewConf(cluster, change.allPeersInNewConf, 0, null); + RaftServerTestUtil.waitAndCheckNewConf(cluster, change.getPeersInNewConf(), 0, null); Timer timer = getTakeSnapshotTimer(cluster.getLeader()); long count = timer.getCount(); @@ -293,12 +292,11 @@ void runTestInstallSnapshotDuringBootstrap(CLUSTER cluster) throws Exception { assertLeaderContent(cluster); // add a new peer - final MiniRaftCluster.PeerChanges change = cluster.addNewPeers(1, true, true); + final PeerChanges change = cluster.addNewPeers(1, true, true); // trigger setConfiguration - RaftServerTestUtil.runWithMinorityPeers(cluster, Arrays.asList(change.allPeersInNewConf), - peers -> cluster.setConfiguration(peers.toArray(RaftPeer.emptyArray()))); + RaftServerTestUtil.runWithMinorityPeers(cluster, change.getPeersInNewConf(), cluster::setConfiguration); - for (RaftPeer newPeer : change.newPeers) { + for (RaftPeer newPeer : change.getAddedPeers()) { final RaftServer.Division s = cluster.getDivision(newPeer.getId()); SimpleStateMachine4Testing simpleStateMachine = SimpleStateMachine4Testing.get(s); Assertions.assertSame(LifeCycle.State.RUNNING, simpleStateMachine.getLifeCycleState()); @@ -306,7 +304,7 @@ void runTestInstallSnapshotDuringBootstrap(CLUSTER cluster) throws Exception { // Verify installSnapshot counter on leader verifyInstallSnapshotMetric(cluster.getLeader()); - RaftServerTestUtil.waitAndCheckNewConf(cluster, change.allPeersInNewConf, 0, null); + RaftServerTestUtil.waitAndCheckNewConf(cluster, change.getPeersInNewConf(), 0, null); } finally { cluster.shutdown(); } diff --git a/ratis-test/src/test/java/org/apache/ratis/shell/cli/sh/PeerCommandIntegrationTest.java b/ratis-test/src/test/java/org/apache/ratis/shell/cli/sh/PeerCommandIntegrationTest.java index b3c35147a9..f12fc7d4e5 100644 --- a/ratis-test/src/test/java/org/apache/ratis/shell/cli/sh/PeerCommandIntegrationTest.java +++ b/ratis-test/src/test/java/org/apache/ratis/shell/cli/sh/PeerCommandIntegrationTest.java @@ -28,7 +28,6 @@ import org.apache.ratis.server.raftlog.RaftLog; import org.apache.ratis.statemachine.impl.SimpleStateMachine4Testing; import org.apache.ratis.statemachine.StateMachine; -import org.apache.ratis.thirdparty.com.google.common.collect.ObjectArrays; import org.apache.ratis.util.JavaUtils; import org.apache.ratis.util.Slf4jUtils; import org.apache.ratis.util.SizeInBytes; @@ -37,6 +36,8 @@ import org.junit.jupiter.api.Test; import org.slf4j.event.Level; +import java.util.Arrays; +import java.util.List; import java.util.concurrent.TimeUnit; public abstract class PeerCommandIntegrationTest @@ -65,7 +66,7 @@ void runTestPeerRemoveCommand(MiniRaftCluster cluster) throws Exception { final RaftServer.Division leader = RaftTestUtil.waitForLeader(cluster); final String address = getClusterAddress(cluster); RaftServer.Division toRemove = cluster.getFollowers().get(0); - RaftPeer[] peers = new RaftPeer[]{cluster.getFollowers().get(1).getPeer(), leader.getPeer()}; + final List peers = Arrays.asList(cluster.getFollowers().get(1).getPeer(), leader.getPeer()); final StringPrintStream out = new StringPrintStream(); RatisShell shell = new RatisShell(out.getPrintStream()); Assertions.assertTrue(cluster.getFollowers().contains(toRemove)); @@ -80,8 +81,8 @@ void runTestPeerAddCommand(MiniRaftCluster cluster) throws Exception { LOG.info("Start testMultiGroup" + cluster.printServers()); RaftTestUtil.waitForLeader(cluster); - RaftPeer[] peers = cluster.getPeers().toArray(new RaftPeer[0]); - RaftPeer[] newPeers = cluster.addNewPeers(1, true, true).newPeers; + final List peers = cluster.getPeers(); + final List newPeers = cluster.addNewPeers(1, true, true).getAddedPeers(); RaftServerTestUtil.waitAndCheckNewConf(cluster, peers, 0, null); StringBuilder sb = new StringBuilder(); @@ -92,11 +93,13 @@ void runTestPeerAddCommand(MiniRaftCluster cluster) throws Exception { final StringPrintStream out = new StringPrintStream(); RatisShell shell = new RatisShell(out.getPrintStream()); + final RaftPeer newPeer0 = newPeers.get(0); int ret = shell.run("peer", "add", "-peers", sb.toString(), "-address", - newPeers[0].getAdminAddress(), "-peerId", newPeers[0].getId().toString()); + newPeer0.getAdminAddress(), "-peerId", newPeer0.getId().toString()); Assertions.assertEquals(0, ret); - RaftServerTestUtil.waitAndCheckNewConf(cluster, ObjectArrays.concat(peers, newPeers[0]), 0, null); + peers.add(newPeer0); + RaftServerTestUtil.waitAndCheckNewConf(cluster, peers, 0, null); } From e7d867706f6aa066a4ce10b3b157f5a3c5f20519 Mon Sep 17 00:00:00 2001 From: Tsz-Wo Nicholas Sze Date: Sat, 27 Sep 2025 11:00:09 -0700 Subject: [PATCH 205/397] RATIS-2331. Reuse SslContext in gRPC. (#1288) --- .../java/org/apache/ratis/util/LifeCycle.java | 3 + .../org/apache/ratis/grpc/GrpcFactory.java | 78 +++++++------ .../java/org/apache/ratis/grpc/GrpcUtil.java | 40 +++++++ .../grpc/client/GrpcClientProtocolClient.java | 28 ++--- .../grpc/client/GrpcClientProtocolProxy.java | 108 ------------------ .../ratis/grpc/client/GrpcClientRpc.java | 6 +- .../grpc/server/GrpcServerProtocolClient.java | 36 ++---- .../ratis/grpc/server/GrpcServicesImpl.java | 56 ++++----- .../ratis/grpc/server/GrpcStubPool.java | 29 ++--- 9 files changed, 134 insertions(+), 250 deletions(-) delete mode 100644 ratis-grpc/src/main/java/org/apache/ratis/grpc/client/GrpcClientProtocolProxy.java diff --git a/ratis-common/src/main/java/org/apache/ratis/util/LifeCycle.java b/ratis-common/src/main/java/org/apache/ratis/util/LifeCycle.java index 9870fe3719..e96ba88a5c 100644 --- a/ratis-common/src/main/java/org/apache/ratis/util/LifeCycle.java +++ b/ratis-common/src/main/java/org/apache/ratis/util/LifeCycle.java @@ -117,6 +117,9 @@ static void validate(Object name, State from, State to) { if (LOG.isTraceEnabled()) { LOG.trace("TRACE", new Throwable()); } + if (to == EXCEPTION) { + LOG.error("{} has failed ({} -> {})", name, from, to, new Throwable("TRACE")); + } Preconditions.assertTrue(isValid(from, to), "ILLEGAL TRANSITION: In %s, %s -> %s", name, from, to); diff --git a/ratis-grpc/src/main/java/org/apache/ratis/grpc/GrpcFactory.java b/ratis-grpc/src/main/java/org/apache/ratis/grpc/GrpcFactory.java index 331d1a8585..1053cab80e 100644 --- a/ratis-grpc/src/main/java/org/apache/ratis/grpc/GrpcFactory.java +++ b/ratis-grpc/src/main/java/org/apache/ratis/grpc/GrpcFactory.java @@ -32,11 +32,15 @@ import org.apache.ratis.server.leader.FollowerInfo; import org.apache.ratis.server.leader.LeaderState; import org.apache.ratis.thirdparty.io.netty.buffer.PooledByteBufAllocator; +import org.apache.ratis.thirdparty.io.netty.handler.ssl.SslContext; import org.apache.ratis.util.JavaUtils; +import org.apache.ratis.util.MemoizedSupplier; import org.slf4j.Logger; import org.slf4j.LoggerFactory; +import java.util.function.BiFunction; import java.util.function.Consumer; +import java.util.function.Supplier; public class GrpcFactory implements ServerFactory, ClientFactory { @@ -65,19 +69,32 @@ static boolean checkPooledByteBufAllocatorUseCacheForAllThreads(Consumer return value; } - private final GrpcServices.Customizer servicesCustomizer; + static final BiFunction BUILD_SSL_CONTEXT_FOR_SERVER + = (tlsConf, defaultContext) -> tlsConf == null ? defaultContext : GrpcUtil.buildSslContextForServer(tlsConf); + + static final BiFunction BUILD_SSL_CONTEXT_FOR_CLIENT + = (tlsConf, defaultContext) -> tlsConf == null ? defaultContext : GrpcUtil.buildSslContextForClient(tlsConf); - private final GrpcTlsConfig tlsConfig; - private final GrpcTlsConfig adminTlsConfig; - private final GrpcTlsConfig clientTlsConfig; - private final GrpcTlsConfig serverTlsConfig; + static final class SslContexts { + private final SslContext adminSslContext; + private final SslContext clientSslContext; + private final SslContext serverSslContext; - public static Parameters newRaftParameters(GrpcTlsConfig conf) { - final Parameters p = new Parameters(); - GrpcConfigKeys.TLS.setConf(p, conf); - return p; + private SslContexts(GrpcTlsConfig tlsConfig, GrpcTlsConfig adminTlsConfig, + GrpcTlsConfig clientTlsConfig, GrpcTlsConfig serverTlsConfig, + BiFunction buildMethod) { + final SslContext defaultSslContext = buildMethod.apply(tlsConfig, null); + this.adminSslContext = buildMethod.apply(adminTlsConfig, defaultSslContext); + this.clientSslContext = buildMethod.apply(clientTlsConfig, defaultSslContext); + this.serverSslContext = buildMethod.apply(serverTlsConfig, defaultSslContext); + } } + private final GrpcServices.Customizer servicesCustomizer; + + private final Supplier forServerSupplier; + private final Supplier forClientSupplier; + public GrpcFactory(Parameters parameters) { this(GrpcConfigKeys.Server.servicesCustomizer(parameters), GrpcConfigKeys.TLS.conf(parameters), @@ -87,35 +104,15 @@ public GrpcFactory(Parameters parameters) { ); } - public GrpcFactory(GrpcTlsConfig tlsConfig) { - this(null, tlsConfig, null, null, null); - } - private GrpcFactory(GrpcServices.Customizer servicesCustomizer, GrpcTlsConfig tlsConfig, GrpcTlsConfig adminTlsConfig, GrpcTlsConfig clientTlsConfig, GrpcTlsConfig serverTlsConfig) { this.servicesCustomizer = servicesCustomizer; - this.tlsConfig = tlsConfig; - this.adminTlsConfig = adminTlsConfig; - this.clientTlsConfig = clientTlsConfig; - this.serverTlsConfig = serverTlsConfig; - } - - public GrpcTlsConfig getTlsConfig() { - return tlsConfig; - } - - public GrpcTlsConfig getAdminTlsConfig() { - return adminTlsConfig != null ? adminTlsConfig : tlsConfig; - } - - public GrpcTlsConfig getClientTlsConfig() { - return clientTlsConfig != null ? clientTlsConfig : tlsConfig; - } - - public GrpcTlsConfig getServerTlsConfig() { - return serverTlsConfig != null ? serverTlsConfig : tlsConfig; + this.forServerSupplier = MemoizedSupplier.valueOf(() -> new SslContexts( + tlsConfig, adminTlsConfig, clientTlsConfig, serverTlsConfig, BUILD_SSL_CONTEXT_FOR_SERVER)); + this.forClientSupplier = MemoizedSupplier.valueOf(() -> new SslContexts( + tlsConfig, adminTlsConfig, clientTlsConfig, serverTlsConfig, BUILD_SSL_CONTEXT_FOR_CLIENT)); } @Override @@ -131,19 +128,24 @@ public LogAppender newLogAppender(RaftServer.Division server, LeaderState state, @Override public GrpcServices newRaftServerRpc(RaftServer server) { checkPooledByteBufAllocatorUseCacheForAllThreads(LOG::info); + + final SslContexts forServer = forServerSupplier.get(); + final SslContexts forClient = forClientSupplier.get(); return GrpcServicesImpl.newBuilder() .setServer(server) .setCustomizer(servicesCustomizer) - .setAdminTlsConfig(getAdminTlsConfig()) - .setServerTlsConfig(getServerTlsConfig()) - .setClientTlsConfig(getClientTlsConfig()) + .setAdminSslContext(forServer.adminSslContext) + .setServerSslContextForServer(forServer.serverSslContext) + .setServerSslContextForClient(forClient.serverSslContext) + .setClientSslContext(forServer.clientSslContext) .build(); } @Override public GrpcClientRpc newRaftClientRpc(ClientId clientId, RaftProperties properties) { checkPooledByteBufAllocatorUseCacheForAllThreads(LOG::debug); - return new GrpcClientRpc(clientId, properties, - getAdminTlsConfig(), getClientTlsConfig()); + + final SslContexts forClient = forClientSupplier.get(); + return new GrpcClientRpc(clientId, properties, forClient.adminSslContext, forClient.clientSslContext); } } diff --git a/ratis-grpc/src/main/java/org/apache/ratis/grpc/GrpcUtil.java b/ratis-grpc/src/main/java/org/apache/ratis/grpc/GrpcUtil.java index 2f9ee01ecb..8dcfb65443 100644 --- a/ratis-grpc/src/main/java/org/apache/ratis/grpc/GrpcUtil.java +++ b/ratis-grpc/src/main/java/org/apache/ratis/grpc/GrpcUtil.java @@ -28,7 +28,10 @@ import org.apache.ratis.thirdparty.io.grpc.Metadata; import org.apache.ratis.thirdparty.io.grpc.Status; import org.apache.ratis.thirdparty.io.grpc.StatusRuntimeException; +import org.apache.ratis.thirdparty.io.grpc.netty.GrpcSslContexts; import org.apache.ratis.thirdparty.io.grpc.stub.StreamObserver; +import org.apache.ratis.thirdparty.io.netty.handler.ssl.ClientAuth; +import org.apache.ratis.thirdparty.io.netty.handler.ssl.SslContext; import org.apache.ratis.thirdparty.io.netty.handler.ssl.SslContextBuilder; import org.apache.ratis.util.IOUtils; import org.apache.ratis.util.JavaUtils; @@ -39,6 +42,7 @@ import org.slf4j.LoggerFactory; import javax.net.ssl.KeyManager; +import javax.net.ssl.SSLException; import javax.net.ssl.TrustManager; import java.io.IOException; import java.util.concurrent.CompletableFuture; @@ -46,6 +50,8 @@ import java.util.function.Function; import java.util.function.Supplier; +import static org.apache.ratis.thirdparty.io.netty.handler.ssl.SslProvider.OPENSSL; + public interface GrpcUtil { Logger LOG = LoggerFactory.getLogger(GrpcUtil.class); @@ -299,4 +305,38 @@ static void setKeyManager(SslContextBuilder b, KeyManagerConf keyManagerConfig) b.keyManager(privateKey.get(), certificates.get()); } } + + static SslContext buildSslContextForServer(GrpcTlsConfig tlsConf) { + if (tlsConf == null) { + return null; + } + SslContextBuilder b = initSslContextBuilderForServer(tlsConf.getKeyManager()); + if (tlsConf.getMtlsEnabled()) { + b.clientAuth(ClientAuth.REQUIRE); + setTrustManager(b, tlsConf.getTrustManager()); + } + b = GrpcSslContexts.configure(b, OPENSSL); + try { + return b.build(); + } catch (Exception e) { + throw new IllegalArgumentException("Failed to buildSslContextForServer from tlsConfig " + tlsConf, e); + } + } + + static SslContext buildSslContextForClient(GrpcTlsConfig tlsConf) { + if (tlsConf == null) { + return null; + } + + final SslContextBuilder b = GrpcSslContexts.forClient(); + setTrustManager(b, tlsConf.getTrustManager()); + if (tlsConf.getMtlsEnabled()) { + setKeyManager(b, tlsConf.getKeyManager()); + } + try { + return b.build(); + } catch (SSLException e) { + throw new IllegalArgumentException("Failed to buildSslContextForClient from tlsConfig " + tlsConf, e); + } + } } diff --git a/ratis-grpc/src/main/java/org/apache/ratis/grpc/client/GrpcClientProtocolClient.java b/ratis-grpc/src/main/java/org/apache/ratis/grpc/client/GrpcClientProtocolClient.java index 3b9d512683..159919fabc 100644 --- a/ratis-grpc/src/main/java/org/apache/ratis/grpc/client/GrpcClientProtocolClient.java +++ b/ratis-grpc/src/main/java/org/apache/ratis/grpc/client/GrpcClientProtocolClient.java @@ -21,7 +21,6 @@ import org.apache.ratis.client.impl.ClientProtoUtils; import org.apache.ratis.conf.RaftProperties; import org.apache.ratis.grpc.GrpcConfigKeys; -import org.apache.ratis.grpc.GrpcTlsConfig; import org.apache.ratis.grpc.GrpcUtil; import org.apache.ratis.grpc.metrics.intercept.client.MetricClientInterceptor; import org.apache.ratis.proto.RaftProtos.GroupInfoReplyProto; @@ -49,11 +48,10 @@ import org.apache.ratis.protocol.exceptions.TimeoutIOException; import org.apache.ratis.thirdparty.io.grpc.ManagedChannel; import org.apache.ratis.thirdparty.io.grpc.StatusRuntimeException; -import org.apache.ratis.thirdparty.io.grpc.netty.GrpcSslContexts; import org.apache.ratis.thirdparty.io.grpc.netty.NegotiationType; import org.apache.ratis.thirdparty.io.grpc.netty.NettyChannelBuilder; import org.apache.ratis.thirdparty.io.grpc.stub.StreamObserver; -import org.apache.ratis.thirdparty.io.netty.handler.ssl.SslContextBuilder; +import org.apache.ratis.thirdparty.io.netty.handler.ssl.SslContext; import org.apache.ratis.util.CollectionUtils; import org.apache.ratis.util.JavaUtils; import org.apache.ratis.util.SizeInBytes; @@ -97,7 +95,7 @@ public class GrpcClientProtocolClient implements Closeable { private final MetricClientInterceptor metricClientInterceptor; GrpcClientProtocolClient(ClientId id, RaftPeer target, RaftProperties properties, - GrpcTlsConfig adminTlsConfig, GrpcTlsConfig clientTlsConfig) { + SslContext adminSslContext, SslContext clientSslContext) { this.name = JavaUtils.memoize(() -> id + "->" + target.getId()); this.target = target; final SizeInBytes flowControlWindow = GrpcConfigKeys.flowControlWindow(properties, LOG::debug); @@ -110,11 +108,9 @@ public class GrpcClientProtocolClient implements Closeable { .filter(x -> !x.isEmpty()).orElse(target.getAddress()); final boolean separateAdminChannel = !Objects.equals(clientAddress, adminAddress); - clientChannel = buildChannel(clientAddress, clientTlsConfig, - flowControlWindow, maxMessageSize); + clientChannel = buildChannel(clientAddress, clientSslContext, flowControlWindow, maxMessageSize); adminChannel = separateAdminChannel - ? buildChannel(adminAddress, adminTlsConfig, - flowControlWindow, maxMessageSize) + ? buildChannel(adminAddress, adminSslContext, flowControlWindow, maxMessageSize) : clientChannel; asyncStub = RaftClientProtocolServiceGrpc.newStub(clientChannel); @@ -124,26 +120,16 @@ public class GrpcClientProtocolClient implements Closeable { RaftClientConfigKeys.Rpc.watchRequestTimeout(properties); } - private ManagedChannel buildChannel(String address, GrpcTlsConfig tlsConf, + private ManagedChannel buildChannel(String address, SslContext sslContext, SizeInBytes flowControlWindow, SizeInBytes maxMessageSize) { NettyChannelBuilder channelBuilder = NettyChannelBuilder.forTarget(address); // ignore any http proxy for grpc channelBuilder.proxyDetector(uri -> null); - if (tlsConf != null) { + if (sslContext != null) { LOG.debug("Setting TLS for {}", address); - SslContextBuilder sslContextBuilder = GrpcSslContexts.forClient(); - GrpcUtil.setTrustManager(sslContextBuilder, tlsConf.getTrustManager()); - if (tlsConf.getMtlsEnabled()) { - GrpcUtil.setKeyManager(sslContextBuilder, tlsConf.getKeyManager()); - } - try { - channelBuilder.useTransportSecurity().sslContext( - sslContextBuilder.build()); - } catch (Exception ex) { - throw new RuntimeException(ex); - } + channelBuilder.useTransportSecurity().sslContext(sslContext); } else { channelBuilder.negotiationType(NegotiationType.PLAINTEXT); } diff --git a/ratis-grpc/src/main/java/org/apache/ratis/grpc/client/GrpcClientProtocolProxy.java b/ratis-grpc/src/main/java/org/apache/ratis/grpc/client/GrpcClientProtocolProxy.java deleted file mode 100644 index 95119ef7d7..0000000000 --- a/ratis-grpc/src/main/java/org/apache/ratis/grpc/client/GrpcClientProtocolProxy.java +++ /dev/null @@ -1,108 +0,0 @@ -/** - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ -package org.apache.ratis.grpc.client; - -import org.apache.ratis.conf.RaftProperties; -import org.apache.ratis.grpc.GrpcTlsConfig; -import org.apache.ratis.protocol.ClientId; -import org.apache.ratis.thirdparty.io.grpc.stub.StreamObserver; -import org.apache.ratis.proto.RaftProtos.RaftClientReplyProto; -import org.apache.ratis.proto.RaftProtos.RaftClientRequestProto; -import org.apache.ratis.protocol.RaftPeer; - -import java.io.Closeable; -import java.io.IOException; -import java.util.function.Function; - -public class GrpcClientProtocolProxy implements Closeable { - private final GrpcClientProtocolClient proxy; - private final Function responseHandlerCreation; - private RpcSession currentSession; - - public GrpcClientProtocolProxy(ClientId clientId, RaftPeer target, - Function responseHandlerCreation, - RaftProperties properties, GrpcTlsConfig tlsConfig) { - proxy = new GrpcClientProtocolClient(clientId, target, properties, tlsConfig, tlsConfig); - this.responseHandlerCreation = responseHandlerCreation; - } - - @Override - public void close() throws IOException { - closeCurrentSession(); - proxy.close(); - } - - @Override - public String toString() { - return "ProxyTo:" + proxy.getTarget(); - } - - public void closeCurrentSession() { - if (currentSession != null) { - currentSession.close(); - currentSession = null; - } - } - - public void onNext(RaftClientRequestProto request) { - if (currentSession == null) { - currentSession = new RpcSession( - responseHandlerCreation.apply(proxy.getTarget())); - } - currentSession.requestObserver.onNext(request); - } - - public void onError() { - if (currentSession != null) { - currentSession.onError(); - } - } - - public interface CloseableStreamObserver - extends StreamObserver, Closeable { - } - - class RpcSession implements Closeable { - private final StreamObserver requestObserver; - private final CloseableStreamObserver responseHandler; - private boolean hasError = false; - - RpcSession(CloseableStreamObserver responseHandler) { - this.responseHandler = responseHandler; - this.requestObserver = proxy.ordered(responseHandler); - } - - void onError() { - hasError = true; - } - - @Override - public void close() { - if (!hasError) { - try { - requestObserver.onCompleted(); - } catch (Exception ignored) { - } - } - try { - responseHandler.close(); - } catch (IOException ignored) { - } - } - } -} diff --git a/ratis-grpc/src/main/java/org/apache/ratis/grpc/client/GrpcClientRpc.java b/ratis-grpc/src/main/java/org/apache/ratis/grpc/client/GrpcClientRpc.java index b825429ae4..4010ade27b 100644 --- a/ratis-grpc/src/main/java/org/apache/ratis/grpc/client/GrpcClientRpc.java +++ b/ratis-grpc/src/main/java/org/apache/ratis/grpc/client/GrpcClientRpc.java @@ -21,7 +21,6 @@ import org.apache.ratis.client.impl.RaftClientRpcWithProxy; import org.apache.ratis.conf.RaftProperties; import org.apache.ratis.grpc.GrpcConfigKeys; -import org.apache.ratis.grpc.GrpcTlsConfig; import org.apache.ratis.grpc.GrpcUtil; import org.apache.ratis.protocol.*; import org.apache.ratis.protocol.exceptions.AlreadyClosedException; @@ -36,6 +35,7 @@ import org.apache.ratis.proto.RaftProtos.TransferLeadershipRequestProto; import org.apache.ratis.proto.RaftProtos.SnapshotManagementRequestProto; import org.apache.ratis.proto.RaftProtos.LeaderElectionManagementRequestProto; +import org.apache.ratis.thirdparty.io.netty.handler.ssl.SslContext; import org.apache.ratis.util.IOUtils; import org.apache.ratis.util.JavaUtils; import org.apache.ratis.util.PeerProxyMap; @@ -54,9 +54,9 @@ public class GrpcClientRpc extends RaftClientRpcWithProxy(clientId.toString(), - p -> new GrpcClientProtocolClient(clientId, p, properties, adminTlsConfig, clientTlsConfig))); + p -> new GrpcClientProtocolClient(clientId, p, properties, adminSslContext, clientSslContext))); this.clientId = clientId; this.maxMessageSize = GrpcConfigKeys.messageSizeMax(properties, LOG::debug).getSizeInt(); } diff --git a/ratis-grpc/src/main/java/org/apache/ratis/grpc/server/GrpcServerProtocolClient.java b/ratis-grpc/src/main/java/org/apache/ratis/grpc/server/GrpcServerProtocolClient.java index 2e936bb0b4..a0a17dc9ff 100644 --- a/ratis-grpc/src/main/java/org/apache/ratis/grpc/server/GrpcServerProtocolClient.java +++ b/ratis-grpc/src/main/java/org/apache/ratis/grpc/server/GrpcServerProtocolClient.java @@ -1,4 +1,4 @@ -/** +/* * 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 @@ -17,13 +17,11 @@ */ package org.apache.ratis.grpc.server; -import org.apache.ratis.grpc.GrpcTlsConfig; import org.apache.ratis.grpc.GrpcUtil; import org.apache.ratis.grpc.util.StreamObserverWithTimeout; import org.apache.ratis.protocol.RaftPeerId; import org.apache.ratis.server.util.ServerStringUtils; import org.apache.ratis.thirdparty.io.grpc.ManagedChannel; -import org.apache.ratis.thirdparty.io.grpc.netty.GrpcSslContexts; import org.apache.ratis.thirdparty.io.grpc.netty.NegotiationType; import org.apache.ratis.thirdparty.io.grpc.netty.NettyChannelBuilder; import org.apache.ratis.thirdparty.io.grpc.stub.CallStreamObserver; @@ -33,7 +31,7 @@ import org.apache.ratis.proto.grpc.RaftServerProtocolServiceGrpc.RaftServerProtocolServiceBlockingStub; import org.apache.ratis.proto.grpc.RaftServerProtocolServiceGrpc.RaftServerProtocolServiceStub; import org.apache.ratis.protocol.RaftPeer; -import org.apache.ratis.thirdparty.io.netty.handler.ssl.SslContextBuilder; +import org.apache.ratis.thirdparty.io.netty.handler.ssl.SslContext; import org.apache.ratis.util.TimeDuration; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -44,7 +42,7 @@ * This is a RaftClient implementation that supports streaming data to the raft * ring. The stream implementation utilizes gRPC. */ -public class GrpcServerProtocolClient implements Closeable { +class GrpcServerProtocolClient implements Closeable { // Common channel private final ManagedChannel channel; private final GrpcStubPool pool; @@ -60,42 +58,30 @@ public class GrpcServerProtocolClient implements Closeable { //visible for using in log / error messages AND to use in instrumented tests private final RaftPeerId raftPeerId; - public GrpcServerProtocolClient(RaftPeer target, int connections, int flowControlWindow, - TimeDuration requestTimeout, GrpcTlsConfig tlsConfig, boolean separateHBChannel) { + GrpcServerProtocolClient(RaftPeer target, int connections, int flowControlWindow, + TimeDuration requestTimeout, SslContext sslContext, boolean separateHBChannel) { raftPeerId = target.getId(); LOG.info("Build channel for {}", target); useSeparateHBChannel = separateHBChannel; - channel = buildChannel(target, flowControlWindow, tlsConfig); + channel = buildChannel(target, flowControlWindow, sslContext); blockingStub = RaftServerProtocolServiceGrpc.newBlockingStub(channel); asyncStub = RaftServerProtocolServiceGrpc.newStub(channel); if (useSeparateHBChannel) { - hbChannel = buildChannel(target, flowControlWindow, tlsConfig); + hbChannel = buildChannel(target, flowControlWindow, sslContext); hbAsyncStub = RaftServerProtocolServiceGrpc.newStub(hbChannel); } requestTimeoutDuration = requestTimeout; - this.pool = new GrpcStubPool(target, connections, - ch -> RaftServerProtocolServiceGrpc.newStub(ch), tlsConfig); + this.pool = new GrpcStubPool<>(target, connections, RaftServerProtocolServiceGrpc::newStub, sslContext); } - private ManagedChannel buildChannel(RaftPeer target, int flowControlWindow, - GrpcTlsConfig tlsConfig) { + private ManagedChannel buildChannel(RaftPeer target, int flowControlWindow, SslContext sslContext) { NettyChannelBuilder channelBuilder = NettyChannelBuilder.forTarget(target.getAddress()); // ignore any http proxy for grpc channelBuilder.proxyDetector(uri -> null); - if (tlsConfig!= null) { - SslContextBuilder sslContextBuilder = GrpcSslContexts.forClient(); - GrpcUtil.setTrustManager(sslContextBuilder, tlsConfig.getTrustManager()); - if (tlsConfig.getMtlsEnabled()) { - GrpcUtil.setKeyManager(sslContextBuilder, tlsConfig.getKeyManager()); - } - try { - channelBuilder.useTransportSecurity().sslContext(sslContextBuilder.build()); - } catch (Exception ex) { - throw new IllegalArgumentException("Failed to build SslContext, peerId=" + raftPeerId - + ", tlsConfig=" + tlsConfig, ex); - } + if (sslContext != null) { + channelBuilder.useTransportSecurity().sslContext(sslContext); } else { channelBuilder.negotiationType(NegotiationType.PLAINTEXT); } diff --git a/ratis-grpc/src/main/java/org/apache/ratis/grpc/server/GrpcServicesImpl.java b/ratis-grpc/src/main/java/org/apache/ratis/grpc/server/GrpcServicesImpl.java index b686be0a29..b1af0960dc 100644 --- a/ratis-grpc/src/main/java/org/apache/ratis/grpc/server/GrpcServicesImpl.java +++ b/ratis-grpc/src/main/java/org/apache/ratis/grpc/server/GrpcServicesImpl.java @@ -19,8 +19,6 @@ import org.apache.ratis.conf.RaftProperties; import org.apache.ratis.grpc.GrpcConfigKeys; -import org.apache.ratis.grpc.GrpcTlsConfig; -import org.apache.ratis.grpc.GrpcUtil; import org.apache.ratis.grpc.metrics.MessageMetrics; import org.apache.ratis.grpc.metrics.intercept.server.MetricServerInterceptor; import org.apache.ratis.protocol.AdminAsynchronousProtocol; @@ -34,13 +32,11 @@ import org.apache.ratis.server.protocol.RaftServerAsynchronousProtocol; import org.apache.ratis.thirdparty.io.grpc.ServerInterceptor; import org.apache.ratis.thirdparty.io.grpc.ServerInterceptors; -import org.apache.ratis.thirdparty.io.grpc.netty.GrpcSslContexts; import org.apache.ratis.thirdparty.io.grpc.netty.NettyServerBuilder; import org.apache.ratis.thirdparty.io.grpc.Server; import org.apache.ratis.thirdparty.io.grpc.stub.StreamObserver; import org.apache.ratis.thirdparty.io.netty.channel.ChannelOption; -import org.apache.ratis.thirdparty.io.netty.handler.ssl.ClientAuth; -import org.apache.ratis.thirdparty.io.netty.handler.ssl.SslContextBuilder; +import org.apache.ratis.thirdparty.io.netty.handler.ssl.SslContext; import org.apache.ratis.proto.RaftProtos.*; import org.apache.ratis.util.*; @@ -56,8 +52,6 @@ import java.util.concurrent.ExecutorService; import java.util.function.Supplier; -import static org.apache.ratis.thirdparty.io.netty.handler.ssl.SslProvider.OPENSSL; - /** A grpc implementation of {@link org.apache.ratis.server.RaftServerRpc}. */ public final class GrpcServicesImpl extends RaftServerRpcWithProxy> @@ -106,13 +100,14 @@ public static final class Builder { private String adminHost; private int adminPort; - private GrpcTlsConfig adminTlsConfig; + private SslContext adminSslContext; private String clientHost; private int clientPort; - private GrpcTlsConfig clientTlsConfig; + private SslContext clientSslContext; private String serverHost; private int serverPort; - private GrpcTlsConfig serverTlsConfig; + private SslContext serverSslContextForServer; + private SslContext serverSslContextForClient; private int serverStubPoolSize; private SizeInBytes messageSizeMax; @@ -158,7 +153,7 @@ public Builder setCustomizer(Customizer customizer) { private GrpcServerProtocolClient newGrpcServerProtocolClient(RaftPeer target) { return new GrpcServerProtocolClient(target, serverStubPoolSize, flowControlWindow.getSizeInt(), - requestTimeoutDuration, serverTlsConfig, separateHeartbeatChannel); + requestTimeoutDuration, serverSslContextForClient, separateHeartbeatChannel); } private ExecutorService newExecutor() { @@ -188,18 +183,18 @@ Server buildServer(NettyServerBuilder builder, EnumSet types) } private NettyServerBuilder newNettyServerBuilderForServer() { - return newNettyServerBuilder(serverHost, serverPort, serverTlsConfig); + return newNettyServerBuilder(serverHost, serverPort, serverSslContextForServer); } private NettyServerBuilder newNettyServerBuilderForAdmin() { - return newNettyServerBuilder(adminHost, adminPort, adminTlsConfig); + return newNettyServerBuilder(adminHost, adminPort, adminSslContext); } private NettyServerBuilder newNettyServerBuilderForClient() { - return newNettyServerBuilder(clientHost, clientPort, clientTlsConfig); + return newNettyServerBuilder(clientHost, clientPort, clientSslContext); } - private NettyServerBuilder newNettyServerBuilder(String hostname, int port, GrpcTlsConfig tlsConfig) { + private NettyServerBuilder newNettyServerBuilder(String hostname, int port, SslContext sslContext) { final InetSocketAddress address = hostname == null || hostname.isEmpty() ? new InetSocketAddress(port) : new InetSocketAddress(hostname, port); final NettyServerBuilder nettyServerBuilder = NettyServerBuilder.forAddress(address) @@ -207,19 +202,9 @@ private NettyServerBuilder newNettyServerBuilder(String hostname, int port, Grpc .maxInboundMessageSize(messageSizeMax.getSizeInt()) .flowControlWindow(flowControlWindow.getSizeInt()); - if (tlsConfig != null) { + if (sslContext != null) { LOG.info("Setting TLS for {}", address); - SslContextBuilder sslContextBuilder = GrpcUtil.initSslContextBuilderForServer(tlsConfig.getKeyManager()); - if (tlsConfig.getMtlsEnabled()) { - sslContextBuilder.clientAuth(ClientAuth.REQUIRE); - GrpcUtil.setTrustManager(sslContextBuilder, tlsConfig.getTrustManager()); - } - sslContextBuilder = GrpcSslContexts.configure(sslContextBuilder, OPENSSL); - try { - nettyServerBuilder.sslContext(sslContextBuilder.build()); - } catch (Exception ex) { - throw new IllegalArgumentException("Failed to build SslContext, tlsConfig=" + tlsConfig, ex); - } + nettyServerBuilder.sslContext(sslContext); } return nettyServerBuilder; } @@ -253,18 +238,23 @@ public GrpcServicesImpl build() { return new GrpcServicesImpl(this); } - public Builder setAdminTlsConfig(GrpcTlsConfig config) { - this.adminTlsConfig = config; + public Builder setAdminSslContext(SslContext adminSslContext) { + this.adminSslContext = adminSslContext; + return this; + } + + public Builder setClientSslContext(SslContext clientSslContext) { + this.clientSslContext = clientSslContext; return this; } - public Builder setClientTlsConfig(GrpcTlsConfig config) { - this.clientTlsConfig = config; + public Builder setServerSslContextForServer(SslContext serverSslContextForServer) { + this.serverSslContextForServer = serverSslContextForServer; return this; } - public Builder setServerTlsConfig(GrpcTlsConfig config) { - this.serverTlsConfig = config; + public Builder setServerSslContextForClient(SslContext serverSslContextForClient) { + this.serverSslContextForClient = serverSslContextForClient; return this; } } diff --git a/ratis-grpc/src/main/java/org/apache/ratis/grpc/server/GrpcStubPool.java b/ratis-grpc/src/main/java/org/apache/ratis/grpc/server/GrpcStubPool.java index fcfb0f1b89..c949707a42 100644 --- a/ratis-grpc/src/main/java/org/apache/ratis/grpc/server/GrpcStubPool.java +++ b/ratis-grpc/src/main/java/org/apache/ratis/grpc/server/GrpcStubPool.java @@ -17,11 +17,8 @@ */ package org.apache.ratis.grpc.server; -import org.apache.ratis.grpc.GrpcTlsConfig; -import org.apache.ratis.grpc.GrpcUtil; import org.apache.ratis.protocol.RaftPeer; import org.apache.ratis.thirdparty.io.grpc.ManagedChannel; -import org.apache.ratis.thirdparty.io.grpc.netty.GrpcSslContexts; import org.apache.ratis.thirdparty.io.grpc.netty.NegotiationType; import org.apache.ratis.thirdparty.io.grpc.netty.NettyChannelBuilder; import org.apache.ratis.thirdparty.io.grpc.stub.AbstractStub; @@ -29,7 +26,7 @@ import org.apache.ratis.thirdparty.io.netty.channel.WriteBufferWaterMark; import org.apache.ratis.thirdparty.io.netty.channel.nio.NioEventLoopGroup; import org.apache.ratis.thirdparty.io.netty.channel.socket.nio.NioSocketChannel; -import org.apache.ratis.thirdparty.io.netty.handler.ssl.SslContextBuilder; +import org.apache.ratis.thirdparty.io.netty.handler.ssl.SslContext; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -39,7 +36,6 @@ import java.util.concurrent.Semaphore; import java.util.concurrent.ThreadLocalRandom; import java.util.concurrent.TimeUnit; -import java.util.concurrent.atomic.AtomicInteger; import java.util.function.Function; final class GrpcStubPool> { @@ -66,16 +62,14 @@ void release() { } private final List> pool; - private final AtomicInteger rr = new AtomicInteger(); private final NioEventLoopGroup elg; private final int size; - GrpcStubPool(RaftPeer target, int n, Function stubFactory, GrpcTlsConfig tlsConfig) { - this(target, n, stubFactory, tlsConfig, Math.max(2, Runtime.getRuntime().availableProcessors() / 2), 16); + GrpcStubPool(RaftPeer target, int n, Function stubFactory, SslContext sslContext) { + this(target, n, stubFactory, sslContext, Math.max(2, Runtime.getRuntime().availableProcessors() / 2), 16); } - GrpcStubPool(RaftPeer target, int n, - Function stubFactory, GrpcTlsConfig tlsConf, + GrpcStubPool(RaftPeer target, int n, Function stubFactory, SslContext sslContext, int elgThreads, int maxInflightPerConn) { this.elg = new NioEventLoopGroup(elgThreads); ArrayList> tmp = new ArrayList<>(n); @@ -87,18 +81,9 @@ void release() { .keepAliveWithoutCalls(true) .idleTimeout(24, TimeUnit.HOURS) .withOption(ChannelOption.WRITE_BUFFER_WATER_MARK, new WriteBufferWaterMark(64 << 10, 128 << 10)); - if (tlsConf != null) { + if (sslContext != null) { LOG.debug("Setting TLS for {}", target.getAddress()); - SslContextBuilder sslContextBuilder = GrpcSslContexts.forClient(); - GrpcUtil.setTrustManager(sslContextBuilder, tlsConf.getTrustManager()); - if (tlsConf.getMtlsEnabled()) { - GrpcUtil.setKeyManager(sslContextBuilder, tlsConf.getKeyManager()); - } - try { - channelBuilder.useTransportSecurity().sslContext(sslContextBuilder.build()); - } catch (Exception ex) { - throw new RuntimeException(ex); - } + channelBuilder.useTransportSecurity().sslContext(sslContext); } else { channelBuilder.negotiationType(NegotiationType.PLAINTEXT); } @@ -124,7 +109,7 @@ PooledStub acquire() throws InterruptedException { } public void close() { - for (PooledStub p : pool) { + for (PooledStub p : pool) { p.ch.shutdown(); } elg.shutdownGracefully(); From d9d705f208561c24acc73a6feb3e89518f420a51 Mon Sep 17 00:00:00 2001 From: Tsz-Wo Nicholas Sze Date: Sun, 28 Sep 2025 23:58:26 -0700 Subject: [PATCH 206/397] RATIS-2338. Print version info when starting a server. (#1292) --- pom.xml | 4 + .../org/apache/ratis/util/VersionInfo.java | 149 ++++++++++++++++++ ratis-grpc/pom.xml | 11 -- ratis-metrics-default/pom.xml | 10 -- ratis-server/pom.xml | 6 - .../ratis/server/impl/RaftServerProxy.java | 3 + .../ratis/server/impl/ServerImplUtils.java | 2 +- .../ratis/server/ServerBuilderTest.java | 9 -- src/main/resources/ratis-version.properties | 1 + 9 files changed, 158 insertions(+), 37 deletions(-) create mode 100644 ratis-common/src/main/java/org/apache/ratis/util/VersionInfo.java diff --git a/pom.xml b/pom.xml index b32c7ac1db..9ea7379770 100644 --- a/pom.xml +++ b/pom.xml @@ -789,6 +789,10 @@ ${project.basedir} + + */src/main/java/**/*.java + */src/main/proto/*.proto + diff --git a/ratis-common/src/main/java/org/apache/ratis/util/VersionInfo.java b/ratis-common/src/main/java/org/apache/ratis/util/VersionInfo.java new file mode 100644 index 0000000000..98f662f373 --- /dev/null +++ b/ratis-common/src/main/java/org/apache/ratis/util/VersionInfo.java @@ -0,0 +1,149 @@ +/* + * 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.ratis.util; + +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.io.IOException; +import java.io.InputStream; +import java.util.Collections; +import java.util.EnumMap; +import java.util.LinkedHashMap; +import java.util.Map; +import java.util.Objects; +import java.util.Properties; +import java.util.function.Consumer; + +/** + * This class is for the resource generated by hadoop-maven-plugins:version-info. + *

+ * This class is immutable. + */ +public final class VersionInfo { + static final Logger LOG = LoggerFactory.getLogger(VersionInfo.class); + + private static final String RATIS_VERSION_PROPERTIES = "ratis-version.properties"; + private static final String UNKNOWN = ""; + private static final String FORMAT = " %20s: %s"; + + private enum SoftwareInfo { + // the ordering is the output ordering + NAME, VERSION, URL, REVISION; + + static SoftwareInfo parse(String key) { + for (SoftwareInfo info : SoftwareInfo.values()) { + if (info.name().toLowerCase().equals(key)) { + return info; + } + } + return null; + } + } + + private enum RuntimeInfo { + // the ordering is the output ordering + JAVA, USER; + + static final InfoMap MAP; + + static { + final EnumMap map = new EnumMap<>(RuntimeInfo.class); + final Properties properties = System.getProperties(); + map.put(JAVA, properties.getProperty("java.vm.name") + " " + properties.getProperty("java.runtime.version")); + map.put(USER, properties.getProperty("user.name")); + MAP = new InfoMap<>(map); + } + } + + private static class InfoMap> { + private final Map map; + + InfoMap(EnumMap map) { + this.map = Collections.unmodifiableMap(map); + } + + String getOrDefault(INFO info) { + return map.getOrDefault(info, UNKNOWN); + } + + String format(INFO info) { + return String.format(FORMAT, info.name().toLowerCase(), getOrDefault(info)); + } + } + + public static VersionInfo load(Class clazz) { + final Properties properties = new Properties(); + + try (InputStream in = clazz.getClassLoader().getResourceAsStream(RATIS_VERSION_PROPERTIES)) { + if (in != null) { + properties.load(in); + } else { + LOG.warn("Resource '{}' not found for {}", RATIS_VERSION_PROPERTIES, clazz); + } + } catch (IOException e) { + LOG.warn("Failed to load resource '{}' for {}", RATIS_VERSION_PROPERTIES, clazz, e); + } + return new VersionInfo(clazz, properties); + } + + private final Class clazz; + private final InfoMap runtimeInfos = RuntimeInfo.MAP; + private final InfoMap softwareInfos; + private final Map otherInfos; + + private VersionInfo(Class clazz, Properties properties) { + this.clazz = Objects.requireNonNull(clazz, "clazz == null"); + + final EnumMap softwareInfoMap = new EnumMap<>(SoftwareInfo.class); + final Map others = new LinkedHashMap<>(); // preserve insertion order + for (Map.Entry e : properties.entrySet()) { + final String key = e.getKey().toString(); + final String value = e.getValue().toString(); + final SoftwareInfo k = SoftwareInfo.parse(key); + if (k != null) { + softwareInfoMap.put(k, value); + } else { + others.put(key, value); + } + } + + this.softwareInfos = new InfoMap<>(softwareInfoMap); + this.otherInfos = Collections.unmodifiableMap(others); + } + + public void printStartupMessages(Object name, Consumer log) { + Objects.requireNonNull(name, "name == null"); + log.accept(String.format("Starting %s -- %s %s", + softwareInfos.getOrDefault(SoftwareInfo.NAME), clazz.getSimpleName(), name)); + final SoftwareInfo[] softwareInfoValues = SoftwareInfo.values(); + for(int i = 1; i < softwareInfoValues.length; i++) { + log.accept(softwareInfos.format(softwareInfoValues[i])); + } + for(RuntimeInfo runtimeInfo : RuntimeInfo.values()) { + log.accept(runtimeInfos.format(runtimeInfo)); + } + for (Map.Entry e : otherInfos.entrySet()) { + log.accept(String.format(FORMAT, e.getKey(), e.getValue())); + } + } + + public static void main(String[] args) { + VersionInfo.load(VersionInfo.class).printStartupMessages(":", System.out::println); + } +} diff --git a/ratis-grpc/pom.xml b/ratis-grpc/pom.xml index 554a1a763a..1ae667b185 100644 --- a/ratis-grpc/pom.xml +++ b/ratis-grpc/pom.xml @@ -37,12 +37,6 @@ ratis-common org.apache.ratis - - ratis-common - org.apache.ratis - test - test-jar - ratis-client org.apache.ratis @@ -79,10 +73,5 @@ junit-jupiter-api test - - org.mockito - mockito-core - test - diff --git a/ratis-metrics-default/pom.xml b/ratis-metrics-default/pom.xml index 1f9e368622..cbdb2dfc66 100644 --- a/ratis-metrics-default/pom.xml +++ b/ratis-metrics-default/pom.xml @@ -29,19 +29,9 @@ ratis-metrics-api org.apache.ratis - - ratis-proto - org.apache.ratis - - - ratis-common - org.apache.ratis - ratis-common org.apache.ratis - test - test-jar diff --git a/ratis-server/pom.xml b/ratis-server/pom.xml index 54ea8e1deb..7a4f929951 100644 --- a/ratis-server/pom.xml +++ b/ratis-server/pom.xml @@ -48,12 +48,6 @@ ratis-client org.apache.ratis - - ratis-client - org.apache.ratis - test - test-jar - ratis-server-api diff --git a/ratis-server/src/main/java/org/apache/ratis/server/impl/RaftServerProxy.java b/ratis-server/src/main/java/org/apache/ratis/server/impl/RaftServerProxy.java index 2d265cf95d..8539fa99ec 100644 --- a/ratis-server/src/main/java/org/apache/ratis/server/impl/RaftServerProxy.java +++ b/ratis-server/src/main/java/org/apache/ratis/server/impl/RaftServerProxy.java @@ -53,6 +53,7 @@ import org.apache.ratis.util.Preconditions; import org.apache.ratis.util.ProtoUtils; import org.apache.ratis.util.TimeDuration; +import org.apache.ratis.util.VersionInfo; import java.io.Closeable; import java.io.File; @@ -207,6 +208,8 @@ String toString(RaftGroupId groupId, CompletableFuture f) { RaftServerProxy(RaftPeerId id, StateMachine.Registry stateMachineRegistry, RaftProperties properties, Parameters parameters, ThreadGroup threadGroup) { + VersionInfo.load(getClass()).printStartupMessages(id, LOG::info); + this.properties = properties; this.stateMachineRegistry = stateMachineRegistry; diff --git a/ratis-server/src/main/java/org/apache/ratis/server/impl/ServerImplUtils.java b/ratis-server/src/main/java/org/apache/ratis/server/impl/ServerImplUtils.java index 864b402a23..1a5fcfc855 100644 --- a/ratis-server/src/main/java/org/apache/ratis/server/impl/ServerImplUtils.java +++ b/ratis-server/src/main/java/org/apache/ratis/server/impl/ServerImplUtils.java @@ -170,8 +170,8 @@ public static RaftServerProxy newRaftServer( RaftPeerId id, RaftGroup group, RaftStorage.StartupOption option, StateMachine.Registry stateMachineRegistry, ThreadGroup threadGroup, RaftProperties properties, Parameters parameters) throws IOException { RaftServer.LOG.debug("newRaftServer: {}, {}", id, group); + Objects.requireNonNull(id, "id == null"); if (group != null && !group.getPeers().isEmpty()) { - Objects.requireNonNull(id, () -> "RaftPeerId " + id + " is not in RaftGroup " + group); Objects.requireNonNull(group.getPeer(id), () -> "RaftPeerId " + id + " is not in RaftGroup " + group); } final RaftServerProxy proxy = newRaftServer(id, stateMachineRegistry, threadGroup, properties, parameters); diff --git a/ratis-test/src/test/java/org/apache/ratis/server/ServerBuilderTest.java b/ratis-test/src/test/java/org/apache/ratis/server/ServerBuilderTest.java index 58d553367c..15040d3d2d 100644 --- a/ratis-test/src/test/java/org/apache/ratis/server/ServerBuilderTest.java +++ b/ratis-test/src/test/java/org/apache/ratis/server/ServerBuilderTest.java @@ -90,13 +90,4 @@ public void testPeerIdWithNullRaftGroup() throws Exception { .build(); server.close(); } - - @Test - public void testNullPeerIdWithNullRaftGroup() throws Exception { - RaftServer server = RaftServer.newBuilder() - .setStateMachine(new BaseStateMachine()) - .setProperties(new RaftProperties()) - .build(); - server.close(); - } } diff --git a/src/main/resources/ratis-version.properties b/src/main/resources/ratis-version.properties index f34dc73dc7..7413ff9942 100644 --- a/src/main/resources/ratis-version.properties +++ b/src/main/resources/ratis-version.properties @@ -17,4 +17,5 @@ # name=${project.name} version=${project.version} +url=${version-info.scm.uri} revision=${version-info.scm.commit} From af1822026c40edb2e724c8cbd338fd93bc7390c8 Mon Sep 17 00:00:00 2001 From: Tsz-Wo Nicholas Sze Date: Tue, 30 Sep 2025 21:35:21 -0700 Subject: [PATCH 207/397] RATIS-2332. Start new servers with empty peer list in tests (#1294) --- .../ratis/InstallSnapshotFromLeaderTests.java | 6 +-- .../InstallSnapshotNotificationTests.java | 8 ++-- .../apache/ratis/RaftExceptionBaseTest.java | 3 +- .../server/impl/LeaderElectionTests.java | 6 +-- .../ratis/server/impl/MiniRaftCluster.java | 41 +++---------------- .../impl/RaftReconfigurationBaseTest.java | 4 +- .../statemachine/RaftSnapshotBaseTest.java | 4 +- .../cli/sh/PeerCommandIntegrationTest.java | 2 +- 8 files changed, 20 insertions(+), 54 deletions(-) diff --git a/ratis-server/src/test/java/org/apache/ratis/InstallSnapshotFromLeaderTests.java b/ratis-server/src/test/java/org/apache/ratis/InstallSnapshotFromLeaderTests.java index 687e1c1a51..e7683a3991 100644 --- a/ratis-server/src/test/java/org/apache/ratis/InstallSnapshotFromLeaderTests.java +++ b/ratis-server/src/test/java/org/apache/ratis/InstallSnapshotFromLeaderTests.java @@ -116,8 +116,7 @@ private void testMultiFileInstallSnapshot(CLUSTER cluster) throws Exception { Assertions.assertEquals(3, snapshot.getFiles().size()); // add two more peers - final PeerChanges change = cluster.addNewPeers(2, true, - true); + final PeerChanges change = cluster.addNewPeers(2, true); // trigger setConfiguration cluster.setConfiguration(change.getPeersInNewConf()); @@ -161,8 +160,7 @@ private void testInstallSnapshotDuringLeaderSwitch(CLUSTER cluster) throws Excep } // add two more peers and install snapshot from leaders - final PeerChanges change = cluster.addNewPeers(2, true, - true); + final PeerChanges change = cluster.addNewPeers(2, true); try (final RaftClient client = cluster.createClient(leaderId, RetryPolicies.noRetry())) { final RaftException e = Assertions.assertThrows(RaftException.class, () -> client.admin().setConfiguration(change.getPeersInNewConf())); diff --git a/ratis-server/src/test/java/org/apache/ratis/InstallSnapshotNotificationTests.java b/ratis-server/src/test/java/org/apache/ratis/InstallSnapshotNotificationTests.java index e03171d66b..931bf6317f 100644 --- a/ratis-server/src/test/java/org/apache/ratis/InstallSnapshotNotificationTests.java +++ b/ratis-server/src/test/java/org/apache/ratis/InstallSnapshotNotificationTests.java @@ -241,7 +241,7 @@ private void testAddNewFollowers(CLUSTER cluster, int numRequests) throws Except Assertions.assertTrue(set); // Add new peer(s) - final PeerChanges change = cluster.addNewPeers(1, true, true); + final PeerChanges change = cluster.addNewPeers(1, true); // trigger setConfiguration RaftServerTestUtil.runWithMinorityPeers(cluster, change.getPeersInNewConf(), cluster::setConfiguration); @@ -389,7 +389,7 @@ private void testInstallSnapshotNotificationCount(CLUSTER cluster) throws Except // Add new peer(s) who will need snapshots from the leader. final int numNewPeers = 1; - final PeerChanges change = cluster.addNewPeers(numNewPeers, true, true); + final PeerChanges change = cluster.addNewPeers(numNewPeers, true); // trigger setConfiguration RaftServerTestUtil.runWithMinorityPeers(cluster, change.getPeersInNewConf(), cluster::setConfiguration); RaftServerTestUtil.waitAndCheckNewConf(cluster, change.getPeersInNewConf(), 0, null); @@ -475,7 +475,7 @@ private void testInstallSnapshotInstalledEvent(CLUSTER cluster) throws Exception Assertions.assertTrue(set); // add one new peer - final PeerChanges change = cluster.addNewPeers(1, true, true); + final PeerChanges change = cluster.addNewPeers(1, true); // trigger setConfiguration RaftServerTestUtil.runWithMinorityPeers(cluster, change.getPeersInNewConf(), cluster::setConfiguration); RaftServerTestUtil.waitAndCheckNewConf(cluster, change.getPeersInNewConf(), 0, null); @@ -551,7 +551,7 @@ private void testInstallSnapshotDuringBootstrap(CLUSTER cluster) throws Exceptio // Add new peer(s) final int numNewPeers = 1; - final PeerChanges change = cluster.addNewPeers(numNewPeers, true, true); + final PeerChanges change = cluster.addNewPeers(numNewPeers, true); // trigger setConfiguration RaftServerTestUtil.runWithMinorityPeers(cluster, change.getPeersInNewConf(), cluster::setConfiguration); diff --git a/ratis-server/src/test/java/org/apache/ratis/RaftExceptionBaseTest.java b/ratis-server/src/test/java/org/apache/ratis/RaftExceptionBaseTest.java index a08731c580..b06d6e904a 100644 --- a/ratis-server/src/test/java/org/apache/ratis/RaftExceptionBaseTest.java +++ b/ratis-server/src/test/java/org/apache/ratis/RaftExceptionBaseTest.java @@ -110,8 +110,7 @@ void runTestNotLeaderExceptionWithReconf(CLUSTER cluster) throws Exception { final RaftPeerId newLeader = RaftTestUtil.changeLeader(cluster, oldLeader); // add two more peers - PeerChanges change = cluster.addNewPeers(new String[]{ - "ss1", "ss2"}, true, false); + PeerChanges change = cluster.addNewPeers(2, true); // trigger setConfiguration LOG.info("Start changing the configuration: {}", change.getPeersInNewConf()); try (final RaftClient c2 = cluster.createClient(newLeader)) { diff --git a/ratis-server/src/test/java/org/apache/ratis/server/impl/LeaderElectionTests.java b/ratis-server/src/test/java/org/apache/ratis/server/impl/LeaderElectionTests.java index d4023a23d2..724a066430 100644 --- a/ratis-server/src/test/java/org/apache/ratis/server/impl/LeaderElectionTests.java +++ b/ratis-server/src/test/java/org/apache/ratis/server/impl/LeaderElectionTests.java @@ -156,7 +156,7 @@ public void testAddServerForWaitReady() throws IOException, InterruptedException } // add 3 new servers and wait longer time CodeInjectionForTesting.put(RaftServerImpl.START_COMPLETE, new SleepCode(2000)); - final PeerChanges peerChanges = cluster.addNewPeers(2, true, false); + final PeerChanges peerChanges = cluster.addNewPeers(2, true); LOG.info("add new 3 servers"); LOG.info(cluster.printServers()); RaftClientReply reply = client.admin().setConfiguration(SetConfigurationRequest.Arguments.newBuilder() @@ -461,7 +461,7 @@ public void testAddListener() throws Exception { client.io().send(new RaftTestUtil.SimpleMessage("message")); List servers = cluster.getPeers(); assertEquals(servers.size(), 3); - final PeerChanges changes = cluster.addNewPeers(1, true, false, LISTENER); + final PeerChanges changes = cluster.addNewPeers(1, true); final List added = changes.getAddedPeers(); final RaftClientReply reply = client.admin().setConfiguration(servers, added); assertTrue(reply.isSuccess()); @@ -486,7 +486,7 @@ public void testAddFollowerWhenExistsListener() throws Exception { List listener = new ArrayList<>( leader.getRaftConf().getAllPeers(RaftProtos.RaftPeerRole.LISTENER)); assertEquals(1, listener.size()); - final PeerChanges changes = cluster.addNewPeers(1, true, false); + final PeerChanges changes = cluster.addNewPeers(1, true); final List newPeers = new ArrayList<>(changes.getAddedPeers()); newPeers.addAll(leader.getRaftConf().getAllPeers(RaftProtos.RaftPeerRole.FOLLOWER)); RaftClientReply reply = client.admin().setConfiguration(newPeers, listener); diff --git a/ratis-server/src/test/java/org/apache/ratis/server/impl/MiniRaftCluster.java b/ratis-server/src/test/java/org/apache/ratis/server/impl/MiniRaftCluster.java index 6dd5a12768..253b2cdeb1 100644 --- a/ratis-server/src/test/java/org/apache/ratis/server/impl/MiniRaftCluster.java +++ b/ratis-server/src/test/java/org/apache/ratis/server/impl/MiniRaftCluster.java @@ -22,7 +22,6 @@ import org.apache.ratis.client.RaftClient; import org.apache.ratis.conf.Parameters; import org.apache.ratis.conf.RaftProperties; -import org.apache.ratis.proto.RaftProtos; import org.apache.ratis.protocol.ClientId; import org.apache.ratis.protocol.Message; import org.apache.ratis.protocol.RaftClientReply; @@ -82,6 +81,8 @@ import java.util.stream.Stream; import java.util.stream.StreamSupport; +import static org.apache.ratis.proto.RaftProtos.RaftPeerRole.LISTENER; + public abstract class MiniRaftCluster implements Closeable { public static final Logger LOG = LoggerFactory.getLogger(MiniRaftCluster.class); @@ -246,7 +247,7 @@ public static RaftGroup initRaftGroup(Collection ids, Collection Stream listener = listenerIds.stream() .map(id -> RaftPeer.newBuilder().setId(id)) .map(MiniRaftCluster::assignAddresses) - .map(p -> p.setStartupRole(RaftProtos.RaftPeerRole.LISTENER)) + .map(p -> p.setStartupRole(LISTENER)) .map(RaftPeer.Builder::build); final RaftPeer[] peers = Stream.concat(peer, listener).toArray(RaftPeer[]::new); @@ -433,43 +434,11 @@ private static List toRaftPeers(Iterable servers) { public PeerChanges addNewPeers(int number, boolean startNewPeer) throws IOException { - return addNewPeers(generateIds(number, servers.size()), startNewPeer, false); - } - - public PeerChanges addNewPeers(int number, boolean startNewPeer, - boolean emptyPeer) throws IOException { - return addNewPeers(generateIds(number, servers.size()), startNewPeer, emptyPeer, - RaftProtos.RaftPeerRole.FOLLOWER); - } - - public PeerChanges addNewPeers(String[] ids, boolean startNewPeer, - boolean emptyPeer) throws IOException { - return addNewPeers(ids, startNewPeer, emptyPeer, RaftProtos.RaftPeerRole.FOLLOWER); - } - - public PeerChanges addNewPeers(int number, boolean startNewPeer, - boolean emptyPeer, RaftProtos.RaftPeerRole startRole) throws IOException { - return addNewPeers(generateIds(number, servers.size()), startNewPeer, emptyPeer, startRole); - } - - public PeerChanges addNewPeers(String[] ids, boolean startNewPeer, - boolean emptyPeer, RaftProtos.RaftPeerRole startRole) throws IOException { + final String[] ids = generateIds(number, servers.size()); LOG.info("Add new peers {}", Arrays.asList(ids)); final Iterable peerIds = CollectionUtils.as(Arrays.asList(ids), RaftPeerId::valueOf); - final RaftGroup raftGroup; - if (emptyPeer) { - raftGroup = RaftGroup.valueOf(group.getGroupId(), Collections.emptyList()); - } else { - final Collection newPeers = StreamSupport.stream(peerIds.spliterator(), false) - .map(id -> RaftPeer.newBuilder().setId(id) - .setStartupRole(startRole)) - .map(MiniRaftCluster::assignAddresses) - .map(RaftPeer.Builder::build) - .collect(Collectors.toSet()); - newPeers.addAll(group.getPeers()); - raftGroup = RaftGroup.valueOf(group.getGroupId(), newPeers); - } + final RaftGroup raftGroup = RaftGroup.valueOf(group.getGroupId(), Collections.emptyList()); // create and add new RaftServers final Collection newServers = putNewServers(peerIds, true, raftGroup); diff --git a/ratis-server/src/test/java/org/apache/ratis/server/impl/RaftReconfigurationBaseTest.java b/ratis-server/src/test/java/org/apache/ratis/server/impl/RaftReconfigurationBaseTest.java index 036718ea3b..7a2f1a24ee 100644 --- a/ratis-server/src/test/java/org/apache/ratis/server/impl/RaftReconfigurationBaseTest.java +++ b/ratis-server/src/test/java/org/apache/ratis/server/impl/RaftReconfigurationBaseTest.java @@ -159,7 +159,7 @@ public void testLeaderElectionWhenChangeFromSingleToHA() throws Exception { RaftGroupId groupId = cluster.getGroup().getGroupId(); RaftPeer curPeer = cluster.getGroup().getPeers().iterator().next(); - RaftPeer newPeer = cluster.addNewPeers(1, true, true).getAddedPeers().get(0); + RaftPeer newPeer = cluster.addNewPeers(1, true).getAddedPeers().get(0); RaftServerProxy leaderServer = cluster.getServer(curPeer.getId()); @@ -336,7 +336,7 @@ void runTestReconfTwice(CLUSTER cluster) throws Exception { CountDownLatch latch = new CountDownLatch(1); Thread clientThread = new Thread(() -> { try { - PeerChanges c1 = cluster.addNewPeers(2, true, true); + PeerChanges c1 = cluster.addNewPeers(2, true); LOG.info("Start changing the configuration: {}", c1.getPeersInNewConf()); RaftClientReply reply = client.admin().setConfiguration(c1.getPeersInNewConf()); diff --git a/ratis-server/src/test/java/org/apache/ratis/statemachine/RaftSnapshotBaseTest.java b/ratis-server/src/test/java/org/apache/ratis/statemachine/RaftSnapshotBaseTest.java index 91f974dbfa..09b5ee59a6 100644 --- a/ratis-server/src/test/java/org/apache/ratis/statemachine/RaftSnapshotBaseTest.java +++ b/ratis-server/src/test/java/org/apache/ratis/statemachine/RaftSnapshotBaseTest.java @@ -227,7 +227,7 @@ void runTestBasicInstallSnapshot(CLUSTER cluster) throws Exception { } // add a new peer - final PeerChanges change = cluster.addNewPeers(1, true, true); + final PeerChanges change = cluster.addNewPeers(1, true); // trigger setConfiguration RaftServerTestUtil.runWithMinorityPeers(cluster, change.getPeersInNewConf(), cluster::setConfiguration); @@ -292,7 +292,7 @@ void runTestInstallSnapshotDuringBootstrap(CLUSTER cluster) throws Exception { assertLeaderContent(cluster); // add a new peer - final PeerChanges change = cluster.addNewPeers(1, true, true); + final PeerChanges change = cluster.addNewPeers(1, true); // trigger setConfiguration RaftServerTestUtil.runWithMinorityPeers(cluster, change.getPeersInNewConf(), cluster::setConfiguration); diff --git a/ratis-test/src/test/java/org/apache/ratis/shell/cli/sh/PeerCommandIntegrationTest.java b/ratis-test/src/test/java/org/apache/ratis/shell/cli/sh/PeerCommandIntegrationTest.java index f12fc7d4e5..22f15e531a 100644 --- a/ratis-test/src/test/java/org/apache/ratis/shell/cli/sh/PeerCommandIntegrationTest.java +++ b/ratis-test/src/test/java/org/apache/ratis/shell/cli/sh/PeerCommandIntegrationTest.java @@ -82,7 +82,7 @@ void runTestPeerAddCommand(MiniRaftCluster cluster) throws Exception { RaftTestUtil.waitForLeader(cluster); final List peers = cluster.getPeers(); - final List newPeers = cluster.addNewPeers(1, true, true).getAddedPeers(); + final List newPeers = cluster.addNewPeers(1, true).getAddedPeers(); RaftServerTestUtil.waitAndCheckNewConf(cluster, peers, 0, null); StringBuilder sb = new StringBuilder(); From e5f0eb68b7237350c5de68edaf2a23fedb1abce4 Mon Sep 17 00:00:00 2001 From: Potato Date: Thu, 2 Oct 2025 20:48:12 +0800 Subject: [PATCH 208/397] RATIS-2339. Avoid "commit does not belong to any branch" during release (#1293) Signed-off-by: OneSizeFitQuorum Co-authored-by: Doroszlai, Attila <6454655+adoroszlai@users.noreply.github.com> --- dev-support/make_rc.sh | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/dev-support/make_rc.sh b/dev-support/make_rc.sh index c6a93a6d9d..b94305a054 100755 --- a/dev-support/make_rc.sh +++ b/dev-support/make_rc.sh @@ -145,6 +145,7 @@ mvnFun() { 5-publish-git(){ cd "$projectdir" + git push apache HEAD:"release-${RATISVERSION}" git push apache "ratis-${RATISVERSION}${RC}" } @@ -169,7 +170,7 @@ Please choose from available phases (eg. make_rc.sh 1-prepare-src): 4-assembly: This step copies all the required artifacts to the svn directory and ($SVNDISTDIR) creates the signatures/checksum files. - 5-publish-git: Only do it if everything is fine. It pushes the rc tag to the repository. + 5-publish-git: Only do it if everything is fine. It pushes the rc tag and release branch to the repository. 6-publish-svn: Uploads the artifacts to the apache dev staging area to start the vote. From 4c3a3b4d7bbceae38303bae9e0e35abfafbeb9d8 Mon Sep 17 00:00:00 2001 From: Symious <14933944+symious@users.noreply.github.com> Date: Fri, 3 Oct 2025 18:39:26 +0800 Subject: [PATCH 209/397] RATIS-2342. Remove NIO specification from GrpcStubPool. (#1297) --- .../org/apache/ratis/grpc/server/GrpcStubPool.java | 11 ++--------- 1 file changed, 2 insertions(+), 9 deletions(-) diff --git a/ratis-grpc/src/main/java/org/apache/ratis/grpc/server/GrpcStubPool.java b/ratis-grpc/src/main/java/org/apache/ratis/grpc/server/GrpcStubPool.java index c949707a42..55a1c6d354 100644 --- a/ratis-grpc/src/main/java/org/apache/ratis/grpc/server/GrpcStubPool.java +++ b/ratis-grpc/src/main/java/org/apache/ratis/grpc/server/GrpcStubPool.java @@ -24,8 +24,6 @@ import org.apache.ratis.thirdparty.io.grpc.stub.AbstractStub; import org.apache.ratis.thirdparty.io.netty.channel.ChannelOption; import org.apache.ratis.thirdparty.io.netty.channel.WriteBufferWaterMark; -import org.apache.ratis.thirdparty.io.netty.channel.nio.NioEventLoopGroup; -import org.apache.ratis.thirdparty.io.netty.channel.socket.nio.NioSocketChannel; import org.apache.ratis.thirdparty.io.netty.handler.ssl.SslContext; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -62,21 +60,17 @@ void release() { } private final List> pool; - private final NioEventLoopGroup elg; private final int size; GrpcStubPool(RaftPeer target, int n, Function stubFactory, SslContext sslContext) { - this(target, n, stubFactory, sslContext, Math.max(2, Runtime.getRuntime().availableProcessors() / 2), 16); + this(target, n, stubFactory, sslContext, 16); } GrpcStubPool(RaftPeer target, int n, Function stubFactory, SslContext sslContext, - int elgThreads, int maxInflightPerConn) { - this.elg = new NioEventLoopGroup(elgThreads); + int maxInflightPerConn) { ArrayList> tmp = new ArrayList<>(n); for (int i = 0; i < n; i++) { NettyChannelBuilder channelBuilder = NettyChannelBuilder.forTarget(target.getAddress()) - .eventLoopGroup(elg) - .channelType(NioSocketChannel.class) .keepAliveTime(30, TimeUnit.SECONDS) .keepAliveWithoutCalls(true) .idleTimeout(24, TimeUnit.HOURS) @@ -112,6 +106,5 @@ public void close() { for (PooledStub p : pool) { p.ch.shutdown(); } - elg.shutdownGracefully(); } } From 0438c9e8beb96feaa2a650335d2ea605cec523ea Mon Sep 17 00:00:00 2001 From: Tsz-Wo Nicholas Sze Date: Thu, 9 Oct 2025 15:44:04 -0700 Subject: [PATCH 210/397] RATIS-2344. Remove the unused mock from StateMachineShutdownTests. (#1299) --- .../impl/StateMachineShutdownTests.java | 20 +------------------ 1 file changed, 1 insertion(+), 19 deletions(-) diff --git a/ratis-server/src/test/java/org/apache/ratis/server/impl/StateMachineShutdownTests.java b/ratis-server/src/test/java/org/apache/ratis/server/impl/StateMachineShutdownTests.java index d97c5c4d50..fc00b70bab 100644 --- a/ratis-server/src/test/java/org/apache/ratis/server/impl/StateMachineShutdownTests.java +++ b/ratis-server/src/test/java/org/apache/ratis/server/impl/StateMachineShutdownTests.java @@ -28,12 +28,8 @@ import org.apache.ratis.statemachine.impl.SimpleStateMachine4Testing; import org.apache.ratis.statemachine.StateMachine; import org.apache.ratis.statemachine.TransactionContext; -import org.junit.jupiter.api.AfterEach; import org.junit.jupiter.api.Assertions; -import org.junit.jupiter.api.BeforeEach; import org.junit.jupiter.api.Test; -import org.mockito.MockedStatic; -import org.mockito.Mockito; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -51,7 +47,6 @@ public abstract class StateMachineShutdownTests extends BaseTest implements MiniRaftCluster.Factory.Get { public static Logger LOG = LoggerFactory.getLogger(StateMachineUpdater.class); - private static MockedStatic mocked; protected static class StateMachineWithConditionalWait extends SimpleStateMachine4Testing { boolean unblockAllTxns = false; @@ -124,19 +119,6 @@ public void unblockAllTxns() { } } - @BeforeEach - public void setup() { - mocked = Mockito.mockStatic(CompletableFuture.class, Mockito.CALLS_REAL_METHODS); - } - - @AfterEach - public void tearDownClass() { - if (mocked != null) { - mocked.close(); - } - - } - @Test public void testStateMachineShutdownWaitsForApplyTxn() throws Exception { final RaftProperties prop = getProperties(); @@ -195,7 +177,7 @@ public void testStateMachineShutdownWaitsForApplyTxn() throws Exception { // Now wait for the thread t.join(5000); - Assertions.assertEquals(logIndex, secondFollower.getInfo().getLastAppliedIndex()); + Assertions.assertTrue(logIndex <= secondFollower.getInfo().getLastAppliedIndex()); Assertions.assertEquals(3, StateMachineWithConditionalWait.numTxns.values().stream() .filter(val -> val.get() == 3).count()); From 1ea8fa4a4a87581850aec81437ba6c6f46822d62 Mon Sep 17 00:00:00 2001 From: Tsz-Wo Nicholas Sze Date: Thu, 9 Oct 2025 19:21:27 -0700 Subject: [PATCH 211/397] RATIS-2343. Replace string reasons with enum constants in log messages. (#1298) --- .../ratis/server/impl/RaftServerImpl.java | 19 +++++++------- .../ratis/server/impl/ServerProtoUtils.java | 6 ++++- .../apache/ratis/server/impl/ServerState.java | 9 ++++--- .../impl/SnapshotInstallationHandler.java | 26 +++++++++---------- 4 files changed, 33 insertions(+), 27 deletions(-) diff --git a/ratis-server/src/main/java/org/apache/ratis/server/impl/RaftServerImpl.java b/ratis-server/src/main/java/org/apache/ratis/server/impl/RaftServerImpl.java index 3c10e103b6..689bb8cefd 100644 --- a/ratis-server/src/main/java/org/apache/ratis/server/impl/RaftServerImpl.java +++ b/ratis-server/src/main/java/org/apache/ratis/server/impl/RaftServerImpl.java @@ -83,7 +83,7 @@ import org.apache.ratis.server.impl.RetryCacheImpl.CacheEntry; import org.apache.ratis.server.impl.ServerImplUtils.ConsecutiveIndices; import org.apache.ratis.server.impl.ServerImplUtils.NavigableIndices; -import org.apache.ratis.server.leader.LeaderState; +import org.apache.ratis.server.leader.LeaderState.StepDownReason; import org.apache.ratis.server.metrics.LeaderElectionMetrics; import org.apache.ratis.server.metrics.RaftServerMetricsImpl; import org.apache.ratis.server.protocol.RaftServerAsynchronousProtocol; @@ -139,6 +139,7 @@ import java.util.stream.Collectors; import java.util.stream.Stream; +import static org.apache.ratis.server.impl.LeaderElection.Result.NOT_IN_CONF; import static org.apache.ratis.server.impl.ServerImplUtils.assertEntries; import static org.apache.ratis.server.impl.ServerImplUtils.assertGroup; import static org.apache.ratis.server.impl.ServerImplUtils.effectiveCommitIndex; @@ -409,7 +410,7 @@ boolean start() throws IOException { startAsPeer(RaftPeerRole.LISTENER); } else { LOG.info("{}: start with initializing state, conf={}", getMemberId(), conf); - setRole(RaftPeerRole.FOLLOWER, "start"); + setRole(RaftPeerRole.FOLLOWER, NOT_IN_CONF); } jmxAdapter.registerMBean(); @@ -560,12 +561,12 @@ public void close() { try { ConcurrentUtils.shutdownAndWait(clientExecutor); } catch (Exception e) { - LOG.warn(getMemberId() + ": Failed to shutdown clientExecutor", e); + LOG.warn("{}: Failed to shutdown clientExecutor", getMemberId(), e); } try { ConcurrentUtils.shutdownAndWait(serverExecutor); } catch (Exception e) { - LOG.warn(getMemberId() + ": Failed to shutdown serverExecutor", e); + LOG.warn("{}: Failed to shutdown serverExecutor", getMemberId(), e); } closeFinishedLatch.countDown(); }); @@ -859,7 +860,7 @@ private CompletableFuture appendTransaction( cacheEntry.failWithReply(exceptionReply); // leader will step down here if (e.leaderShouldStepDown() && getInfo().isLeader()) { - leaderState.submitStepDownEvent(LeaderState.StepDownReason.STATE_MACHINE_EXCEPTION); + leaderState.submitStepDownEvent(StepDownReason.STATE_MACHINE_EXCEPTION); } return CompletableFuture.completedFuture(exceptionReply); } @@ -909,7 +910,7 @@ private RaftClientReply combineReplies(RaftClientReply reply, RaftClientReply wa } void stepDownOnJvmPause() { - role.getLeaderState().ifPresent(leader -> leader.submitStepDownEvent(LeaderState.StepDownReason.JVM_PAUSE)); + role.getLeaderState().ifPresent(leader -> leader.submitStepDownEvent(StepDownReason.JVM_PAUSE)); } private RaftClientRequest filterDataStreamRaftClientRequest(RaftClientRequest request) @@ -1529,7 +1530,7 @@ public CompletableFuture readIndexAsync(ReadIndexRequestPro static void logAppendEntries(boolean isHeartbeat, Supplier message) { if (isHeartbeat) { if (LOG.isTraceEnabled()) { - LOG.trace("HEARTBEAT: " + message.get()); + LOG.trace("HEARTBEAT: {}", message.get()); } } else { if (LOG.isDebugEnabled()) { @@ -1580,11 +1581,11 @@ leaderId, getMemberId(), currentTerm, followerCommit, state.getNextIndex(), AppendResult.NOT_LEADER, callId, RaftLog.INVALID_LOG_INDEX, isHeartbeat)); } try { - future = changeToFollowerAndPersistMetadata(leaderTerm, true, "appendEntries"); + future = changeToFollowerAndPersistMetadata(leaderTerm, true, Op.APPEND_ENTRIES); } catch (IOException e) { return JavaUtils.completeExceptionally(e); } - state.setLeader(leaderId, "appendEntries"); + state.setLeader(leaderId, Op.APPEND_ENTRIES); if (!proto.getInitializing() && lifeCycle.compareAndTransition(State.STARTING, State.RUNNING)) { role.startFollowerState(this, Op.APPEND_ENTRIES); diff --git a/ratis-server/src/main/java/org/apache/ratis/server/impl/ServerProtoUtils.java b/ratis-server/src/main/java/org/apache/ratis/server/impl/ServerProtoUtils.java index e6a29189a8..f491aaee26 100644 --- a/ratis-server/src/main/java/org/apache/ratis/server/impl/ServerProtoUtils.java +++ b/ratis-server/src/main/java/org/apache/ratis/server/impl/ServerProtoUtils.java @@ -178,8 +178,12 @@ static ServerRpcProto toServerRpcProto(RaftPeer peer, long delay) { // if no peer information return empty return ServerRpcProto.getDefaultInstance(); } + return toServerRpcProto(peer.getRaftPeerProto(), delay); + } + + static ServerRpcProto toServerRpcProto(RaftPeerProto peer, long delay) { return ServerRpcProto.newBuilder() - .setId(peer.getRaftPeerProto()) + .setId(peer) .setLastRpcElapsedTimeMs(delay) .build(); } diff --git a/ratis-server/src/main/java/org/apache/ratis/server/impl/ServerState.java b/ratis-server/src/main/java/org/apache/ratis/server/impl/ServerState.java index dc05c63e3e..05afc09756 100644 --- a/ratis-server/src/main/java/org/apache/ratis/server/impl/ServerState.java +++ b/ratis-server/src/main/java/org/apache/ratis/server/impl/ServerState.java @@ -24,6 +24,7 @@ import org.apache.ratis.server.RaftConfiguration; import org.apache.ratis.server.RaftServerConfigKeys; import org.apache.ratis.server.impl.LeaderElection.Phase; +import org.apache.ratis.server.protocol.RaftServerProtocol.Op; import org.apache.ratis.server.protocol.TermIndex; import org.apache.ratis.server.raftlog.LogProtoUtils; import org.apache.ratis.server.raftlog.RaftLog; @@ -257,7 +258,7 @@ RaftPeerId getVotedFor() { */ void grantVote(RaftPeerId candidateId) { votedFor = candidateId; - setLeader(null, "grantVote"); + setLeader(null, Op.REQUEST_VOTE); } void setLeader(RaftPeerId newLeaderId, Object op) { @@ -431,7 +432,7 @@ void close() { if (e instanceof InterruptedException) { Thread.currentThread().interrupt(); } - LOG.warn(getMemberId() + ": Failed to join " + getStateMachineUpdater(), e); + LOG.warn("{}: Failed to join {}", getMemberId(), getStateMachineUpdater(), e); } try { @@ -439,7 +440,7 @@ void close() { getLog().close(); } } catch (Throwable e) { - LOG.warn(getMemberId() + ": Failed to close raft log " + getLog(), e); + LOG.warn("{}: Failed to close raft log {}", getMemberId(), getLog(), e); } try { @@ -447,7 +448,7 @@ void close() { getStorage().close(); } } catch (Throwable e) { - LOG.warn(getMemberId() + ": Failed to close raft storage " + getStorage(), e); + LOG.warn("{}: Failed to close raft storage {}", getMemberId(), getStorage(), e); } } diff --git a/ratis-server/src/main/java/org/apache/ratis/server/impl/SnapshotInstallationHandler.java b/ratis-server/src/main/java/org/apache/ratis/server/impl/SnapshotInstallationHandler.java index 2a2e9e2b58..faefaaa38c 100644 --- a/ratis-server/src/main/java/org/apache/ratis/server/impl/SnapshotInstallationHandler.java +++ b/ratis-server/src/main/java/org/apache/ratis/server/impl/SnapshotInstallationHandler.java @@ -29,10 +29,10 @@ import org.apache.ratis.proto.RaftProtos.ServerRpcProto; import org.apache.ratis.protocol.RaftGroupId; import org.apache.ratis.protocol.RaftGroupMemberId; -import org.apache.ratis.protocol.RaftPeer; import org.apache.ratis.protocol.RaftPeerId; import org.apache.ratis.server.RaftServerConfigKeys; -import org.apache.ratis.server.protocol.RaftServerProtocol; +import org.apache.ratis.server.impl.FollowerState.UpdateType; +import org.apache.ratis.server.protocol.RaftServerProtocol.Op; import org.apache.ratis.server.protocol.TermIndex; import org.apache.ratis.server.raftlog.LogProtoUtils; import org.apache.ratis.server.util.ServerStringUtils; @@ -173,16 +173,16 @@ private CompletableFuture checkAndInstallSnapshot(Ins final long lastIncludedIndex = lastIncluded.getIndex(); final CompletableFuture future; synchronized (server) { - final boolean recognized = state.recognizeLeader(RaftServerProtocol.Op.INSTALL_SNAPSHOT, leaderId, leaderTerm); + final boolean recognized = state.recognizeLeader(Op.INSTALL_SNAPSHOT, leaderId, leaderTerm); currentTerm = state.getCurrentTerm(); if (!recognized) { return CompletableFuture.completedFuture(toInstallSnapshotReplyProto(leaderId, getMemberId(), currentTerm, snapshotChunkRequest.getRequestIndex(), InstallSnapshotResult.NOT_LEADER)); } - future = server.changeToFollowerAndPersistMetadata(leaderTerm, true, "installSnapshot"); - state.setLeader(leaderId, "installSnapshot"); + future = server.changeToFollowerAndPersistMetadata(leaderTerm, true, Op.INSTALL_SNAPSHOT); + state.setLeader(leaderId, Op.INSTALL_SNAPSHOT); - server.updateLastRpcTime(FollowerState.UpdateType.INSTALL_SNAPSHOT_START); + server.updateLastRpcTime(UpdateType.INSTALL_SNAPSHOT_START); long callId = chunk0CallId.get(); // 1. leaderTerm < currentTerm will never come here // 2. leaderTerm == currentTerm && callId == request.getCallId() @@ -229,7 +229,7 @@ private CompletableFuture checkAndInstallSnapshot(Ins chunk0CallId.set(-1); } } finally { - server.updateLastRpcTime(FollowerState.UpdateType.INSTALL_SNAPSHOT_COMPLETE); + server.updateLastRpcTime(UpdateType.INSTALL_SNAPSHOT_COMPLETE); } } if (snapshotChunkRequest.getDone()) { @@ -249,15 +249,15 @@ private CompletableFuture notifyStateMachineToInstall final long firstAvailableLogIndex = firstAvailableLogTermIndex.getIndex(); final CompletableFuture future; synchronized (server) { - final boolean recognized = state.recognizeLeader("notifyInstallSnapshot", leaderId, leaderTerm); + final boolean recognized = state.recognizeLeader(UpdateType.INSTALL_SNAPSHOT_NOTIFICATION, leaderId, leaderTerm); currentTerm = state.getCurrentTerm(); if (!recognized) { return CompletableFuture.completedFuture(toInstallSnapshotReplyProto(leaderId, getMemberId(), currentTerm, InstallSnapshotResult.NOT_LEADER)); } - future = server.changeToFollowerAndPersistMetadata(leaderTerm, true, "installSnapshot"); - state.setLeader(leaderId, "installSnapshot"); - server.updateLastRpcTime(FollowerState.UpdateType.INSTALL_SNAPSHOT_NOTIFICATION); + future = server.changeToFollowerAndPersistMetadata(leaderTerm, true, UpdateType.INSTALL_SNAPSHOT_NOTIFICATION); + state.setLeader(leaderId, UpdateType.INSTALL_SNAPSHOT_NOTIFICATION); + server.updateLastRpcTime(UpdateType.INSTALL_SNAPSHOT_NOTIFICATION); if (inProgressInstallSnapshotIndex.compareAndSet(INVALID_LOG_INDEX, firstAvailableLogIndex)) { LOG.info("{}: Received notification to install snapshot at index {}", getMemberId(), firstAvailableLogIndex); @@ -292,7 +292,7 @@ private CompletableFuture notifyStateMachineToInstall // For the cases where RaftConf is empty on newly started peer with empty peer list, // we retrieve leader info from installSnapShotRequestProto. final RoleInfoProto proto = leaderProto == null || server.getRaftConf().getPeer(state.getLeaderId()) != null? - server.getRoleInfoProto(): getRoleInfoProto(ProtoUtils.toRaftPeer(leaderProto)); + server.getRoleInfoProto(): getRoleInfoProto(leaderProto); // This is the first installSnapshot notify request for this term and // index. Notify the state machine to install the snapshot. LOG.info("{}: notifyInstallSnapshot: nextIndex is {} but the leader's first available index is {}.", @@ -386,7 +386,7 @@ private CompletableFuture notifyStateMachineToInstall } } - private RoleInfoProto getRoleInfoProto(RaftPeer leader) { + private RoleInfoProto getRoleInfoProto(RaftPeerProto leader) { final RoleInfo role = server.getRole(); final Optional fs = role.getFollowerState(); final ServerRpcProto leaderInfo = toServerRpcProto(leader, From 39bbb5db07b2e570dcf2d77dd1f232a3625da01e Mon Sep 17 00:00:00 2001 From: Xianming Lei <31424839+leixm@users.noreply.github.com> Date: Fri, 10 Oct 2025 19:43:49 +0800 Subject: [PATCH 212/397] RATIS-2329. NettyRpcProxy should support handling netty channel exception to prevent replication stuck (#1285) --- .../java/org/apache/ratis/util/IOUtils.java | 11 +++++--- .../org/apache/ratis/netty/NettyRpcProxy.java | 25 +++++++++++++++++-- .../java/org/apache/ratis/RaftAsyncTests.java | 14 ++++++----- 3 files changed, 39 insertions(+), 11 deletions(-) diff --git a/ratis-common/src/main/java/org/apache/ratis/util/IOUtils.java b/ratis-common/src/main/java/org/apache/ratis/util/IOUtils.java index f1fe6c35c8..8e91b3fb05 100644 --- a/ratis-common/src/main/java/org/apache/ratis/util/IOUtils.java +++ b/ratis-common/src/main/java/org/apache/ratis/util/IOUtils.java @@ -91,9 +91,14 @@ static T getFromFuture(CompletableFuture future, Supplier name, T } static boolean shouldReconnect(Throwable e) { - return ReflectionUtils.isInstance(e, - SocketException.class, SocketTimeoutException.class, ClosedChannelException.class, EOFException.class, - AlreadyClosedException.class); + for (; e != null; e = e.getCause()) { + if (ReflectionUtils.isInstance(e, + SocketException.class, SocketTimeoutException.class, ClosedChannelException.class, EOFException.class, + AlreadyClosedException.class, TimeoutIOException.class)) { + return true; + } + } + return false; } static void readFully(InputStream in, int buffSize) throws IOException { diff --git a/ratis-netty/src/main/java/org/apache/ratis/netty/NettyRpcProxy.java b/ratis-netty/src/main/java/org/apache/ratis/netty/NettyRpcProxy.java index 41269f76e3..2c49de0364 100644 --- a/ratis-netty/src/main/java/org/apache/ratis/netty/NettyRpcProxy.java +++ b/ratis-netty/src/main/java/org/apache/ratis/netty/NettyRpcProxy.java @@ -30,10 +30,13 @@ import org.apache.ratis.proto.RaftProtos.RaftRpcRequestProto; import org.apache.ratis.proto.netty.NettyProtos.RaftNettyServerReplyProto; import org.apache.ratis.proto.netty.NettyProtos.RaftNettyServerRequestProto; +import org.apache.ratis.protocol.exceptions.AlreadyClosedException; import org.apache.ratis.util.IOUtils; import org.apache.ratis.util.PeerProxyMap; import org.apache.ratis.util.ProtoUtils; import org.apache.ratis.util.TimeDuration; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; import java.io.Closeable; import java.io.IOException; @@ -47,6 +50,7 @@ import static org.apache.ratis.proto.netty.NettyProtos.RaftNettyServerReplyProto.RaftNettyServerReplyCase.EXCEPTIONREPLY; public class NettyRpcProxy implements Closeable { + public static final Logger LOG = LoggerFactory.getLogger(NettyRpcProxy.class); public static class PeerMap extends PeerProxyMap { private final EventLoopGroup group; @@ -121,6 +125,18 @@ protected void channelRead0(ChannelHandlerContext ctx, future.complete(proto); } } + + @Override + public void exceptionCaught(ChannelHandlerContext ctx, Throwable cause) { + client.close(); + failOutstandingRequests(new IOException("Caught an exception for the connection to " + peer, cause)); + } + + @Override + public void channelInactive(ChannelHandlerContext ctx) throws Exception { + failOutstandingRequests(new AlreadyClosedException("Channel to " + peer + " is inactive.")); + super.channelInactive(ctx); + } }; final ChannelInitializer initializer = new ChannelInitializer() { @@ -153,9 +169,14 @@ synchronized CompletableFuture pollReply() { @Override public synchronized void close() { client.close(); + failOutstandingRequests(new AlreadyClosedException("Closing connection to " + peer)); + } + + private synchronized void failOutstandingRequests(Throwable cause) { if (!replies.isEmpty()) { - final IOException e = new IOException("Connection to " + peer + " is closed."); - replies.stream().forEach(f -> f.completeExceptionally(e)); + LOG.warn("Still have {} requests outstanding from {} connection: {}", + replies.size(), peer, cause.toString()); + replies.forEach(f -> f.completeExceptionally(cause)); replies.clear(); } } diff --git a/ratis-server/src/test/java/org/apache/ratis/RaftAsyncTests.java b/ratis-server/src/test/java/org/apache/ratis/RaftAsyncTests.java index 4119bea71f..3c765d7171 100644 --- a/ratis-server/src/test/java/org/apache/ratis/RaftAsyncTests.java +++ b/ratis-server/src/test/java/org/apache/ratis/RaftAsyncTests.java @@ -181,12 +181,8 @@ void runTestRequestAsyncWithRetryFailure(boolean initialMessages, CLUSTER cluste // the second half still have retry time remaining. sleepTime.apply(t -> t*2).sleep(); - if (leader != null) { - cluster.restartServer(leader, false); - } else { - cluster.start(); - } - + // The client will try to reconnect, but the server is + // not started at this time and the retry will fail anyway. // all the calls should fail for ordering guarantee for(int i = 0; i < replies.size(); i++) { final CheckedRunnable getReply = replies.get(i)::get; @@ -203,6 +199,12 @@ void runTestRequestAsyncWithRetryFailure(boolean initialMessages, CLUSTER cluste testFailureCaseAsync("last-request", () -> client.async().send(new SimpleMessage("last")), AlreadyClosedException.class, RaftRetryFailureException.class); + + if (leader != null) { + cluster.restartServer(leader, false); + } else { + cluster.start(); + } } } From 38ccb22fd489fb03e83156af6114b4417def1943 Mon Sep 17 00:00:00 2001 From: Tsz-Wo Nicholas Sze Date: Tue, 14 Oct 2025 08:18:01 -0700 Subject: [PATCH 213/397] RATIS-2345. Leader stepDown could cause a deadlock. (#1300) --- .../org/apache/ratis/server/impl/LeaderStateImpl.java | 8 +++++--- .../org/apache/ratis/server/impl/PendingStepDown.java | 10 +++++++++- .../src/test/java/org/apache/ratis/RaftBasicTests.java | 6 +++--- 3 files changed, 17 insertions(+), 7 deletions(-) diff --git a/ratis-server/src/main/java/org/apache/ratis/server/impl/LeaderStateImpl.java b/ratis-server/src/main/java/org/apache/ratis/server/impl/LeaderStateImpl.java index 8358f063d2..0835802bd1 100644 --- a/ratis-server/src/main/java/org/apache/ratis/server/impl/LeaderStateImpl.java +++ b/ratis-server/src/main/java/org/apache/ratis/server/impl/LeaderStateImpl.java @@ -703,10 +703,12 @@ void submitStepDownEvent(long term, StepDownReason reason) { private void stepDown(long term, StepDownReason reason) { try { lease.getAndSetEnabled(false); - server.changeToFollowerAndPersistMetadata(term, false, reason).join(); + server.changeToFollowerAndPersistMetadata(term, false, reason) + .get(5, TimeUnit.SECONDS); pendingStepDown.complete(server::newSuccessReply); - } catch(IOException e) { - final String s = this + ": Failed to persist metadata for term " + term; + } catch(Exception e) { + pendingStepDown.completeExceptionally(e); + final String s = this + ": Failed to step down for term " + term; LOG.warn(s, e); // the failure should happen while changing the state to follower // thus the in-memory state should have been updated diff --git a/ratis-server/src/main/java/org/apache/ratis/server/impl/PendingStepDown.java b/ratis-server/src/main/java/org/apache/ratis/server/impl/PendingStepDown.java index b7bfde3f6e..c1e5cc5f53 100644 --- a/ratis-server/src/main/java/org/apache/ratis/server/impl/PendingStepDown.java +++ b/ratis-server/src/main/java/org/apache/ratis/server/impl/PendingStepDown.java @@ -56,8 +56,12 @@ void complete(Function newSuccessRep replyFuture.complete(newSuccessReply.apply(request)); } + void completeExceptionally(Exception e) { + replyFuture.completeExceptionally(e); + } + void timeout() { - replyFuture.completeExceptionally(new TimeoutIOException( + completeExceptionally(new TimeoutIOException( ": Failed to step down leader on " + leader + "request " + request.getTimeoutMs() + "ms")); } @@ -105,6 +109,10 @@ void complete(Function newSuccessRep pending.getAndSetNull().ifPresent(p -> p.complete(newSuccessReply)); } + void completeExceptionally(Exception e) { + pending.getAndSetNull().ifPresent(p -> p.completeExceptionally(e)); + } + void timeout() { pending.getAndSetNull().ifPresent(PendingRequest::timeout); } diff --git a/ratis-server/src/test/java/org/apache/ratis/RaftBasicTests.java b/ratis-server/src/test/java/org/apache/ratis/RaftBasicTests.java index 156cecf0b0..f1319cde7a 100644 --- a/ratis-server/src/test/java/org/apache/ratis/RaftBasicTests.java +++ b/ratis-server/src/test/java/org/apache/ratis/RaftBasicTests.java @@ -360,7 +360,7 @@ static void testWithLoad(final int numClients, final int numMessages, final Timer timer = new Timer(); timer.schedule(new TimerTask() { - private int previousLastStep = lastStep.get(); + private final AtomicInteger previousLastStep = new AtomicInteger(lastStep.get()); @Override public void run() { @@ -371,8 +371,8 @@ public void run() { JavaUtils.dumpAllThreads(s -> log.info(s)); final int last = lastStep.get(); - if (last != previousLastStep) { - previousLastStep = last; + if (last != previousLastStep.get()) { + previousLastStep.set(last); } else { final RaftServer.Division leader = cluster.getLeader(); log.info("NO PROGRESS at " + last + ", try to restart leader=" + leader); From 33413407c7d46449eba38c508290e4daf025f8a9 Mon Sep 17 00:00:00 2001 From: Tsz-Wo Nicholas Sze Date: Mon, 20 Oct 2025 07:56:54 -0700 Subject: [PATCH 214/397] RATIS-2346. SimpleStateMachineStorage prints too many log messages. (#1302) --- .../ratis/statemachine/impl/SimpleStateMachineStorage.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/ratis-server/src/main/java/org/apache/ratis/statemachine/impl/SimpleStateMachineStorage.java b/ratis-server/src/main/java/org/apache/ratis/statemachine/impl/SimpleStateMachineStorage.java index 2552c33c25..0ca6734a07 100644 --- a/ratis-server/src/main/java/org/apache/ratis/statemachine/impl/SimpleStateMachineStorage.java +++ b/ratis-server/src/main/java/org/apache/ratis/statemachine/impl/SimpleStateMachineStorage.java @@ -227,7 +227,7 @@ public SingleFileSnapshotInfo loadLatestSnapshot() { } try { final SingleFileSnapshotInfo latest = updateLatestSnapshot(findLatestSnapshot(dir.toPath())); - LOG.info("Latest snapshot is {} in {}", latest, dir); + LOG.debug("Latest snapshot is {} in {}", latest, dir); return latest; } catch (IOException e) { LOG.warn("Failed to updateLatestSnapshot from {}", dir, e); From 3120cb22a4bb6c709bf5346e0f0dc52e9977f2e6 Mon Sep 17 00:00:00 2001 From: "dependabot[bot]" <49699333+dependabot[bot]@users.noreply.github.com> Date: Mon, 27 Oct 2025 22:03:33 +0800 Subject: [PATCH 215/397] Bump org.bouncycastle:bcpkix-jdk18on from 1.78.1 to 1.79 in /ratis-test (#1301) Signed-off-by: dependabot[bot] Co-authored-by: dependabot[bot] <49699333+dependabot[bot]@users.noreply.github.com> --- ratis-test/pom.xml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/ratis-test/pom.xml b/ratis-test/pom.xml index efd87bedde..bc6401d455 100644 --- a/ratis-test/pom.xml +++ b/ratis-test/pom.xml @@ -24,7 +24,7 @@ Apache Ratis Test - 1.78.1 + 1.79 From 090b18c1d672d854ef2ba66d9df673222229baf4 Mon Sep 17 00:00:00 2001 From: Tsz-Wo Nicholas Sze Date: Sat, 1 Nov 2025 07:42:17 -0700 Subject: [PATCH 216/397] RATIS-2347. NPE in GrpcServerProtocolService. (#1304) --- .../src/main/java/org/apache/ratis/util/ProtoUtils.java | 6 ++++++ .../apache/ratis/grpc/server/GrpcServerProtocolService.java | 3 +-- .../ratis/server/impl/SnapshotInstallationHandler.java | 3 +-- 3 files changed, 8 insertions(+), 4 deletions(-) diff --git a/ratis-common/src/main/java/org/apache/ratis/util/ProtoUtils.java b/ratis-common/src/main/java/org/apache/ratis/util/ProtoUtils.java index cd22ebe6b4..161715bf80 100644 --- a/ratis-common/src/main/java/org/apache/ratis/util/ProtoUtils.java +++ b/ratis-common/src/main/java/org/apache/ratis/util/ProtoUtils.java @@ -35,6 +35,8 @@ import org.apache.ratis.protocol.RaftPeer; import org.apache.ratis.protocol.RaftPeerId; import org.apache.ratis.thirdparty.com.google.protobuf.ByteString; +import org.apache.ratis.thirdparty.com.google.protobuf.MessageOrBuilder; +import org.apache.ratis.thirdparty.com.google.protobuf.TextFormat; import java.io.IOException; import java.io.ObjectOutputStream; @@ -250,4 +252,8 @@ static String toString(RaftRpcReplyProto proto) { + "#" + proto.getCallId() + ":" + (proto.getSuccess()? "OK": "FAIL"); } + + static String shortDebugString(MESSAGE message) { + return message != null ? TextFormat.shortDebugString(message) : "null"; + } } diff --git a/ratis-grpc/src/main/java/org/apache/ratis/grpc/server/GrpcServerProtocolService.java b/ratis-grpc/src/main/java/org/apache/ratis/grpc/server/GrpcServerProtocolService.java index b123c44a77..a13e74b89d 100644 --- a/ratis-grpc/src/main/java/org/apache/ratis/grpc/server/GrpcServerProtocolService.java +++ b/ratis-grpc/src/main/java/org/apache/ratis/grpc/server/GrpcServerProtocolService.java @@ -25,7 +25,6 @@ import org.apache.ratis.server.protocol.RaftServerProtocol; import org.apache.ratis.server.util.ServerStringUtils; import org.apache.ratis.thirdparty.com.google.protobuf.MessageOrBuilder; -import org.apache.ratis.thirdparty.com.google.protobuf.TextFormat; import org.apache.ratis.thirdparty.io.grpc.Status; import org.apache.ratis.thirdparty.io.grpc.StatusRuntimeException; import org.apache.ratis.thirdparty.io.grpc.stub.StreamObserver; @@ -176,7 +175,7 @@ public void onCompleted() { requestFuture.get().thenAccept(reply -> { BatchLogger.print(BatchLogKey.COMPLETED_REPLY, getName(), suffix -> LOG.info("{}: Completed {}, lastReply: {} {}", - getId(), op, TextFormat.shortDebugString(reply), suffix)); + getId(), op, ProtoUtils.shortDebugString(reply), suffix)); responseObserver.onCompleted(); }); } diff --git a/ratis-server/src/main/java/org/apache/ratis/server/impl/SnapshotInstallationHandler.java b/ratis-server/src/main/java/org/apache/ratis/server/impl/SnapshotInstallationHandler.java index faefaaa38c..eac690feb2 100644 --- a/ratis-server/src/main/java/org/apache/ratis/server/impl/SnapshotInstallationHandler.java +++ b/ratis-server/src/main/java/org/apache/ratis/server/impl/SnapshotInstallationHandler.java @@ -36,7 +36,6 @@ import org.apache.ratis.server.protocol.TermIndex; import org.apache.ratis.server.raftlog.LogProtoUtils; import org.apache.ratis.server.util.ServerStringUtils; -import org.apache.ratis.thirdparty.com.google.protobuf.TextFormat; import org.apache.ratis.util.BatchLogger; import org.apache.ratis.util.CodeInjectionForTesting; import org.apache.ratis.util.LifeCycle; @@ -145,7 +144,7 @@ private InstallSnapshotReplyProto installSnapshotImpl(InstallSnapshotRequestProt final LogEntryProto proto = request.getLastRaftConfigurationLogEntryProto(); state.truncate(proto.getIndex()); if (!state.getRaftConf().equals(LogProtoUtils.toRaftConfiguration(proto))) { - LOG.info("{}: set new configuration {} from snapshot", getMemberId(), TextFormat.shortDebugString(proto)); + LOG.info("{}: set new configuration {} from snapshot", getMemberId(), ProtoUtils.shortDebugString(proto)); state.setRaftConf(proto); state.writeRaftConfiguration(proto); server.getStateMachine().event().notifyConfigurationChanged( From 336f9b407bcc63f93ba9b45b63cd86f319ee95bc Mon Sep 17 00:00:00 2001 From: Tsz-Wo Nicholas Sze Date: Fri, 31 Oct 2025 10:58:03 -0700 Subject: [PATCH 217/397] Revert "RATIS-2325. Create GrpcStubPool for GrpcServerProtocolClient (#1283)" This reverts commit d11604cf4d920a47ae3ebcd66c2c088ffe234264. --- .../org/apache/ratis/grpc/GrpcConfigKeys.java | 9 -- .../grpc/server/GrpcServerProtocolClient.java | 37 +----- .../ratis/grpc/server/GrpcServicesImpl.java | 4 +- .../ratis/grpc/server/GrpcStubPool.java | 110 ------------------ 4 files changed, 4 insertions(+), 156 deletions(-) delete mode 100644 ratis-grpc/src/main/java/org/apache/ratis/grpc/server/GrpcStubPool.java diff --git a/ratis-grpc/src/main/java/org/apache/ratis/grpc/GrpcConfigKeys.java b/ratis-grpc/src/main/java/org/apache/ratis/grpc/GrpcConfigKeys.java index cef62779dd..f21a9b99f1 100644 --- a/ratis-grpc/src/main/java/org/apache/ratis/grpc/GrpcConfigKeys.java +++ b/ratis-grpc/src/main/java/org/apache/ratis/grpc/GrpcConfigKeys.java @@ -282,15 +282,6 @@ static GrpcTlsConfig tlsConf(Parameters parameters) { static void setTlsConf(Parameters parameters, GrpcTlsConfig conf) { parameters.put(TLS_CONF_PARAMETER, conf, TLS_CONF_CLASS); } - - String STUB_POOL_SIZE_KEY = PREFIX + ".stub.pool.size"; - int STUB_POOL_SIZE_DEFAULT = 10; - static int stubPoolSize(RaftProperties properties) { - return get(properties::getInt, STUB_POOL_SIZE_KEY, STUB_POOL_SIZE_DEFAULT, getDefaultLog()); - } - static void setStubPoolSize(RaftProperties properties, int size) { - setInt(properties::setInt, STUB_POOL_SIZE_KEY, size); - } } String MESSAGE_SIZE_MAX_KEY = PREFIX + ".message.size.max"; diff --git a/ratis-grpc/src/main/java/org/apache/ratis/grpc/server/GrpcServerProtocolClient.java b/ratis-grpc/src/main/java/org/apache/ratis/grpc/server/GrpcServerProtocolClient.java index a0a17dc9ff..1e40a75ada 100644 --- a/ratis-grpc/src/main/java/org/apache/ratis/grpc/server/GrpcServerProtocolClient.java +++ b/ratis-grpc/src/main/java/org/apache/ratis/grpc/server/GrpcServerProtocolClient.java @@ -45,7 +45,6 @@ class GrpcServerProtocolClient implements Closeable { // Common channel private final ManagedChannel channel; - private final GrpcStubPool pool; // Channel and stub for heartbeat private ManagedChannel hbChannel; private RaftServerProtocolServiceStub hbAsyncStub; @@ -58,7 +57,7 @@ class GrpcServerProtocolClient implements Closeable { //visible for using in log / error messages AND to use in instrumented tests private final RaftPeerId raftPeerId; - GrpcServerProtocolClient(RaftPeer target, int connections, int flowControlWindow, + GrpcServerProtocolClient(RaftPeer target, int flowControlWindow, TimeDuration requestTimeout, SslContext sslContext, boolean separateHBChannel) { raftPeerId = target.getId(); LOG.info("Build channel for {}", target); @@ -71,7 +70,6 @@ class GrpcServerProtocolClient implements Closeable { hbAsyncStub = RaftServerProtocolServiceGrpc.newStub(hbChannel); } requestTimeoutDuration = requestTimeout; - this.pool = new GrpcStubPool<>(target, connections, RaftServerProtocolServiceGrpc::newStub, sslContext); } private ManagedChannel buildChannel(RaftPeer target, int flowControlWindow, SslContext sslContext) { @@ -96,7 +94,6 @@ public void close() { GrpcUtil.shutdownManagedChannel(hbChannel); } GrpcUtil.shutdownManagedChannel(channel); - pool.close(); } public RequestVoteReplyProto requestVote(RequestVoteRequestProto request) { @@ -115,36 +112,8 @@ public StartLeaderElectionReplyProto startLeaderElection(StartLeaderElectionRequ } void readIndex(ReadIndexRequestProto request, StreamObserver s) { - GrpcStubPool.PooledStub p; - try { - p = pool.acquire(); - } catch (InterruptedException e) { - Thread.currentThread().interrupt(); - s.onError(e); return; - } - p.getStub().withDeadlineAfter(requestTimeoutDuration.getDuration(), requestTimeoutDuration.getUnit()) - .readIndex(request, new StreamObserver() { - @Override - public void onNext(ReadIndexReplyProto v) { - s.onNext(v); - } - @Override - public void onError(Throwable t) { - try { - s.onError(t); - } finally { - p.release(); - } - } - @Override - public void onCompleted() { - try { - s.onCompleted(); - } finally { - p.release(); - } - } - }); + asyncStub.withDeadlineAfter(requestTimeoutDuration.getDuration(), requestTimeoutDuration.getUnit()) + .readIndex(request, s); } CallStreamObserver appendEntries( diff --git a/ratis-grpc/src/main/java/org/apache/ratis/grpc/server/GrpcServicesImpl.java b/ratis-grpc/src/main/java/org/apache/ratis/grpc/server/GrpcServicesImpl.java index b1af0960dc..8200aa3ef7 100644 --- a/ratis-grpc/src/main/java/org/apache/ratis/grpc/server/GrpcServicesImpl.java +++ b/ratis-grpc/src/main/java/org/apache/ratis/grpc/server/GrpcServicesImpl.java @@ -108,7 +108,6 @@ public static final class Builder { private int serverPort; private SslContext serverSslContextForServer; private SslContext serverSslContextForClient; - private int serverStubPoolSize; private SizeInBytes messageSizeMax; private SizeInBytes flowControlWindow; @@ -131,7 +130,6 @@ public Builder setServer(RaftServer raftServer) { this.flowControlWindow = GrpcConfigKeys.flowControlWindow(properties, LOG::info); this.requestTimeoutDuration = RaftServerConfigKeys.Rpc.requestTimeout(properties); this.separateHeartbeatChannel = GrpcConfigKeys.Server.heartbeatChannel(properties); - this.serverStubPoolSize = GrpcConfigKeys.Server.stubPoolSize(properties); final SizeInBytes appenderBufferSize = RaftServerConfigKeys.Log.Appender.bufferByteLimit(properties); final SizeInBytes gap = SizeInBytes.ONE_MB; @@ -152,7 +150,7 @@ public Builder setCustomizer(Customizer customizer) { } private GrpcServerProtocolClient newGrpcServerProtocolClient(RaftPeer target) { - return new GrpcServerProtocolClient(target, serverStubPoolSize, flowControlWindow.getSizeInt(), + return new GrpcServerProtocolClient(target, flowControlWindow.getSizeInt(), requestTimeoutDuration, serverSslContextForClient, separateHeartbeatChannel); } diff --git a/ratis-grpc/src/main/java/org/apache/ratis/grpc/server/GrpcStubPool.java b/ratis-grpc/src/main/java/org/apache/ratis/grpc/server/GrpcStubPool.java deleted file mode 100644 index 55a1c6d354..0000000000 --- a/ratis-grpc/src/main/java/org/apache/ratis/grpc/server/GrpcStubPool.java +++ /dev/null @@ -1,110 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ -package org.apache.ratis.grpc.server; - -import org.apache.ratis.protocol.RaftPeer; -import org.apache.ratis.thirdparty.io.grpc.ManagedChannel; -import org.apache.ratis.thirdparty.io.grpc.netty.NegotiationType; -import org.apache.ratis.thirdparty.io.grpc.netty.NettyChannelBuilder; -import org.apache.ratis.thirdparty.io.grpc.stub.AbstractStub; -import org.apache.ratis.thirdparty.io.netty.channel.ChannelOption; -import org.apache.ratis.thirdparty.io.netty.channel.WriteBufferWaterMark; -import org.apache.ratis.thirdparty.io.netty.handler.ssl.SslContext; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -import java.util.ArrayList; -import java.util.Collections; -import java.util.List; -import java.util.concurrent.Semaphore; -import java.util.concurrent.ThreadLocalRandom; -import java.util.concurrent.TimeUnit; -import java.util.function.Function; - -final class GrpcStubPool> { - public static final Logger LOG = LoggerFactory.getLogger(GrpcStubPool.class); - - static final class PooledStub> { - private final ManagedChannel ch; - private final S stub; - private final Semaphore permits; - - PooledStub(ManagedChannel ch, S stub, int maxInflight) { - this.ch = ch; - this.stub = stub; - this.permits = new Semaphore(maxInflight); - } - - S getStub() { - return stub; - } - - void release() { - permits.release(); - } - } - - private final List> pool; - private final int size; - - GrpcStubPool(RaftPeer target, int n, Function stubFactory, SslContext sslContext) { - this(target, n, stubFactory, sslContext, 16); - } - - GrpcStubPool(RaftPeer target, int n, Function stubFactory, SslContext sslContext, - int maxInflightPerConn) { - ArrayList> tmp = new ArrayList<>(n); - for (int i = 0; i < n; i++) { - NettyChannelBuilder channelBuilder = NettyChannelBuilder.forTarget(target.getAddress()) - .keepAliveTime(30, TimeUnit.SECONDS) - .keepAliveWithoutCalls(true) - .idleTimeout(24, TimeUnit.HOURS) - .withOption(ChannelOption.WRITE_BUFFER_WATER_MARK, new WriteBufferWaterMark(64 << 10, 128 << 10)); - if (sslContext != null) { - LOG.debug("Setting TLS for {}", target.getAddress()); - channelBuilder.useTransportSecurity().sslContext(sslContext); - } else { - channelBuilder.negotiationType(NegotiationType.PLAINTEXT); - } - ManagedChannel ch = channelBuilder.build(); - tmp.add(new PooledStub<>(ch, stubFactory.apply(ch), maxInflightPerConn)); - ch.getState(true); - } - this.pool = Collections.unmodifiableList(tmp); - this.size = n; - } - - PooledStub acquire() throws InterruptedException { - final int start = ThreadLocalRandom.current().nextInt(size); - for (int k = 0; k < size; k++) { - PooledStub p = pool.get((start + k) % size); - if (p.permits.tryAcquire()) { - return p; - } - } - final PooledStub p = pool.get(start); - p.permits.acquire(); - return p; - } - - public void close() { - for (PooledStub p : pool) { - p.ch.shutdown(); - } - } -} From 98de25587c645c55357c2f27d91ef6b5c3d5e88d Mon Sep 17 00:00:00 2001 From: Tsz-Wo Nicholas Sze Date: Sun, 2 Nov 2025 18:48:14 -0800 Subject: [PATCH 218/397] RATIS-2348. GrpcLogAppender may print a lot of messages in an error condition. (#1305) --- .../ratis/grpc/server/GrpcLogAppender.java | 52 +++++++++++-------- 1 file changed, 30 insertions(+), 22 deletions(-) diff --git a/ratis-grpc/src/main/java/org/apache/ratis/grpc/server/GrpcLogAppender.java b/ratis-grpc/src/main/java/org/apache/ratis/grpc/server/GrpcLogAppender.java index 1622c5df9f..9ce45d1abb 100644 --- a/ratis-grpc/src/main/java/org/apache/ratis/grpc/server/GrpcLogAppender.java +++ b/ratis-grpc/src/main/java/org/apache/ratis/grpc/server/GrpcLogAppender.java @@ -73,7 +73,11 @@ public class GrpcLogAppender extends LogAppenderBase { private enum BatchLogKey implements BatchLogger.Key { RESET_CLIENT, INCONSISTENCY_REPLY, - APPEND_LOG_RESPONSE_HANDLER_ON_ERROR + APPEND_LOG_RESPONSE_HANDLER_ON_ERROR, + INSTALL_SNAPSHOT_NOTIFY, + INSTALL_SNAPSHOT_REPLY, + INSTALL_SNAPSHOT_IN_PROGRESS, + SNAPSHOT_UNAVAILABLE } public static final int INSTALL_SNAPSHOT_NOTIFICATION_INDEX = 0; @@ -234,7 +238,7 @@ private void resetClient(AppendEntriesRequest request, Event event) { } getFollower().computeNextIndex(getNextIndexForError(nextIndex)); } catch (IOException ie) { - LOG.warn(this + ": Failed to getClient for " + getFollowerId(), ie); + LOG.warn("{}: Failed to resetClient for {}", this, getFollowerId(), ie); } } @@ -497,8 +501,8 @@ public void onNext(AppendEntriesReplyProto reply) { try { onNextImpl(request, reply); } catch(Exception t) { - LOG.error("Failed onNext request=" + request - + ", reply=" + ServerStringUtils.toAppendEntriesReplyString(reply), t); + LOG.error("Failed onNext(reply), request={}, reply={}", + request, ServerStringUtils.toAppendEntriesReplyString(reply), t); } } @@ -573,8 +577,8 @@ private void updateNextIndex(long replyNextIndex) { } private class InstallSnapshotResponseHandler implements StreamObserver { - private final String name = getFollower().getName() + "-" + JavaUtils.getClassSimpleName(getClass()); - private final Queue pending; + private final String name; + private final Queue pending = new LinkedList<>(); private final CompletableFuture done = new CompletableFuture<>(); private final boolean isNotificationOnly; @@ -583,8 +587,8 @@ private class InstallSnapshotResponseHandler implements StreamObserver(); this.isNotificationOnly = notifyOnly; + this.name = getFollower().getName() + "-InstallSnapshot" + (isNotificationOnly ? "Notification" : ""); } void addPending(InstallSnapshotRequestProto request) { @@ -626,8 +630,8 @@ void onFollowerCatchup(long followerSnapshotIndex) { final long leaderStartIndex = getRaftLog().getStartIndex(); final long followerNextIndex = followerSnapshotIndex + 1; if (followerNextIndex >= leaderStartIndex) { - LOG.info("{}: Follower can catch up leader after install the snapshot, as leader's start index is {}", - this, followerNextIndex); + LOG.info("{}: follower nextIndex = {} >= leader startIndex = {}", + this, followerNextIndex, leaderStartIndex); notifyInstallSnapshotFinished(InstallSnapshotResult.SUCCESS, followerSnapshotIndex); } } @@ -659,10 +663,10 @@ boolean hasAllResponse() { @Override public void onNext(InstallSnapshotReplyProto reply) { - if (LOG.isInfoEnabled()) { - LOG.info("{}: received {} reply {}", this, replyState.isFirstReplyReceived()? "a" : "the first", - ServerStringUtils.toInstallSnapshotReplyString(reply)); - } + BatchLogger.print(BatchLogKey.INSTALL_SNAPSHOT_REPLY, name, + suffix -> LOG.info("{}: received {} reply {} {}", this, + replyState.isFirstReplyReceived() ? "a" : "the first", + ServerStringUtils.toInstallSnapshotReplyString(reply), suffix)); // update the last rpc time getFollower().updateLastRpcResponseTime(); @@ -671,12 +675,13 @@ public void onNext(InstallSnapshotReplyProto reply) { final long followerSnapshotIndex; switch (reply.getResult()) { case SUCCESS: - LOG.info("{}: Completed InstallSnapshot. Reply: {}", this, reply); + LOG.info("{}: Completed", this); getFollower().setAttemptedToInstallSnapshot(); removePending(reply); break; case IN_PROGRESS: - LOG.info("{}: InstallSnapshot in progress.", this); + BatchLogger.print(BatchLogKey.INSTALL_SNAPSHOT_IN_PROGRESS, name, + suffix -> LOG.info("{}: in progress, {}", this, suffix)); removePending(reply); break; case ALREADY_INSTALLED: @@ -692,7 +697,7 @@ public void onNext(InstallSnapshotReplyProto reply) { onFollowerTerm(reply.getTerm()); break; case CONF_MISMATCH: - LOG.error("{}: Configuration Mismatch ({}): Leader {} has it set to {} but follower {} has it set to {}", + LOG.error("{}: CONF_MISMATCH ({}): Leader {} has it set to {} but follower {} has it set to {}", this, RaftServerConfigKeys.Log.Appender.INSTALL_SNAPSHOT_ENABLED_KEY, getServer().getId(), installSnapshotEnabled, getFollowerId(), !installSnapshotEnabled); break; @@ -707,17 +712,19 @@ public void onNext(InstallSnapshotReplyProto reply) { removePending(reply); break; case SNAPSHOT_UNAVAILABLE: - LOG.info("{}: Follower could not install snapshot as it is not available.", this); + BatchLogger.print(BatchLogKey.SNAPSHOT_UNAVAILABLE, name, + suffix -> LOG.info("{}: Follower failed since the snapshot is unavailable {}", this, suffix)); getFollower().setAttemptedToInstallSnapshot(); notifyInstallSnapshotFinished(InstallSnapshotResult.SNAPSHOT_UNAVAILABLE, RaftLog.INVALID_LOG_INDEX); removePending(reply); break; case UNRECOGNIZED: - LOG.error("Unrecognized the reply result {}: Leader is {}, follower is {}", - reply.getResult(), getServer().getId(), getFollowerId()); + LOG.error("{}: Reply result {}, {}", + name, reply.getResult(), ServerStringUtils.toInstallSnapshotReplyString(reply)); break; case SNAPSHOT_EXPIRED: - LOG.warn("{}: Follower could not install snapshot as it is expired.", this); + LOG.warn("{}: Follower failed since the request expired, {}", + name, ServerStringUtils.toInstallSnapshotReplyString(reply)); default: break; } @@ -796,8 +803,9 @@ private void installSnapshot(SnapshotInfo snapshot) { * @param firstAvailable the first available log's index on the Leader */ private void notifyInstallSnapshot(TermIndex firstAvailable) { - LOG.info("{}: notifyInstallSnapshot with firstAvailable={}, followerNextIndex={}", - this, firstAvailable, getFollower().getNextIndex()); + BatchLogger.print(BatchLogKey.INSTALL_SNAPSHOT_NOTIFY, getFollower().getName(), + suffix -> LOG.info("{}: notifyInstallSnapshot with firstAvailable={}, followerNextIndex={} {}", + this, firstAvailable, getFollower().getNextIndex(), suffix)); final InstallSnapshotResponseHandler responseHandler = new InstallSnapshotResponseHandler(true); StreamObserver snapshotRequestObserver = null; From 171954c265d21643c0fa541a3d2822fcf7a5669e Mon Sep 17 00:00:00 2001 From: Tsz-Wo Nicholas Sze Date: Sun, 2 Nov 2025 18:51:50 -0800 Subject: [PATCH 219/397] ./mvnw versions:set -DnewVersion=3.3.0-SNAPSHOT --- pom.xml | 4 ++-- ratis-assembly/pom.xml | 2 +- ratis-client/pom.xml | 2 +- ratis-common/pom.xml | 2 +- ratis-docs/pom.xml | 2 +- ratis-examples/pom.xml | 2 +- ratis-experiments/pom.xml | 2 +- ratis-grpc/pom.xml | 2 +- ratis-metrics-api/pom.xml | 2 +- ratis-metrics-default/pom.xml | 2 +- ratis-metrics-dropwizard3/pom.xml | 2 +- ratis-netty/pom.xml | 2 +- ratis-proto/pom.xml | 2 +- ratis-resource-bundle/pom.xml | 2 +- ratis-server-api/pom.xml | 2 +- ratis-server/pom.xml | 2 +- ratis-shell/pom.xml | 2 +- ratis-test/pom.xml | 2 +- ratis-tools/pom.xml | 2 +- 19 files changed, 20 insertions(+), 20 deletions(-) diff --git a/pom.xml b/pom.xml index 9ea7379770..72cd5a646e 100644 --- a/pom.xml +++ b/pom.xml @@ -24,7 +24,7 @@ ratis org.apache.ratis - 3.1.3 + 3.3.0-SNAPSHOT Apache Ratis pom @@ -157,7 +157,7 @@ - 2025-01-10T02:06:09Z + 2025-11-03T02:50:35Z UTF-8 UTF-8 diff --git a/ratis-assembly/pom.xml b/ratis-assembly/pom.xml index e7b0f3c23b..0558693039 100644 --- a/ratis-assembly/pom.xml +++ b/ratis-assembly/pom.xml @@ -17,7 +17,7 @@ ratis org.apache.ratis - 3.1.3 + 3.3.0-SNAPSHOT ratis-assembly diff --git a/ratis-client/pom.xml b/ratis-client/pom.xml index b73bbebbc7..89cd67e7e6 100644 --- a/ratis-client/pom.xml +++ b/ratis-client/pom.xml @@ -17,7 +17,7 @@ ratis org.apache.ratis - 3.1.3 + 3.3.0-SNAPSHOT ratis-client diff --git a/ratis-common/pom.xml b/ratis-common/pom.xml index b040f87964..f60cc0240a 100644 --- a/ratis-common/pom.xml +++ b/ratis-common/pom.xml @@ -17,7 +17,7 @@ ratis org.apache.ratis - 3.1.3 + 3.3.0-SNAPSHOT ratis-common diff --git a/ratis-docs/pom.xml b/ratis-docs/pom.xml index 91b92f2f86..894b6a52e4 100644 --- a/ratis-docs/pom.xml +++ b/ratis-docs/pom.xml @@ -20,7 +20,7 @@ https://maven.apache.org/xsd/maven-4.0.0.xsd"> ratis org.apache.ratis - 3.1.3 + 3.3.0-SNAPSHOT ratis-docs diff --git a/ratis-examples/pom.xml b/ratis-examples/pom.xml index d13e68df5a..8464f93c03 100644 --- a/ratis-examples/pom.xml +++ b/ratis-examples/pom.xml @@ -17,7 +17,7 @@ ratis org.apache.ratis - 3.1.3 + 3.3.0-SNAPSHOT ratis-examples diff --git a/ratis-experiments/pom.xml b/ratis-experiments/pom.xml index 5dc169c360..ed13c993da 100644 --- a/ratis-experiments/pom.xml +++ b/ratis-experiments/pom.xml @@ -17,7 +17,7 @@ ratis org.apache.ratis - 3.1.3 + 3.3.0-SNAPSHOT ratis-experiments diff --git a/ratis-grpc/pom.xml b/ratis-grpc/pom.xml index 1ae667b185..ba3110178d 100644 --- a/ratis-grpc/pom.xml +++ b/ratis-grpc/pom.xml @@ -17,7 +17,7 @@ ratis org.apache.ratis - 3.1.3 + 3.3.0-SNAPSHOT ratis-grpc diff --git a/ratis-metrics-api/pom.xml b/ratis-metrics-api/pom.xml index b101e13f23..16c368084a 100644 --- a/ratis-metrics-api/pom.xml +++ b/ratis-metrics-api/pom.xml @@ -18,7 +18,7 @@ ratis org.apache.ratis - 3.1.3 + 3.3.0-SNAPSHOT ratis-metrics-api diff --git a/ratis-metrics-default/pom.xml b/ratis-metrics-default/pom.xml index cbdb2dfc66..ada045e961 100644 --- a/ratis-metrics-default/pom.xml +++ b/ratis-metrics-default/pom.xml @@ -18,7 +18,7 @@ ratis org.apache.ratis - 3.1.3 + 3.3.0-SNAPSHOT ratis-metrics-default diff --git a/ratis-metrics-dropwizard3/pom.xml b/ratis-metrics-dropwizard3/pom.xml index 45707d8d58..f50ee14a30 100644 --- a/ratis-metrics-dropwizard3/pom.xml +++ b/ratis-metrics-dropwizard3/pom.xml @@ -18,7 +18,7 @@ ratis org.apache.ratis - 3.1.3 + 3.3.0-SNAPSHOT ratis-metrics-dropwizard3 diff --git a/ratis-netty/pom.xml b/ratis-netty/pom.xml index 29480d9987..f9d34fffc5 100644 --- a/ratis-netty/pom.xml +++ b/ratis-netty/pom.xml @@ -17,7 +17,7 @@ ratis org.apache.ratis - 3.1.3 + 3.3.0-SNAPSHOT ratis-netty diff --git a/ratis-proto/pom.xml b/ratis-proto/pom.xml index 771187b583..e189685692 100644 --- a/ratis-proto/pom.xml +++ b/ratis-proto/pom.xml @@ -17,7 +17,7 @@ ratis org.apache.ratis - 3.1.3 + 3.3.0-SNAPSHOT ratis-proto diff --git a/ratis-resource-bundle/pom.xml b/ratis-resource-bundle/pom.xml index 5a3d1ff876..edecf1b633 100644 --- a/ratis-resource-bundle/pom.xml +++ b/ratis-resource-bundle/pom.xml @@ -23,7 +23,7 @@ ratis org.apache.ratis - 3.1.3 + 3.3.0-SNAPSHOT .. diff --git a/ratis-server-api/pom.xml b/ratis-server-api/pom.xml index 889277c146..0dfb3cc1d1 100644 --- a/ratis-server-api/pom.xml +++ b/ratis-server-api/pom.xml @@ -17,7 +17,7 @@ ratis org.apache.ratis - 3.1.3 + 3.3.0-SNAPSHOT ratis-server-api diff --git a/ratis-server/pom.xml b/ratis-server/pom.xml index 7a4f929951..f700ac6c5d 100644 --- a/ratis-server/pom.xml +++ b/ratis-server/pom.xml @@ -17,7 +17,7 @@ ratis org.apache.ratis - 3.1.3 + 3.3.0-SNAPSHOT ratis-server diff --git a/ratis-shell/pom.xml b/ratis-shell/pom.xml index ab2a3da2ad..32361a46c3 100644 --- a/ratis-shell/pom.xml +++ b/ratis-shell/pom.xml @@ -17,7 +17,7 @@ ratis org.apache.ratis - 3.1.3 + 3.3.0-SNAPSHOT ratis-shell diff --git a/ratis-test/pom.xml b/ratis-test/pom.xml index bc6401d455..854c2b6068 100644 --- a/ratis-test/pom.xml +++ b/ratis-test/pom.xml @@ -17,7 +17,7 @@ ratis org.apache.ratis - 3.1.3 + 3.3.0-SNAPSHOT ratis-test diff --git a/ratis-tools/pom.xml b/ratis-tools/pom.xml index f1d0c33d9b..2d81f2467a 100644 --- a/ratis-tools/pom.xml +++ b/ratis-tools/pom.xml @@ -17,7 +17,7 @@ ratis org.apache.ratis - 3.1.3 + 3.3.0-SNAPSHOT ratis-tools From 3853d05def85ecf52f8182ef716bc70bbd0307f5 Mon Sep 17 00:00:00 2001 From: Xianming Lei <31424839+leixm@users.noreply.github.com> Date: Mon, 3 Nov 2025 12:51:22 +0800 Subject: [PATCH 220/397] RATIS-2349. NettyClient#writeAndFlush should support throwing AlreadyClosedException (#1303) --- .../org/apache/ratis/netty/NettyClient.java | 17 ++++++++++++++--- .../org/apache/ratis/netty/NettyRpcProxy.java | 8 ++++++-- 2 files changed, 20 insertions(+), 5 deletions(-) diff --git a/ratis-netty/src/main/java/org/apache/ratis/netty/NettyClient.java b/ratis-netty/src/main/java/org/apache/ratis/netty/NettyClient.java index efea5fd0f8..a42ddaca83 100644 --- a/ratis-netty/src/main/java/org/apache/ratis/netty/NettyClient.java +++ b/ratis-netty/src/main/java/org/apache/ratis/netty/NettyClient.java @@ -17,6 +17,7 @@ */ package org.apache.ratis.netty; +import org.apache.ratis.protocol.exceptions.AlreadyClosedException; import org.apache.ratis.thirdparty.io.netty.bootstrap.Bootstrap; import org.apache.ratis.thirdparty.io.netty.channel.Channel; import org.apache.ratis.thirdparty.io.netty.channel.ChannelFuture; @@ -64,9 +65,19 @@ public void close() { lifeCycle.checkStateAndClose(() -> NettyUtils.closeChannel(channel, serverAddress)); } - public ChannelFuture writeAndFlush(Object msg) { - lifeCycle.assertCurrentState(LifeCycle.States.RUNNING); - return channel.writeAndFlush(msg); + public ChannelFuture writeAndFlush(Object msg) throws AlreadyClosedException { + final LifeCycle.State state = lifeCycle.getCurrentState(); + if (state.isRunning()) { + return channel.writeAndFlush(msg); + } + // For CLOSING, CLOSED, and EXCEPTION states, throw AlreadyClosedException to trigger reconnection + if (state.isClosingOrClosed() || state == LifeCycle.State.EXCEPTION) { + throw new AlreadyClosedException( + "Client is closed or failed: state=" + state + ", channel=" + channel); + } + // For other states (NEW, STARTING, PAUSING, PAUSED), this is a programming error + throw new IllegalStateException("Client is in unexpected state for writeAndFlush: " + + "state=" + state + ", channel=" + channel); } @Override diff --git a/ratis-netty/src/main/java/org/apache/ratis/netty/NettyRpcProxy.java b/ratis-netty/src/main/java/org/apache/ratis/netty/NettyRpcProxy.java index 2c49de0364..f77096e186 100644 --- a/ratis-netty/src/main/java/org/apache/ratis/netty/NettyRpcProxy.java +++ b/ratis-netty/src/main/java/org/apache/ratis/netty/NettyRpcProxy.java @@ -157,7 +157,7 @@ protected void initChannel(SocketChannel ch) throws Exception { } synchronized ChannelFuture offer(RaftNettyServerRequestProto request, - CompletableFuture reply) { + CompletableFuture reply) throws AlreadyClosedException { replies.offer(reply); return client.writeAndFlush(request); } @@ -199,7 +199,11 @@ public void close() { public CompletableFuture sendAsync(RaftNettyServerRequestProto proto) { final CompletableFuture reply = new CompletableFuture<>(); - connection.offer(proto, reply); + try { + connection.offer(proto, reply); + } catch (AlreadyClosedException e) { + reply.completeExceptionally(e); + } return reply; } From e534bf86c869a91388da6102d0dedee3142b049d Mon Sep 17 00:00:00 2001 From: Tsz-Wo Nicholas Sze Date: Tue, 11 Nov 2025 07:41:34 -0800 Subject: [PATCH 221/397] RATIS-2353. Refactor ReadOnlyRequestTests. (#1308) --- .../apache/ratis/LinearizableReadTests.java | 253 ++++++++++++++ .../apache/ratis/ReadOnlyRequestTests.java | 321 +++++------------- .../ratis/server/impl/MiniRaftCluster.java | 3 + ...stLinearizableLeaderLeaseReadWithGrpc.java | 43 +++ .../grpc/TestLinearizableReadWithGrpc.java | 43 +++ 5 files changed, 425 insertions(+), 238 deletions(-) create mode 100644 ratis-server/src/test/java/org/apache/ratis/LinearizableReadTests.java create mode 100644 ratis-test/src/test/java/org/apache/ratis/grpc/TestLinearizableLeaderLeaseReadWithGrpc.java create mode 100644 ratis-test/src/test/java/org/apache/ratis/grpc/TestLinearizableReadWithGrpc.java diff --git a/ratis-server/src/test/java/org/apache/ratis/LinearizableReadTests.java b/ratis-server/src/test/java/org/apache/ratis/LinearizableReadTests.java new file mode 100644 index 0000000000..49176b18a8 --- /dev/null +++ b/ratis-server/src/test/java/org/apache/ratis/LinearizableReadTests.java @@ -0,0 +1,253 @@ +/* + * 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.ratis; + +import org.apache.ratis.client.RaftClient; +import org.apache.ratis.conf.RaftProperties; +import org.apache.ratis.protocol.RaftClientReply; +import org.apache.ratis.protocol.RaftPeerId; +import org.apache.ratis.protocol.exceptions.ReadIndexException; +import org.apache.ratis.retry.ExceptionDependentRetry; +import org.apache.ratis.retry.RetryPolicies; +import org.apache.ratis.retry.RetryPolicy; +import org.apache.ratis.server.RaftServer; +import org.apache.ratis.server.RaftServerConfigKeys; +import org.apache.ratis.server.impl.MiniRaftCluster; +import org.apache.ratis.util.Slf4jUtils; +import org.apache.ratis.util.TimeDuration; +import org.apache.ratis.util.function.CheckedConsumer; +import org.junit.jupiter.api.Assertions; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.Test; +import org.slf4j.event.Level; + +import java.util.ArrayList; +import java.util.List; +import java.util.concurrent.CompletableFuture; +import java.util.concurrent.TimeUnit; + +import static org.apache.ratis.ReadOnlyRequestTests.CounterStateMachine; +import static org.apache.ratis.ReadOnlyRequestTests.INCREMENT; +import static org.apache.ratis.ReadOnlyRequestTests.QUERY; +import static org.apache.ratis.ReadOnlyRequestTests.WAIT_AND_INCREMENT; +import static org.apache.ratis.ReadOnlyRequestTests.assertReplyAtLeast; +import static org.apache.ratis.ReadOnlyRequestTests.assertReplyExact; +import static org.apache.ratis.ReadOnlyRequestTests.retrieve; +import static org.apache.ratis.server.RaftServerConfigKeys.Read.Option.LINEARIZABLE; + +/** Test for the {@link RaftServerConfigKeys.Read.Option#LINEARIZABLE} feature. */ +public abstract class LinearizableReadTests + extends BaseTest + implements MiniRaftCluster.Factory.Get { + + { + Slf4jUtils.setLogLevel(RaftServer.Division.LOG, Level.DEBUG); + } + + public abstract boolean isLeaderLeaseEnabled(); + + public abstract void assertRaftProperties(RaftProperties properties); + + void runWithNewCluster(CheckedConsumer testCase) throws Exception { + runWithNewCluster(3, 0, true, cluster -> { + assertRaftProperties(cluster.getProperties()); + testCase.accept(cluster); + }); + } + + @BeforeEach + public void setup() { + final RaftProperties p = getProperties(); + CounterStateMachine.setProperties(p); + RaftServerConfigKeys.Read.setOption(p, LINEARIZABLE); + RaftServerConfigKeys.Read.setLeaderLeaseEnabled(p, isLeaderLeaseEnabled()); + } + + @Test + public void testLinearizableRead() throws Exception { + runWithNewCluster(ReadOnlyRequestTests::runTestReadOnly); + } + + @Test + public void testLinearizableReadTimeout() throws Exception { + runWithNewCluster(cluster -> ReadOnlyRequestTests.runTestReadTimeout(ReadIndexException.class, cluster)); + } + + @Test + public void testFollowerLinearizableRead() throws Exception { + runWithNewCluster(LinearizableReadTests::runTestFollowerLinearizableRead); + } + + static class Reply { + private final int count; + private final CompletableFuture future; + + Reply(int count, CompletableFuture future) { + this.count = count; + this.future = future; + } + + void assertExact() { + assertReplyExact(count, future.join()); + } + + void assertAtLeast() { + assertReplyAtLeast(count, future.join()); + } + } + + static void runTestFollowerLinearizableRead(C cluster) throws Exception { + final RaftPeerId leaderId = RaftTestUtil.waitForLeader(cluster).getId(); + + final List followers = cluster.getFollowers(); + Assertions.assertEquals(2, followers.size()); + + final RaftPeerId f0 = followers.get(0).getId(); + final RaftPeerId f1 = followers.get(1).getId(); + + final int n = 100; + final List f0Replies = new ArrayList<>(n); + final List f1Replies = new ArrayList<>(n); + try (RaftClient client = cluster.createClient(leaderId)) { + for (int i = 0; i < n; i++) { + final int count = i + 1; + assertReplyExact(count, client.io().send(INCREMENT)); + + f0Replies.add(new Reply(count, client.async().sendReadOnly(QUERY, f0))); + f1Replies.add(new Reply(count, client.async().sendReadOnly(QUERY, f1))); + } + + for (int i = 0; i < n; i++) { + f0Replies.get(i).assertAtLeast(); + f1Replies.get(i).assertAtLeast(); + } + } + } + + @Test + public void testFollowerLinearizableReadParallel() throws Exception { + runWithNewCluster(LinearizableReadTests::runTestFollowerReadOnlyParallel); + } + + static void runTestFollowerReadOnlyParallel(C cluster) throws Exception { + final RaftPeerId leaderId = RaftTestUtil.waitForLeader(cluster).getId(); + + final List followers = cluster.getFollowers(); + Assertions.assertEquals(2, followers.size()); + final RaftPeerId f0 = followers.get(0).getId(); + final RaftPeerId f1 = followers.get(1).getId(); + + try (RaftClient leaderClient = cluster.createClient(leaderId); + RaftClient f0Client = cluster.createClient(f0); + RaftClient f1Client = cluster.createClient(f1)) { + + final int n = 10; + final List writeReplies = new ArrayList<>(n); + final List f1Replies = new ArrayList<>(n); + for (int i = 0; i < n; i++) { + int count = 2*i + 1; + assertReplyExact(count, leaderClient.io().send(INCREMENT)); + + count++; + writeReplies.add(new Reply(count, leaderClient.async().send(WAIT_AND_INCREMENT))); + Thread.sleep(100); + + assertReplyExact(count, f0Client.io().sendReadOnly(QUERY, f0)); + f1Replies.add(new Reply(count, f1Client.async().sendReadOnly(QUERY, f1))); + } + + for (int i = 0; i < n; i++) { + writeReplies.get(i).assertExact(); + f1Replies.get(i).assertAtLeast(); + } + } + } + + @Test + public void testLinearizableReadFailWhenLeaderDown() throws Exception { + runWithNewCluster(LinearizableReadTests::runTestLinearizableReadFailWhenLeaderDown); + } + + static void runTestLinearizableReadFailWhenLeaderDown(C cluster) throws Exception { + final RaftPeerId leaderId = RaftTestUtil.waitForLeader(cluster).getId(); + + final List followers = cluster.getFollowers(); + Assertions.assertEquals(2, followers.size()); + final RaftPeerId f0 = followers.get(0).getId(); + + try (RaftClient leaderClient = cluster.createClient(leaderId); + RaftClient f0Client = cluster.createClient(f0, RetryPolicies.noRetry())) { + assertReplyExact(1, leaderClient.io().send(INCREMENT)); + assertReplyExact(1, f0Client.io().sendReadOnly(QUERY)); + + // kill the leader + // read timeout quicker than election timeout + final RaftClientReply reply = leaderClient.admin().transferLeadership(null, 200); + Assertions.assertTrue(reply.isSuccess()); + + // client should fail and won't retry + Assertions.assertThrows(ReadIndexException.class, () -> f0Client.io().sendReadOnly(QUERY, f0)); + } + } + + @Test + public void testFollowerReadOnlyRetryWhenLeaderDown() throws Exception { + // only retry on ReadIndexException + final RetryPolicy retryPolicy = ExceptionDependentRetry + .newBuilder() + .setDefaultPolicy(RetryPolicies.noRetry()) + .setExceptionToPolicy(ReadIndexException.class, + RetryPolicies.retryForeverWithSleep(TimeDuration.valueOf(500, TimeUnit.MILLISECONDS))) + .build(); + + runWithNewCluster(cluster -> ReadOnlyRequestTests.runTestReadOnlyRetryWhenLeaderDown(retryPolicy, cluster)); + } + + + @Test + public void testReadAfterWrite() throws Exception { + runWithNewCluster(LinearizableReadTests::runTestReadAfterWrite); + } + + static void runTestReadAfterWrite(C cluster) throws Exception { + final RaftPeerId leaderId = RaftTestUtil.waitForLeader(cluster).getId(); + + try (RaftClient client = cluster.createClient(leaderId)) { + // test blocking read-after-write + assertReplyExact(1, client.io().send(INCREMENT)); + assertReplyExact(1, client.io().sendReadAfterWrite(QUERY)); + + // test asynchronous read-after-write + client.async().send(INCREMENT); + final CompletableFuture asyncReply = client.async().sendReadAfterWrite(QUERY); + + for (int i = 0; i < 20; i++) { + client.async().send(INCREMENT); + } + + // read-after-write is more consistent than linearizable read + final CompletableFuture linearizable = client.async().sendReadOnly(QUERY); + final CompletableFuture readAfterWrite = client.async().sendReadAfterWrite(QUERY); + final int r = retrieve(readAfterWrite.get()); + final int l = retrieve(linearizable.get()); + Assertions.assertTrue(r >= l, () -> "readAfterWrite = " + r + " < linearizable = " + l); + + assertReplyAtLeast(2, asyncReply.join()); + } + } +} \ No newline at end of file diff --git a/ratis-server/src/test/java/org/apache/ratis/ReadOnlyRequestTests.java b/ratis-server/src/test/java/org/apache/ratis/ReadOnlyRequestTests.java index 17fd28cbe6..aa77ee5c77 100644 --- a/ratis-server/src/test/java/org/apache/ratis/ReadOnlyRequestTests.java +++ b/ratis-server/src/test/java/org/apache/ratis/ReadOnlyRequestTests.java @@ -19,31 +19,29 @@ import org.apache.ratis.client.RaftClient; import org.apache.ratis.conf.RaftProperties; +import org.apache.ratis.proto.RaftProtos.LogEntryProto; import org.apache.ratis.protocol.Message; import org.apache.ratis.protocol.RaftClientReply; import org.apache.ratis.protocol.RaftPeerId; +import org.apache.ratis.protocol.exceptions.RaftRetryFailureException; import org.apache.ratis.protocol.exceptions.ReadException; -import org.apache.ratis.protocol.exceptions.ReadIndexException; -import org.apache.ratis.retry.ExceptionDependentRetry; import org.apache.ratis.retry.RetryPolicies; import org.apache.ratis.retry.RetryPolicy; import org.apache.ratis.server.RaftServer; import org.apache.ratis.server.RaftServerConfigKeys; import org.apache.ratis.server.impl.MiniRaftCluster; +import org.apache.ratis.server.protocol.TermIndex; import org.apache.ratis.statemachine.StateMachine; import org.apache.ratis.statemachine.TransactionContext; import org.apache.ratis.statemachine.impl.BaseStateMachine; import org.apache.ratis.util.Slf4jUtils; -import org.apache.ratis.util.TimeDuration; import org.junit.jupiter.api.Assertions; import org.junit.jupiter.api.BeforeEach; import org.junit.jupiter.api.Test; import org.slf4j.event.Level; import java.nio.charset.StandardCharsets; -import java.util.List; import java.util.concurrent.CompletableFuture; -import java.util.concurrent.TimeUnit; import java.util.concurrent.atomic.AtomicLong; public abstract class ReadOnlyRequestTests @@ -56,270 +54,109 @@ public abstract class ReadOnlyRequestTests static final int NUM_SERVERS = 3; - static final String INCREMENT = "INCREMENT"; - static final String WAIT_AND_INCREMENT = "WAIT_AND_INCREMENT"; - static final String QUERY = "QUERY"; - final Message incrementMessage = new RaftTestUtil.SimpleMessage(INCREMENT); - final Message waitAndIncrementMessage = new RaftTestUtil.SimpleMessage(WAIT_AND_INCREMENT); - final Message queryMessage = new RaftTestUtil.SimpleMessage(QUERY); + static final String INCREMENT_STRING = "INCREMENT"; + static final String WAIT_AND_INCREMENT_STRING = "WAIT_AND_INCREMENT"; + static final String QUERY_STRING = "QUERY"; + + static final Message INCREMENT = new RaftTestUtil.SimpleMessage(INCREMENT_STRING); + static final Message WAIT_AND_INCREMENT = new RaftTestUtil.SimpleMessage(WAIT_AND_INCREMENT_STRING); + static final Message QUERY = new RaftTestUtil.SimpleMessage(QUERY_STRING); @BeforeEach public void setup() { final RaftProperties p = getProperties(); - p.setClass(MiniRaftCluster.STATEMACHINE_CLASS_KEY, - CounterStateMachine.class, StateMachine.class); + CounterStateMachine.setProperties(p); } - @Test - public void testLinearizableRead() throws Exception { - getProperties().setEnum(RaftServerConfigKeys.Read.OPTION_KEY, RaftServerConfigKeys.Read.Option.LINEARIZABLE); - runWithNewCluster(NUM_SERVERS, this::testReadOnlyImpl); + public static void assertOption(RaftServerConfigKeys.Read.Option expected, RaftProperties properties) { + final RaftServerConfigKeys.Read.Option computed = RaftServerConfigKeys.Read.option(properties); + Assertions.assertEquals(expected, computed); } @Test - public void testLeaseRead() throws Exception { - getProperties().setBoolean(RaftServerConfigKeys.Read.LEADER_LEASE_ENABLED_KEY, true); - runWithNewCluster(NUM_SERVERS, this::testReadOnlyImpl); + public void testReadOnly() throws Exception { + assertOption(RaftServerConfigKeys.Read.Option.DEFAULT, getProperties()); + runWithNewCluster(NUM_SERVERS, ReadOnlyRequestTests::runTestReadOnly); } - private void testReadOnlyImpl(CLUSTER cluster) throws Exception { + static void runTestReadOnly(C cluster) throws Exception { try { RaftTestUtil.waitForLeader(cluster); final RaftPeerId leaderId = cluster.getLeader().getId(); try (final RaftClient client = cluster.createClient(leaderId)) { for (int i = 1; i <= 10; i++) { - RaftClientReply reply = client.io().send(incrementMessage); - Assertions.assertTrue(reply.isSuccess()); - reply = client.io().sendReadOnly(queryMessage); - Assertions.assertEquals(i, retrieve(reply)); - } - } - } finally { - cluster.shutdown(); - } - } - - @Test - public void testLinearizableReadTimeout() throws Exception { - getProperties().setEnum(RaftServerConfigKeys.Read.OPTION_KEY, RaftServerConfigKeys.Read.Option.LINEARIZABLE); - runWithNewCluster(NUM_SERVERS, this::testReadOnlyTimeoutImpl); - } - - @Test - public void testLeaseReadTimeout() throws Exception { - getProperties().setBoolean(RaftServerConfigKeys.Read.LEADER_LEASE_ENABLED_KEY, true); - runWithNewCluster(NUM_SERVERS, this::testReadOnlyTimeoutImpl); - } - - private void testReadOnlyTimeoutImpl(CLUSTER cluster) throws Exception { - try { - RaftTestUtil.waitForLeader(cluster); - final RaftPeerId leaderId = cluster.getLeader().getId(); - - try (final RaftClient client = cluster.createClient(leaderId); - final RaftClient noRetry = cluster.createClient(leaderId, RetryPolicies.noRetry())) { - - CompletableFuture result = client.async().send(incrementMessage); - client.admin().transferLeadership(null, 200); - - Assertions.assertThrows(ReadIndexException.class, () -> { - RaftClientReply timeoutReply = noRetry.io().sendReadOnly(queryMessage); - Assertions.assertNotNull(timeoutReply.getException()); - Assertions.assertTrue(timeoutReply.getException() instanceof ReadException); - }); - } - - } finally { - cluster.shutdown(); - } - } - - @Test - public void testFollowerLinearizableRead() throws Exception { - getProperties().setEnum(RaftServerConfigKeys.Read.OPTION_KEY, RaftServerConfigKeys.Read.Option.LINEARIZABLE); - runWithNewCluster(NUM_SERVERS, this::testFollowerReadOnlyImpl); - } - - @Test - public void testFollowerLeaseRead() throws Exception { - getProperties().setBoolean(RaftServerConfigKeys.Read.LEADER_LEASE_ENABLED_KEY, true); - runWithNewCluster(NUM_SERVERS, this::testFollowerReadOnlyImpl); - } - - private void testFollowerReadOnlyImpl(CLUSTER cluster) throws Exception { - try { - RaftTestUtil.waitForLeader(cluster); - - List followers = cluster.getFollowers(); - Assertions.assertEquals(2, followers.size()); - - final RaftPeerId f0 = followers.get(0).getId(); - final RaftPeerId f1 = followers.get(1).getId(); - try (RaftClient client = cluster.createClient(cluster.getLeader().getId())) { - for (int i = 1; i <= 10; i++) { - final RaftClientReply reply = client.io().send(incrementMessage); - Assertions.assertTrue(reply.isSuccess()); - final RaftClientReply read1 = client.io().sendReadOnly(queryMessage, f0); - Assertions.assertEquals(i, retrieve(read1)); - final CompletableFuture read2 = client.async().sendReadOnly(queryMessage, f1); - Assertions.assertEquals(i, retrieve(read2.get(1, TimeUnit.SECONDS))); + assertReplyExact(i, client.io().send(INCREMENT)); + assertReplyExact(i, client.io().sendReadOnly(QUERY)); } } } finally { cluster.shutdown(); } } - - @Test - public void testFollowerLinearizableReadParallel() throws Exception { - getProperties().setEnum(RaftServerConfigKeys.Read.OPTION_KEY, RaftServerConfigKeys.Read.Option.LINEARIZABLE); - runWithNewCluster(NUM_SERVERS, this::testFollowerReadOnlyParallelImpl); - } - - @Test - public void testFollowerLeaseReadParallel() throws Exception { - getProperties().setBoolean(RaftServerConfigKeys.Read.LEADER_LEASE_ENABLED_KEY, true); - runWithNewCluster(NUM_SERVERS, this::testFollowerReadOnlyParallelImpl); - } - - private void testFollowerReadOnlyParallelImpl(CLUSTER cluster) throws Exception { - try { - RaftTestUtil.waitForLeader(cluster); - - List followers = cluster.getFollowers(); - Assertions.assertEquals(2, followers.size()); - - try (RaftClient leaderClient = cluster.createClient(cluster.getLeader().getId()); - RaftClient followerClient1 = cluster.createClient(followers.get(0).getId())) { - - leaderClient.io().send(incrementMessage); - leaderClient.async().send(waitAndIncrementMessage); - Thread.sleep(100); - - RaftClientReply clientReply = followerClient1.io().sendReadOnly(queryMessage, followers.get(0).getId()); - Assertions.assertEquals(2, retrieve(clientReply)); - } - - } finally { - cluster.shutdown(); - } - } - - @Test - public void testFollowerLinearizableReadFailWhenLeaderDown() throws Exception { - getProperties().setEnum(RaftServerConfigKeys.Read.OPTION_KEY, RaftServerConfigKeys.Read.Option.LINEARIZABLE); - runWithNewCluster(NUM_SERVERS, this::testFollowerReadOnlyFailWhenLeaderDownImpl); - } - @Test - public void testFollowerLeaseReadWhenLeaderDown() throws Exception { - getProperties().setBoolean(RaftServerConfigKeys.Read.LEADER_LEASE_ENABLED_KEY, true); - runWithNewCluster(NUM_SERVERS, this::testFollowerReadOnlyFailWhenLeaderDownImpl); + public void testReadTimeout() throws Exception { + runWithNewCluster(NUM_SERVERS, cluster -> runTestReadTimeout(RaftRetryFailureException.class, cluster)); } - private void testFollowerReadOnlyFailWhenLeaderDownImpl(CLUSTER cluster) throws Exception { - try { - RaftTestUtil.waitForLeader(cluster); + static void runTestReadTimeout(Class exceptionClass, C cluster) + throws Exception { + final RaftPeerId leaderId = RaftTestUtil.waitForLeader(cluster).getId(); - List followers = cluster.getFollowers(); - Assertions.assertEquals(2, followers.size()); + try (final RaftClient client = cluster.createClient(leaderId); + final RaftClient noRetry = cluster.createClient(leaderId, RetryPolicies.noRetry())) { - try (RaftClient leaderClient = cluster.createClient(cluster.getLeader().getId()); - RaftClient followerClient1 = cluster.createClient(followers.get(0).getId(), RetryPolicies.noRetry())) { - leaderClient.io().send(incrementMessage); - - RaftClientReply clientReply = followerClient1.io().sendReadOnly(queryMessage); - Assertions.assertEquals(1, retrieve(clientReply)); - - // kill the leader - // read timeout quicker than election timeout - leaderClient.admin().transferLeadership(null, 200); - - Assertions.assertThrows(ReadIndexException.class, () -> { - followerClient1.io().sendReadOnly(queryMessage, followers.get(0).getId()); - }); - } + assertReplyExact(1, client.io().send(INCREMENT)); + client.admin().transferLeadership(null, 200); - } finally { - cluster.shutdown(); + Assertions.assertThrows(exceptionClass, () -> { + final RaftClientReply timeoutReply = noRetry.io().sendReadOnly(QUERY); + Assertions.assertFalse(timeoutReply.isSuccess()); + Assertions.assertNotNull(timeoutReply.getException()); + Assertions.assertInstanceOf(ReadException.class, timeoutReply.getException()); + }); } } @Test - public void testFollowerReadOnlyRetryWhenLeaderDown() throws Exception { - getProperties().setEnum(RaftServerConfigKeys.Read.OPTION_KEY, RaftServerConfigKeys.Read.Option.LINEARIZABLE); - runWithNewCluster(NUM_SERVERS, this::testFollowerReadOnlyRetryWhenLeaderDown); - } - - @Test - public void testFollowerLeaseReadRetryWhenLeaderDown() throws Exception { - getProperties().setBoolean(RaftServerConfigKeys.Read.LEADER_LEASE_ENABLED_KEY, true); - runWithNewCluster(NUM_SERVERS, this::testFollowerReadOnlyRetryWhenLeaderDown); + public void testReadOnlyRetryWhenLeaderDown() throws Exception { + runWithNewCluster(NUM_SERVERS, cluster -> runTestReadOnlyRetryWhenLeaderDown(null, cluster)); } - private void testFollowerReadOnlyRetryWhenLeaderDown(CLUSTER cluster) throws Exception { - // only retry on readIndexException - final RetryPolicy retryPolicy = ExceptionDependentRetry - .newBuilder() - .setDefaultPolicy(RetryPolicies.noRetry()) - .setExceptionToPolicy(ReadIndexException.class, - RetryPolicies.retryForeverWithSleep(TimeDuration.valueOf(500, TimeUnit.MILLISECONDS))) - .build(); + static void runTestReadOnlyRetryWhenLeaderDown(RetryPolicy retryPolicy, C cluster) + throws Exception { + final RaftPeerId leaderId = RaftTestUtil.waitForLeader(cluster).getId(); - RaftTestUtil.waitForLeader(cluster); - - try (RaftClient client = cluster.createClient(cluster.getLeader().getId(), retryPolicy)) { - client.io().send(incrementMessage); - - final RaftClientReply clientReply = client.io().sendReadOnly(queryMessage); - Assertions.assertEquals(1, retrieve(clientReply)); + try (RaftClient client = cluster.createClient(leaderId, retryPolicy)) { + assertReplyExact(1, client.io().send(INCREMENT)); + assertReplyExact(1, client.io().sendReadOnly(QUERY)); // kill the leader client.admin().transferLeadership(null, 200); // readOnly will success after re-election - final RaftClientReply replySuccess = client.io().sendReadOnly(queryMessage); - Assertions.assertEquals(1, retrieve(clientReply)); + assertReplyExact(1, client.io().sendReadOnly(QUERY)); } } - @Test - public void testReadAfterWrite() throws Exception { - runWithNewCluster(NUM_SERVERS, this::testReadAfterWriteImpl); + static int retrieve(RaftClientReply reply) { + Assertions.assertTrue(reply.isSuccess()); + return Integer.parseInt(reply.getMessage().getContent().toString(StandardCharsets.UTF_8)); } - private void testReadAfterWriteImpl(CLUSTER cluster) throws Exception { - RaftTestUtil.waitForLeader(cluster); - try (RaftClient client = cluster.createClient()) { - // test blocking read-after-write - client.io().send(incrementMessage); - final RaftClientReply blockReply = client.io().sendReadAfterWrite(queryMessage); - Assertions.assertEquals(1, retrieve(blockReply)); - - // test asynchronous read-after-write - client.async().send(incrementMessage); - client.async().sendReadAfterWrite(queryMessage).thenAccept(reply -> { - Assertions.assertEquals(2, retrieve(reply)); - }); - - for (int i = 0; i < 20; i++) { - client.async().send(incrementMessage); - } - final CompletableFuture linearizable = client.async().sendReadOnly(queryMessage); - final CompletableFuture readAfterWrite = client.async().sendReadAfterWrite(queryMessage); - - CompletableFuture.allOf(linearizable, readAfterWrite).get(); - // read-after-write is more consistent than linearizable read - Assertions.assertTrue(retrieve(readAfterWrite.get()) >= retrieve(linearizable.get())); - } + static void assertReplyExact(int expectedCount, RaftClientReply reply) { + Assertions.assertTrue(reply.isSuccess()); + final int retrieved = retrieve(reply); + Assertions.assertEquals(expectedCount, retrieved, () -> "reply=" + reply); } - static int retrieve(RaftClientReply reply) { - return Integer.parseInt(reply.getMessage().getContent().toString(StandardCharsets.UTF_8)); + static void assertReplyAtLeast(int minCount, RaftClientReply reply) { + Assertions.assertTrue(reply.isSuccess()); + final int retrieved = retrieve(reply); + Assertions.assertTrue(retrieved >= minCount, + () -> "retrieved = " + retrieved + " < minCount = " + minCount + ", reply=" + reply); } - /** * CounterStateMachine support 3 operations * 1. increment @@ -327,12 +164,19 @@ static int retrieve(RaftClientReply reply) { * 3. waitAndIncrement */ static class CounterStateMachine extends BaseStateMachine { + static void setProperties(RaftProperties properties) { + properties.setClass(MiniRaftCluster.STATEMACHINE_CLASS_KEY, CounterStateMachine.class, StateMachine.class); + } + private final AtomicLong counter = new AtomicLong(0L); @Override public CompletableFuture query(Message request) { - return CompletableFuture.completedFuture( - Message.valueOf(String.valueOf(counter.get()))); + return toMessageFuture(counter.get()); + } + + static CompletableFuture toMessageFuture(long count) { + return CompletableFuture.completedFuture(Message.valueOf(String.valueOf(count))); } @Override @@ -349,39 +193,40 @@ private void sleepQuietly(int millis) { } } - private void increment() { - counter.incrementAndGet(); + private long increment() { + return counter.incrementAndGet(); } - private void waitAndIncrement() { + private long waitAndIncrement() { sleepQuietly(500); - increment(); + return increment(); } - private void timeoutIncrement() { + private long timeoutIncrement() { sleepQuietly(5000); - increment(); + return increment(); } @Override public CompletableFuture applyTransaction(TransactionContext trx) { - LOG.debug("apply trx with index=" + trx.getLogEntry().getIndex()); - updateLastAppliedTermIndex(trx.getLogEntry().getTerm(), trx.getLogEntry().getIndex()); + final LogEntryProto logEntry = trx.getLogEntry(); + final TermIndex ti = TermIndex.valueOf(logEntry); + updateLastAppliedTermIndex(ti); - String command = trx.getLogEntry().getStateMachineLogEntry() - .getLogData().toString(StandardCharsets.UTF_8); + final String command = logEntry.getStateMachineLogEntry().getLogData().toString(StandardCharsets.UTF_8); - LOG.info("receive command: {}", command); - if (command.equals(INCREMENT)) { - increment(); - } else if (command.equals(WAIT_AND_INCREMENT)) { - waitAndIncrement(); + final long updatedCount; + if (command.equals(INCREMENT_STRING)) { + updatedCount = increment(); + } else if (command.equals(WAIT_AND_INCREMENT_STRING)) { + updatedCount = waitAndIncrement(); } else { - timeoutIncrement(); + updatedCount = timeoutIncrement(); } + LOG.info("Applied {} command {}, updatedCount={}", ti, command, updatedCount); - return CompletableFuture.completedFuture(Message.valueOf("OK")); + return toMessageFuture(updatedCount); } } } diff --git a/ratis-server/src/test/java/org/apache/ratis/server/impl/MiniRaftCluster.java b/ratis-server/src/test/java/org/apache/ratis/server/impl/MiniRaftCluster.java index 253b2cdeb1..87ffa8d198 100644 --- a/ratis-server/src/test/java/org/apache/ratis/server/impl/MiniRaftCluster.java +++ b/ratis-server/src/test/java/org/apache/ratis/server/impl/MiniRaftCluster.java @@ -727,6 +727,9 @@ public RaftClient createClient(RetryPolicy retryPolicy) { } public RaftClient createClient(RaftPeerId leaderId, RetryPolicy retryPolicy) { + if (retryPolicy == null) { + retryPolicy = getDefaultRetryPolicy(); + } return createClient(leaderId, group, retryPolicy); } diff --git a/ratis-test/src/test/java/org/apache/ratis/grpc/TestLinearizableLeaderLeaseReadWithGrpc.java b/ratis-test/src/test/java/org/apache/ratis/grpc/TestLinearizableLeaderLeaseReadWithGrpc.java new file mode 100644 index 0000000000..e45d8f4ff4 --- /dev/null +++ b/ratis-test/src/test/java/org/apache/ratis/grpc/TestLinearizableLeaderLeaseReadWithGrpc.java @@ -0,0 +1,43 @@ +/* + * 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.ratis.grpc; + +import org.apache.ratis.LinearizableReadTests; +import org.apache.ratis.conf.RaftProperties; +import org.apache.ratis.server.RaftServerConfigKeys; + +import static org.apache.ratis.ReadOnlyRequestTests.assertOption; +import static org.apache.ratis.server.RaftServerConfigKeys.Read.Option.LINEARIZABLE; +import static org.junit.jupiter.api.Assertions.assertTrue; + +public class TestLinearizableLeaderLeaseReadWithGrpc + extends LinearizableReadTests + implements MiniRaftClusterWithGrpc.FactoryGet { + + @Override + public boolean isLeaderLeaseEnabled() { + return true; + } + + @Override + public void assertRaftProperties(RaftProperties p) { + assertOption(LINEARIZABLE, p); + assertTrue(RaftServerConfigKeys.Read.leaderLeaseEnabled(p)); + assertTrue(isLeaderLeaseEnabled()); + } +} diff --git a/ratis-test/src/test/java/org/apache/ratis/grpc/TestLinearizableReadWithGrpc.java b/ratis-test/src/test/java/org/apache/ratis/grpc/TestLinearizableReadWithGrpc.java new file mode 100644 index 0000000000..a434fe0003 --- /dev/null +++ b/ratis-test/src/test/java/org/apache/ratis/grpc/TestLinearizableReadWithGrpc.java @@ -0,0 +1,43 @@ +/* + * 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.ratis.grpc; + +import org.apache.ratis.LinearizableReadTests; +import org.apache.ratis.conf.RaftProperties; +import org.apache.ratis.server.RaftServerConfigKeys; + +import static org.apache.ratis.ReadOnlyRequestTests.assertOption; +import static org.apache.ratis.server.RaftServerConfigKeys.Read.Option.LINEARIZABLE; +import static org.junit.jupiter.api.Assertions.assertFalse; + +public class TestLinearizableReadWithGrpc + extends LinearizableReadTests + implements MiniRaftClusterWithGrpc.FactoryGet { + + @Override + public boolean isLeaderLeaseEnabled() { + return false; + } + + @Override + public void assertRaftProperties(RaftProperties p) { + assertOption(LINEARIZABLE, p); + assertFalse(RaftServerConfigKeys.Read.leaderLeaseEnabled(p)); + assertFalse(isLeaderLeaseEnabled()); + } +} From 0f329cbdd515e41ac1140708498c23bb1211180f Mon Sep 17 00:00:00 2001 From: "Doroszlai, Attila" <6454655+adoroszlai@users.noreply.github.com> Date: Tue, 11 Nov 2025 17:47:49 +0100 Subject: [PATCH 222/397] RATIS-2355. Duplicate output in checkstyle check (#1310) --- dev-support/checks/checkstyle.sh | 9 +++++---- 1 file changed, 5 insertions(+), 4 deletions(-) diff --git a/dev-support/checks/checkstyle.sh b/dev-support/checks/checkstyle.sh index cb06fdaacd..473035bf11 100755 --- a/dev-support/checks/checkstyle.sh +++ b/dev-support/checks/checkstyle.sh @@ -26,16 +26,17 @@ REPORT_FILE="$REPORT_DIR/summary.txt" MAVEN_OPTIONS='-B -fae -Dcheckstyle.failOnViolation=false' declare -i rc -${MVN} ${MAVEN_OPTIONS} checkstyle:check | tee "${REPORT_DIR}/output.log" +${MVN} ${MAVEN_OPTIONS} checkstyle:check > "${REPORT_DIR}/output.log" rc=$? if [[ ${rc} -ne 0 ]]; then - ${MVN} ${MAVEN_OPTIONS} clean test-compile checkstyle:check + ${MVN} ${MAVEN_OPTIONS} clean test-compile checkstyle:check > output.log rc=$? mkdir -p "$REPORT_DIR" # removed by mvn clean -else - cat "${REPORT_DIR}/output.log" + mv output.log "${REPORT_DIR}"/ fi +cat "${REPORT_DIR}/output.log" + #Print out the exact violations with parsing XML results with sed find "." -name checkstyle-result.xml -print0 \ | xargs -0 sed '$!N; / Date: Tue, 11 Nov 2025 19:10:23 +0100 Subject: [PATCH 223/397] RATIS-2354. Bump Apache parent POM to 35 (#1309) --- pom.xml | 70 ++------------------------------------------------------- 1 file changed, 2 insertions(+), 68 deletions(-) diff --git a/pom.xml b/pom.xml index 72cd5a646e..820214c863 100644 --- a/pom.xml +++ b/pom.xml @@ -18,7 +18,7 @@ org.apache apache - 25 + 35 @@ -168,17 +168,10 @@ 5.1.8 - 3.3.0 - 4.0.6 - 1.6.1 - 3.3.0 - 3.6.0 - 3.5.3 3.5.3 3.4.0 5.0.0.4389 - 3.3.0 3.1.0 @@ -457,10 +450,6 @@ ${license-maven-plugin.version} - - org.apache.maven.plugins - maven-install-plugin - org.apache.maven.plugins maven-javadoc-plugin @@ -473,11 +462,6 @@ - - org.apache.maven.plugins - maven-shade-plugin - ${maven-shade-plugin.version} - com.github.spotbugs spotbugs-maven-plugin @@ -499,16 +483,10 @@ exec-maven-plugin ${exec-maven-plugin.version} - - org.apache.maven.plugins - maven-pdf-plugin - ${maven-pdf-plugin.version} - org.apache.maven.plugins maven-enforcer-plugin - 3.2.1 @@ -571,21 +549,6 @@ - - - org.apache.maven.plugins - maven-source-plugin - - - attach-sources - prepare-package - - jar-no-fork - - - - @@ -614,7 +577,6 @@ org.apache.maven.plugins maven-surefire-plugin - ${maven-surefire-plugin.version} false false @@ -657,10 +619,6 @@ - - org.apache.maven.plugins - maven-deploy-plugin - org.apache.rat apache-rat-plugin @@ -673,15 +631,6 @@ - - org.apache.maven.plugins - maven-antrun-plugin - - - org.apache.maven.plugins - maven-remote-resources-plugin - ${maven-remote-resources-plugin.version} - org.apache.maven.plugins maven-site-plugin @@ -693,11 +642,6 @@ - - com.atlassian.maven.plugins - maven-clover2-plugin - ${maven-clover2-plugin.version} - org.apache.felix maven-bundle-plugin @@ -801,7 +745,6 @@ org.apache.maven.plugins maven-checkstyle-plugin - ${maven-checkstyle-plugin.version} @@ -810,15 +753,6 @@ false - - org.apache.maven.plugins - maven-pdf-plugin - - ${project.reporting.outputDirectory} - - false - - org.codehaus.mojo build-helper-maven-plugin @@ -974,7 +908,7 @@ - ratis-java-sources + attach-sources package jar-no-fork From de687e1d1402a20310087a491ff8e493f7a364c9 Mon Sep 17 00:00:00 2001 From: "Doroszlai, Attila" <6454655+adoroszlai@users.noreply.github.com> Date: Wed, 12 Nov 2025 19:15:01 +0100 Subject: [PATCH 224/397] RATIS-2356. Define maven.compiler.release globally (#1312) --- pom.xml | 20 +------------------- 1 file changed, 1 insertion(+), 19 deletions(-) diff --git a/pom.xml b/pom.xml index 820214c863..3be2213fc3 100644 --- a/pom.xml +++ b/pom.xml @@ -197,6 +197,7 @@ 8 ${javac.version} + ${javac.version} 3.3.9 @@ -814,25 +815,6 @@ - - java8 - - [,8] - - - ${javac.version} - ${javac.version} - - - - java9-or-later - - [9,] - - - ${javac.version} - - experiments-build From 82dbcf9de57852b07809b9c6d2e73c839222bd13 Mon Sep 17 00:00:00 2001 From: "Doroszlai, Attila" <6454655+adoroszlai@users.noreply.github.com> Date: Thu, 13 Nov 2025 19:25:24 +0100 Subject: [PATCH 225/397] RATIS-2358. Remove unnecessary repository definitions (#1314) --- pom.xml | 18 ------------------ 1 file changed, 18 deletions(-) diff --git a/pom.xml b/pom.xml index 3be2213fc3..04dc36635d 100644 --- a/pom.xml +++ b/pom.xml @@ -46,24 +46,6 @@ - - - ${distMgmtSnapshotsId} - ${distMgmtSnapshotsName} - ${distMgmtSnapshotsUrl} - - false - - - - repository.jboss.org - https://repository.jboss.org/nexus/content/groups/public/ - - false - - - - Apache License, Version 2.0 From 027965ad1a52dcbe7ac331cd0974b7f89898ac36 Mon Sep 17 00:00:00 2001 From: Symious <14933944+symious@users.noreply.github.com> Date: Fri, 14 Nov 2025 03:08:44 +0800 Subject: [PATCH 226/397] RATIS-2352. Update spotbugs to 4.8.6 and suppress new warnings (#1307) Co-authored-by: Yiyang Zhou --- pom.xml | 4 +- .../dev-support/findbugsExcludeFile.xml | 42 ++++++ ratis-client/pom.xml | 11 ++ .../dev-support/findbugsExcludeFile.xml | 130 ++++++++++++++++++ ratis-common/pom.xml | 11 ++ .../dev-support/findbugsExcludeFile.xml | 26 ++++ ratis-examples/pom.xml | 7 + .../dev-support/findbugsExcludeFile.xml | 26 ++++ ratis-grpc/pom.xml | 11 ++ .../dev-support/findbugsExcludeFile.xml | 22 +++ ratis-metrics-api/pom.xml | 11 ++ .../dev-support/findbugsExcludeFile.xml | 22 +++ ratis-metrics-default/pom.xml | 11 ++ .../dev-support/findbugsExcludeFile.xml | 22 +++ ratis-metrics-dropwizard3/pom.xml | 11 ++ .../dev-support/findbugsExcludeFile.xml | 38 +++++ ratis-netty/pom.xml | 12 ++ .../dev-support/findbugsExcludeFile.xml | 34 +++++ ratis-server-api/pom.xml | 11 ++ .../dev-support/findbugsExcludeFile.xml | 90 ++++++++++++ ratis-server/pom.xml | 11 ++ .../dev-support/findbugsExcludeFile.xml | 30 ++++ ratis-shell/pom.xml | 7 + 23 files changed, 598 insertions(+), 2 deletions(-) create mode 100644 ratis-client/dev-support/findbugsExcludeFile.xml create mode 100644 ratis-common/dev-support/findbugsExcludeFile.xml create mode 100644 ratis-examples/dev-support/findbugsExcludeFile.xml create mode 100644 ratis-grpc/dev-support/findbugsExcludeFile.xml create mode 100644 ratis-metrics-api/dev-support/findbugsExcludeFile.xml create mode 100644 ratis-metrics-default/dev-support/findbugsExcludeFile.xml create mode 100644 ratis-metrics-dropwizard3/dev-support/findbugsExcludeFile.xml create mode 100644 ratis-netty/dev-support/findbugsExcludeFile.xml create mode 100644 ratis-server-api/dev-support/findbugsExcludeFile.xml create mode 100644 ratis-server/dev-support/findbugsExcludeFile.xml create mode 100644 ratis-shell/dev-support/findbugsExcludeFile.xml diff --git a/pom.xml b/pom.xml index 04dc36635d..718a1a3e27 100644 --- a/pom.xml +++ b/pom.xml @@ -164,8 +164,8 @@ 0.6.1 2.8.0 - 4.2.1 - 4.2.0 + 4.8.6 + 4.8.6.2 apache.snapshots.https Apache Development Snapshot Repository diff --git a/ratis-client/dev-support/findbugsExcludeFile.xml b/ratis-client/dev-support/findbugsExcludeFile.xml new file mode 100644 index 0000000000..3a808c4486 --- /dev/null +++ b/ratis-client/dev-support/findbugsExcludeFile.xml @@ -0,0 +1,42 @@ + + + + + + + + + + + + + + + + + + + + + + + + + + + \ No newline at end of file diff --git a/ratis-client/pom.xml b/ratis-client/pom.xml index 89cd67e7e6..26b2034983 100644 --- a/ratis-client/pom.xml +++ b/ratis-client/pom.xml @@ -59,4 +59,15 @@ test + + + + com.github.spotbugs + spotbugs-maven-plugin + + ${basedir}/dev-support/findbugsExcludeFile.xml + + + + diff --git a/ratis-common/dev-support/findbugsExcludeFile.xml b/ratis-common/dev-support/findbugsExcludeFile.xml new file mode 100644 index 0000000000..3148752344 --- /dev/null +++ b/ratis-common/dev-support/findbugsExcludeFile.xml @@ -0,0 +1,130 @@ + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + \ No newline at end of file diff --git a/ratis-common/pom.xml b/ratis-common/pom.xml index f60cc0240a..f6bc0ee413 100644 --- a/ratis-common/pom.xml +++ b/ratis-common/pom.xml @@ -59,4 +59,15 @@ test + + + + com.github.spotbugs + spotbugs-maven-plugin + + ${basedir}/dev-support/findbugsExcludeFile.xml + + + + diff --git a/ratis-examples/dev-support/findbugsExcludeFile.xml b/ratis-examples/dev-support/findbugsExcludeFile.xml new file mode 100644 index 0000000000..26260f03ea --- /dev/null +++ b/ratis-examples/dev-support/findbugsExcludeFile.xml @@ -0,0 +1,26 @@ + + + + + + + + + + + \ No newline at end of file diff --git a/ratis-examples/pom.xml b/ratis-examples/pom.xml index 8464f93c03..c52019334f 100644 --- a/ratis-examples/pom.xml +++ b/ratis-examples/pom.xml @@ -179,6 +179,13 @@ + + com.github.spotbugs + spotbugs-maven-plugin + + ${basedir}/dev-support/findbugsExcludeFile.xml + + diff --git a/ratis-grpc/dev-support/findbugsExcludeFile.xml b/ratis-grpc/dev-support/findbugsExcludeFile.xml new file mode 100644 index 0000000000..c13c34ade0 --- /dev/null +++ b/ratis-grpc/dev-support/findbugsExcludeFile.xml @@ -0,0 +1,26 @@ + + + + + + + + + + + \ No newline at end of file diff --git a/ratis-grpc/pom.xml b/ratis-grpc/pom.xml index ba3110178d..360131d55b 100644 --- a/ratis-grpc/pom.xml +++ b/ratis-grpc/pom.xml @@ -74,4 +74,15 @@ test + + + + com.github.spotbugs + spotbugs-maven-plugin + + ${basedir}/dev-support/findbugsExcludeFile.xml + + + + diff --git a/ratis-metrics-api/dev-support/findbugsExcludeFile.xml b/ratis-metrics-api/dev-support/findbugsExcludeFile.xml new file mode 100644 index 0000000000..2b3c140b14 --- /dev/null +++ b/ratis-metrics-api/dev-support/findbugsExcludeFile.xml @@ -0,0 +1,22 @@ + + + + + + + \ No newline at end of file diff --git a/ratis-metrics-api/pom.xml b/ratis-metrics-api/pom.xml index 16c368084a..db368269e2 100644 --- a/ratis-metrics-api/pom.xml +++ b/ratis-metrics-api/pom.xml @@ -51,4 +51,15 @@ test + + + + com.github.spotbugs + spotbugs-maven-plugin + + ${basedir}/dev-support/findbugsExcludeFile.xml + + + + diff --git a/ratis-metrics-default/dev-support/findbugsExcludeFile.xml b/ratis-metrics-default/dev-support/findbugsExcludeFile.xml new file mode 100644 index 0000000000..3b4b52c664 --- /dev/null +++ b/ratis-metrics-default/dev-support/findbugsExcludeFile.xml @@ -0,0 +1,22 @@ + + + + + + + \ No newline at end of file diff --git a/ratis-metrics-default/pom.xml b/ratis-metrics-default/pom.xml index ada045e961..b561cbe924 100644 --- a/ratis-metrics-default/pom.xml +++ b/ratis-metrics-default/pom.xml @@ -56,4 +56,15 @@ test + + + + com.github.spotbugs + spotbugs-maven-plugin + + ${basedir}/dev-support/findbugsExcludeFile.xml + + + + diff --git a/ratis-metrics-dropwizard3/dev-support/findbugsExcludeFile.xml b/ratis-metrics-dropwizard3/dev-support/findbugsExcludeFile.xml new file mode 100644 index 0000000000..c413237ff8 --- /dev/null +++ b/ratis-metrics-dropwizard3/dev-support/findbugsExcludeFile.xml @@ -0,0 +1,22 @@ + + + + + + + \ No newline at end of file diff --git a/ratis-metrics-dropwizard3/pom.xml b/ratis-metrics-dropwizard3/pom.xml index f50ee14a30..3d06f57cc5 100644 --- a/ratis-metrics-dropwizard3/pom.xml +++ b/ratis-metrics-dropwizard3/pom.xml @@ -94,4 +94,15 @@ + + + + com.github.spotbugs + spotbugs-maven-plugin + + ${basedir}/dev-support/findbugsExcludeFile.xml + + + + diff --git a/ratis-netty/dev-support/findbugsExcludeFile.xml b/ratis-netty/dev-support/findbugsExcludeFile.xml new file mode 100644 index 0000000000..0e1646df77 --- /dev/null +++ b/ratis-netty/dev-support/findbugsExcludeFile.xml @@ -0,0 +1,38 @@ + + + + + + + + + + + + + + + + + + + + + + + \ No newline at end of file diff --git a/ratis-netty/pom.xml b/ratis-netty/pom.xml index f9d34fffc5..5688a1fa4a 100644 --- a/ratis-netty/pom.xml +++ b/ratis-netty/pom.xml @@ -90,4 +90,16 @@ + + + + + com.github.spotbugs + spotbugs-maven-plugin + + ${basedir}/dev-support/findbugsExcludeFile.xml + + + + diff --git a/ratis-server-api/dev-support/findbugsExcludeFile.xml b/ratis-server-api/dev-support/findbugsExcludeFile.xml new file mode 100644 index 0000000000..33c6d02a47 --- /dev/null +++ b/ratis-server-api/dev-support/findbugsExcludeFile.xml @@ -0,0 +1,34 @@ + + + + + + + + + + + + + + + + + + + \ No newline at end of file diff --git a/ratis-server-api/pom.xml b/ratis-server-api/pom.xml index 0dfb3cc1d1..feed49190c 100644 --- a/ratis-server-api/pom.xml +++ b/ratis-server-api/pom.xml @@ -64,4 +64,15 @@ test + + + + com.github.spotbugs + spotbugs-maven-plugin + + ${basedir}/dev-support/findbugsExcludeFile.xml + + + + diff --git a/ratis-server/dev-support/findbugsExcludeFile.xml b/ratis-server/dev-support/findbugsExcludeFile.xml new file mode 100644 index 0000000000..9180856202 --- /dev/null +++ b/ratis-server/dev-support/findbugsExcludeFile.xml @@ -0,0 +1,90 @@ + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + \ No newline at end of file diff --git a/ratis-server/pom.xml b/ratis-server/pom.xml index f700ac6c5d..2c0bc93a25 100644 --- a/ratis-server/pom.xml +++ b/ratis-server/pom.xml @@ -94,4 +94,15 @@ test + + + + com.github.spotbugs + spotbugs-maven-plugin + + ${basedir}/dev-support/findbugsExcludeFile.xml + + + + diff --git a/ratis-shell/dev-support/findbugsExcludeFile.xml b/ratis-shell/dev-support/findbugsExcludeFile.xml new file mode 100644 index 0000000000..c29ededa06 --- /dev/null +++ b/ratis-shell/dev-support/findbugsExcludeFile.xml @@ -0,0 +1,30 @@ + + + + + + + + + + + + + + + \ No newline at end of file diff --git a/ratis-shell/pom.xml b/ratis-shell/pom.xml index 32361a46c3..efa96a7866 100644 --- a/ratis-shell/pom.xml +++ b/ratis-shell/pom.xml @@ -85,6 +85,13 @@ + + com.github.spotbugs + spotbugs-maven-plugin + + ${basedir}/dev-support/findbugsExcludeFile.xml + + From c899285ccd9c565fe8a5d4ac6cec72031bc5f8c6 Mon Sep 17 00:00:00 2001 From: "Doroszlai, Attila" <6454655+adoroszlai@users.noreply.github.com> Date: Fri, 14 Nov 2025 08:52:16 +0100 Subject: [PATCH 227/397] RATIS-2359. Use JDK 17 for CI checks (#1315) --- .github/workflows/check.yaml | 4 ++-- .github/workflows/ci.yaml | 2 +- 2 files changed, 3 insertions(+), 3 deletions(-) diff --git a/.github/workflows/check.yaml b/.github/workflows/check.yaml index 1655d60148..81651c1fa7 100644 --- a/.github/workflows/check.yaml +++ b/.github/workflows/check.yaml @@ -30,8 +30,8 @@ on: # OPTIONAL (ordered alphabetically) java-version: type: string - description: "Java version to set up (default: 8)" - default: '8' + description: "Java version to set up (default: 17)" + default: '17' required: false needs-binary-tarball: diff --git a/.github/workflows/ci.yaml b/.github/workflows/ci.yaml index 20704f5ea9..6ef2b0c66d 100644 --- a/.github/workflows/ci.yaml +++ b/.github/workflows/ci.yaml @@ -38,7 +38,7 @@ jobs: - build strategy: matrix: - java: [ 11, 17, 21 ] + java: [ 8, 11, 17, 21 ] fail-fast: false uses: ./.github/workflows/check.yaml with: From 9e058dd362973c1e43a1d7edea051e9207e85ad9 Mon Sep 17 00:00:00 2001 From: Tsz-Wo Nicholas Sze Date: Fri, 14 Nov 2025 11:12:51 -0800 Subject: [PATCH 228/397] RATIS-2350. Fix readAfterWrite bugs. (#1311) --- .../ratis/server/impl/LeaderStateImpl.java | 8 +-- .../ratis/server/impl/ReadRequests.java | 50 ++++++++++++------- .../apache/ratis/server/impl/ServerState.java | 2 +- .../server/impl/StateMachineUpdater.java | 5 +- .../ratis/server/impl/WriteIndexCache.java | 5 +- .../apache/ratis/LinearizableReadTests.java | 23 +++++---- 6 files changed, 56 insertions(+), 37 deletions(-) diff --git a/ratis-server/src/main/java/org/apache/ratis/server/impl/LeaderStateImpl.java b/ratis-server/src/main/java/org/apache/ratis/server/impl/LeaderStateImpl.java index 0835802bd1..836b15bcdb 100644 --- a/ratis-server/src/main/java/org/apache/ratis/server/impl/LeaderStateImpl.java +++ b/ratis-server/src/main/java/org/apache/ratis/server/impl/LeaderStateImpl.java @@ -1140,13 +1140,15 @@ public boolean checkLeadership() { * @return current readIndex. */ CompletableFuture getReadIndex(Long readAfterWriteConsistentIndex) { + final long commitIndex = server.getRaftLog().getLastCommittedIndex(); final long readIndex; - if (readAfterWriteConsistentIndex != null) { + if (readAfterWriteConsistentIndex != null && readAfterWriteConsistentIndex > commitIndex) { readIndex = readAfterWriteConsistentIndex; } else { - readIndex = server.getRaftLog().getLastCommittedIndex(); + readIndex = commitIndex; } - LOG.debug("readIndex={}, readAfterWriteConsistentIndex={}", readIndex, readAfterWriteConsistentIndex); + LOG.debug("readIndex={} (commitIndex={}, readAfterWriteConsistentIndex={})", + readIndex, commitIndex, readAfterWriteConsistentIndex); // if group contains only one member, fast path if (server.getRaftConf().isSingleton()) { diff --git a/ratis-server/src/main/java/org/apache/ratis/server/impl/ReadRequests.java b/ratis-server/src/main/java/org/apache/ratis/server/impl/ReadRequests.java index e63a23a0b8..6112a46009 100644 --- a/ratis-server/src/main/java/org/apache/ratis/server/impl/ReadRequests.java +++ b/ratis-server/src/main/java/org/apache/ratis/server/impl/ReadRequests.java @@ -20,7 +20,7 @@ import org.apache.ratis.conf.RaftProperties; import org.apache.ratis.protocol.exceptions.ReadException; import org.apache.ratis.server.RaftServerConfigKeys; -import org.apache.ratis.statemachine.StateMachine; +import org.apache.ratis.util.Preconditions; import org.apache.ratis.util.TimeDuration; import org.apache.ratis.util.TimeoutExecutor; import org.slf4j.Logger; @@ -29,7 +29,7 @@ import java.util.NavigableMap; import java.util.TreeMap; import java.util.concurrent.CompletableFuture; -import java.util.function.Consumer; +import java.util.function.LongConsumer; /** For supporting linearizable read. */ class ReadRequests { @@ -37,10 +37,18 @@ class ReadRequests { static class ReadIndexQueue { private final TimeoutExecutor scheduler = TimeoutExecutor.getInstance(); + /** The log index known to be applied. */ + private long lastAppliedIndex; + /** + * Map : readIndex -> appliedIndexFuture (when completes, readIndex <= appliedIndex). + * Invariant: all keys > lastAppliedIndex. + */ private final NavigableMap> sorted = new TreeMap<>(); + private final TimeDuration readTimeout; - ReadIndexQueue(TimeDuration readTimeout) { + ReadIndexQueue(long lastAppliedIndex, TimeDuration readTimeout) { + this.lastAppliedIndex = lastAppliedIndex; this.readTimeout = readTimeout; } @@ -48,6 +56,9 @@ CompletableFuture add(long readIndex) { final CompletableFuture returned; final boolean create; synchronized (this) { + if (readIndex <= lastAppliedIndex) { + return CompletableFuture.completedFuture(lastAppliedIndex); + } // The same as computeIfAbsent except that it also tells if a new value is created. final CompletableFuture existing = sorted.get(readIndex); create = existing == null; @@ -79,7 +90,19 @@ private void handleTimeout(long readIndex) { /** Complete all the entries less than or equal to the given applied index. */ - synchronized void complete(Long appliedIndex) { + synchronized void complete(long appliedIndex) { + if (appliedIndex > lastAppliedIndex) { + lastAppliedIndex = appliedIndex; + } else { + // appliedIndex <= lastAppliedIndex: nothing to do + if (!sorted.isEmpty()) { + // Assert: all keys > lastAppliedIndex. + final long first = sorted.firstKey(); + Preconditions.assertTrue(first > lastAppliedIndex, + () -> "first = " + first + " <= lastAppliedIndex = " + lastAppliedIndex); + } + return; + } final NavigableMap> headMap = sorted.headMap(appliedIndex, true); headMap.values().forEach(f -> f.complete(appliedIndex)); headMap.clear(); @@ -87,27 +110,16 @@ synchronized void complete(Long appliedIndex) { } private final ReadIndexQueue readIndexQueue; - private final StateMachine stateMachine; - ReadRequests(RaftProperties properties, StateMachine stateMachine) { - this.readIndexQueue = new ReadIndexQueue(RaftServerConfigKeys.Read.timeout(properties)); - this.stateMachine = stateMachine; + ReadRequests(long appliedIndex, RaftProperties properties) { + this.readIndexQueue = new ReadIndexQueue(appliedIndex, RaftServerConfigKeys.Read.timeout(properties)); } - Consumer getAppliedIndexConsumer() { + LongConsumer getAppliedIndexConsumer() { return readIndexQueue::complete; } CompletableFuture waitToAdvance(long readIndex) { - final long lastApplied = stateMachine.getLastAppliedTermIndex().getIndex(); - if (lastApplied >= readIndex) { - return CompletableFuture.completedFuture(lastApplied); - } - final CompletableFuture f = readIndexQueue.add(readIndex); - final long current = stateMachine.getLastAppliedTermIndex().getIndex(); - if (current > lastApplied) { - readIndexQueue.complete(current); - } - return f; + return readIndexQueue.add(readIndex); } } diff --git a/ratis-server/src/main/java/org/apache/ratis/server/impl/ServerState.java b/ratis-server/src/main/java/org/apache/ratis/server/impl/ServerState.java index 05afc09756..ee1b7d37b7 100644 --- a/ratis-server/src/main/java/org/apache/ratis/server/impl/ServerState.java +++ b/ratis-server/src/main/java/org/apache/ratis/server/impl/ServerState.java @@ -125,7 +125,7 @@ class ServerState { this.lastNoLeaderTime = new AtomicReference<>(Timestamp.currentTime()); this.noLeaderTimeout = RaftServerConfigKeys.Notification.noLeaderTimeout(prop); this.log = JavaUtils.memoize(() -> initRaftLog(() -> getSnapshotIndexFromStateMachine(stateMachine), prop)); - this.readRequests = new ReadRequests(prop, stateMachine); + this.readRequests = new ReadRequests(stateMachine.getLastAppliedTermIndex().getIndex(), prop); this.stateMachineUpdater = JavaUtils.memoize(() -> new StateMachineUpdater( stateMachine, server, this, getLog().getSnapshotIndex(), prop, this.readRequests.getAppliedIndexConsumer())); diff --git a/ratis-server/src/main/java/org/apache/ratis/server/impl/StateMachineUpdater.java b/ratis-server/src/main/java/org/apache/ratis/server/impl/StateMachineUpdater.java index bd7f26a8a2..041693195f 100644 --- a/ratis-server/src/main/java/org/apache/ratis/server/impl/StateMachineUpdater.java +++ b/ratis-server/src/main/java/org/apache/ratis/server/impl/StateMachineUpdater.java @@ -45,6 +45,7 @@ import java.util.concurrent.TimeUnit; import java.util.concurrent.atomic.AtomicReference; import java.util.function.Consumer; +import java.util.function.LongConsumer; import java.util.stream.LongStream; /** @@ -90,12 +91,12 @@ enum State { private final MemoizedSupplier stateMachineMetrics; - private final Consumer appliedIndexConsumer; + private final LongConsumer appliedIndexConsumer; private volatile boolean isRemoving; StateMachineUpdater(StateMachine stateMachine, RaftServerImpl server, - ServerState serverState, long lastAppliedIndex, RaftProperties properties, Consumer appliedIndexConsumer) { + ServerState serverState, long lastAppliedIndex, RaftProperties properties, LongConsumer appliedIndexConsumer) { this.name = ServerStringUtils.generateUnifiedName(serverState.getMemberId(), getClass()); this.appliedIndexConsumer = appliedIndexConsumer; this.infoIndexChange = s -> LOG.info("{}: {}", name, s); diff --git a/ratis-server/src/main/java/org/apache/ratis/server/impl/WriteIndexCache.java b/ratis-server/src/main/java/org/apache/ratis/server/impl/WriteIndexCache.java index df4448622f..98250ca229 100644 --- a/ratis-server/src/main/java/org/apache/ratis/server/impl/WriteIndexCache.java +++ b/ratis-server/src/main/java/org/apache/ratis/server/impl/WriteIndexCache.java @@ -46,14 +46,15 @@ class WriteIndexCache { .build(); } - void add(ClientId key, CompletableFuture future) { + void add(ClientId key, CompletableFuture current) { final AtomicReference> ref; try { ref = cache.get(key, AtomicReference::new); } catch (ExecutionException e) { throw new IllegalStateException(e); } - ref.set(future); + ref.updateAndGet(previous -> previous == null ? current + : previous.thenCombine(current, Math::max)); } CompletableFuture getWriteIndexFuture(RaftClientRequest request) { diff --git a/ratis-server/src/test/java/org/apache/ratis/LinearizableReadTests.java b/ratis-server/src/test/java/org/apache/ratis/LinearizableReadTests.java index 49176b18a8..91bd2f28d6 100644 --- a/ratis-server/src/test/java/org/apache/ratis/LinearizableReadTests.java +++ b/ratis-server/src/test/java/org/apache/ratis/LinearizableReadTests.java @@ -47,7 +47,6 @@ import static org.apache.ratis.ReadOnlyRequestTests.WAIT_AND_INCREMENT; import static org.apache.ratis.ReadOnlyRequestTests.assertReplyAtLeast; import static org.apache.ratis.ReadOnlyRequestTests.assertReplyExact; -import static org.apache.ratis.ReadOnlyRequestTests.retrieve; import static org.apache.ratis.server.RaftServerConfigKeys.Read.Option.LINEARIZABLE; /** Test for the {@link RaftServerConfigKeys.Read.Option#LINEARIZABLE} feature. */ @@ -233,20 +232,24 @@ static void runTestReadAfterWrite(C cluster) throws assertReplyExact(1, client.io().sendReadAfterWrite(QUERY)); // test asynchronous read-after-write - client.async().send(INCREMENT); + final CompletableFuture writeReply = client.async().send(INCREMENT); final CompletableFuture asyncReply = client.async().sendReadAfterWrite(QUERY); - for (int i = 0; i < 20; i++) { - client.async().send(INCREMENT); + final int n = 100; + final List writeReplies = new ArrayList<>(n); + final List readAfterWriteReplies = new ArrayList<>(n); + for (int i = 0; i < n; i++) { + final int count = i + 3; + writeReplies.add(new Reply(count, client.async().send(INCREMENT))); + readAfterWriteReplies.add(new Reply(count, client.async().sendReadAfterWrite(QUERY))); } - // read-after-write is more consistent than linearizable read - final CompletableFuture linearizable = client.async().sendReadOnly(QUERY); - final CompletableFuture readAfterWrite = client.async().sendReadAfterWrite(QUERY); - final int r = retrieve(readAfterWrite.get()); - final int l = retrieve(linearizable.get()); - Assertions.assertTrue(r >= l, () -> "readAfterWrite = " + r + " < linearizable = " + l); + for (int i = 0; i < n; i++) { + writeReplies.get(i).assertExact(); + readAfterWriteReplies.get(i).assertAtLeast(); + } + assertReplyAtLeast(2, writeReply.join()); assertReplyAtLeast(2, asyncReply.join()); } } From f6e9c71a34b01025f3df2b3a0e0409d746a2f4cd Mon Sep 17 00:00:00 2001 From: Symious <14933944+symious@users.noreply.github.com> Date: Tue, 18 Nov 2025 00:42:09 +0800 Subject: [PATCH 229/397] RATIS-2325. Create GrpcStubPool for GrpcServerProtocolClient (#1306) --- .../org/apache/ratis/grpc/GrpcConfigKeys.java | 9 ++ .../grpc/server/GrpcServerProtocolClient.java | 51 +++++++- .../ratis/grpc/server/GrpcServicesImpl.java | 4 +- .../ratis/grpc/server/GrpcStubPool.java | 121 ++++++++++++++++++ 4 files changed, 181 insertions(+), 4 deletions(-) create mode 100644 ratis-grpc/src/main/java/org/apache/ratis/grpc/server/GrpcStubPool.java diff --git a/ratis-grpc/src/main/java/org/apache/ratis/grpc/GrpcConfigKeys.java b/ratis-grpc/src/main/java/org/apache/ratis/grpc/GrpcConfigKeys.java index f21a9b99f1..f31794ac36 100644 --- a/ratis-grpc/src/main/java/org/apache/ratis/grpc/GrpcConfigKeys.java +++ b/ratis-grpc/src/main/java/org/apache/ratis/grpc/GrpcConfigKeys.java @@ -282,6 +282,15 @@ static GrpcTlsConfig tlsConf(Parameters parameters) { static void setTlsConf(Parameters parameters, GrpcTlsConfig conf) { parameters.put(TLS_CONF_PARAMETER, conf, TLS_CONF_CLASS); } + + String STUB_POOL_SIZE_KEY = PREFIX + ".stub.pool.size"; + int STUB_POOL_SIZE_DEFAULT = 1; + static int stubPoolSize(RaftProperties properties) { + return get(properties::getInt, STUB_POOL_SIZE_KEY, STUB_POOL_SIZE_DEFAULT, getDefaultLog()); + } + static void setStubPoolSize(RaftProperties properties, int size) { + setInt(properties::setInt, STUB_POOL_SIZE_KEY, size); + } } String MESSAGE_SIZE_MAX_KEY = PREFIX + ".message.size.max"; diff --git a/ratis-grpc/src/main/java/org/apache/ratis/grpc/server/GrpcServerProtocolClient.java b/ratis-grpc/src/main/java/org/apache/ratis/grpc/server/GrpcServerProtocolClient.java index 1e40a75ada..d2748c7be2 100644 --- a/ratis-grpc/src/main/java/org/apache/ratis/grpc/server/GrpcServerProtocolClient.java +++ b/ratis-grpc/src/main/java/org/apache/ratis/grpc/server/GrpcServerProtocolClient.java @@ -45,6 +45,7 @@ class GrpcServerProtocolClient implements Closeable { // Common channel private final ManagedChannel channel; + private final GrpcStubPool pool; // Channel and stub for heartbeat private ManagedChannel hbChannel; private RaftServerProtocolServiceStub hbAsyncStub; @@ -57,7 +58,7 @@ class GrpcServerProtocolClient implements Closeable { //visible for using in log / error messages AND to use in instrumented tests private final RaftPeerId raftPeerId; - GrpcServerProtocolClient(RaftPeer target, int flowControlWindow, + GrpcServerProtocolClient(RaftPeer target, int connections, int flowControlWindow, TimeDuration requestTimeout, SslContext sslContext, boolean separateHBChannel) { raftPeerId = target.getId(); LOG.info("Build channel for {}", target); @@ -70,6 +71,11 @@ class GrpcServerProtocolClient implements Closeable { hbAsyncStub = RaftServerProtocolServiceGrpc.newStub(hbChannel); } requestTimeoutDuration = requestTimeout; + this.pool = connections == 1? null : newGrpcStubPool(target.getAddress(), sslContext, connections); + } + + GrpcStubPool newGrpcStubPool(String address, SslContext sslContext, int connections) { + return new GrpcStubPool<>(connections, address, sslContext, RaftServerProtocolServiceGrpc::newStub, 16); } private ManagedChannel buildChannel(RaftPeer target, int flowControlWindow, SslContext sslContext) { @@ -94,6 +100,9 @@ public void close() { GrpcUtil.shutdownManagedChannel(hbChannel); } GrpcUtil.shutdownManagedChannel(channel); + if (pool != null) { + pool.close(); + } } public RequestVoteReplyProto requestVote(RequestVoteRequestProto request) { @@ -112,8 +121,44 @@ public StartLeaderElectionReplyProto startLeaderElection(StartLeaderElectionRequ } void readIndex(ReadIndexRequestProto request, StreamObserver s) { - asyncStub.withDeadlineAfter(requestTimeoutDuration.getDuration(), requestTimeoutDuration.getUnit()) - .readIndex(request, s); + if (pool == null) { + asyncStub.withDeadlineAfter(requestTimeoutDuration.getDuration(), requestTimeoutDuration.getUnit()) + .readIndex(request, s); + } else { + GrpcStubPool.Stub p; + try { + p = pool.acquire(); + } catch (InterruptedException e) { + Thread.currentThread().interrupt(); + s.onError(e); + return; + } + p.getStub().withDeadlineAfter(requestTimeoutDuration.getDuration(), requestTimeoutDuration.getUnit()) + .readIndex(request, new StreamObserver() { + @Override + public void onNext(ReadIndexReplyProto v) { + s.onNext(v); + } + + @Override + public void onError(Throwable t) { + try { + s.onError(t); + } finally { + p.release(); + } + } + + @Override + public void onCompleted() { + try { + s.onCompleted(); + } finally { + p.release(); + } + } + }); + } } CallStreamObserver appendEntries( diff --git a/ratis-grpc/src/main/java/org/apache/ratis/grpc/server/GrpcServicesImpl.java b/ratis-grpc/src/main/java/org/apache/ratis/grpc/server/GrpcServicesImpl.java index 8200aa3ef7..b1af0960dc 100644 --- a/ratis-grpc/src/main/java/org/apache/ratis/grpc/server/GrpcServicesImpl.java +++ b/ratis-grpc/src/main/java/org/apache/ratis/grpc/server/GrpcServicesImpl.java @@ -108,6 +108,7 @@ public static final class Builder { private int serverPort; private SslContext serverSslContextForServer; private SslContext serverSslContextForClient; + private int serverStubPoolSize; private SizeInBytes messageSizeMax; private SizeInBytes flowControlWindow; @@ -130,6 +131,7 @@ public Builder setServer(RaftServer raftServer) { this.flowControlWindow = GrpcConfigKeys.flowControlWindow(properties, LOG::info); this.requestTimeoutDuration = RaftServerConfigKeys.Rpc.requestTimeout(properties); this.separateHeartbeatChannel = GrpcConfigKeys.Server.heartbeatChannel(properties); + this.serverStubPoolSize = GrpcConfigKeys.Server.stubPoolSize(properties); final SizeInBytes appenderBufferSize = RaftServerConfigKeys.Log.Appender.bufferByteLimit(properties); final SizeInBytes gap = SizeInBytes.ONE_MB; @@ -150,7 +152,7 @@ public Builder setCustomizer(Customizer customizer) { } private GrpcServerProtocolClient newGrpcServerProtocolClient(RaftPeer target) { - return new GrpcServerProtocolClient(target, flowControlWindow.getSizeInt(), + return new GrpcServerProtocolClient(target, serverStubPoolSize, flowControlWindow.getSizeInt(), requestTimeoutDuration, serverSslContextForClient, separateHeartbeatChannel); } diff --git a/ratis-grpc/src/main/java/org/apache/ratis/grpc/server/GrpcStubPool.java b/ratis-grpc/src/main/java/org/apache/ratis/grpc/server/GrpcStubPool.java new file mode 100644 index 0000000000..fd27ac996a --- /dev/null +++ b/ratis-grpc/src/main/java/org/apache/ratis/grpc/server/GrpcStubPool.java @@ -0,0 +1,121 @@ +/* + * 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.ratis.grpc.server; + +import org.apache.ratis.thirdparty.io.grpc.ManagedChannel; +import org.apache.ratis.thirdparty.io.grpc.netty.NegotiationType; +import org.apache.ratis.thirdparty.io.grpc.netty.NettyChannelBuilder; +import org.apache.ratis.thirdparty.io.grpc.stub.AbstractStub; +import org.apache.ratis.thirdparty.io.netty.channel.ChannelOption; +import org.apache.ratis.thirdparty.io.netty.channel.WriteBufferWaterMark; +import org.apache.ratis.thirdparty.io.netty.handler.ssl.SslContext; +import org.apache.ratis.util.MemoizedSupplier; +import org.apache.ratis.util.Preconditions; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.util.ArrayList; +import java.util.Collections; +import java.util.List; +import java.util.concurrent.Semaphore; +import java.util.concurrent.ThreadLocalRandom; +import java.util.concurrent.TimeUnit; +import java.util.function.Function; + +final class GrpcStubPool> { + public static final Logger LOG = LoggerFactory.getLogger(GrpcStubPool.class); + + static ManagedChannel buildManagedChannel(String address, SslContext sslContext) { + NettyChannelBuilder channelBuilder = NettyChannelBuilder.forTarget(address) + .keepAliveTime(10, TimeUnit.MINUTES) + .keepAliveWithoutCalls(false) + .idleTimeout(30, TimeUnit.MINUTES) + .withOption(ChannelOption.WRITE_BUFFER_WATER_MARK, new WriteBufferWaterMark(64 << 10, 128 << 10)); + if (sslContext != null) { + LOG.debug("Setting TLS for {}", address); + channelBuilder.useTransportSecurity().sslContext(sslContext); + } else { + channelBuilder.negotiationType(NegotiationType.PLAINTEXT); + } + ManagedChannel ch = channelBuilder.build(); + ch.getState(true); + return ch; + } + + static final class Stub> { + private final ManagedChannel ch; + private final S stub; + private final Semaphore permits; + + Stub(String address, SslContext sslContext, Function stubFactory, int maxInflight) { + this.ch = buildManagedChannel(address, sslContext); + this.stub = stubFactory.apply(ch); + this.permits = new Semaphore(maxInflight); + } + + S getStub() { + return stub; + } + + void release() { + permits.release(); + } + + void shutdown() { + ch.shutdown(); + } + } + + private final List>> pool; + + GrpcStubPool(int connections, String address, SslContext sslContext, Function stubFactory, + int maxInflightPerConn) { + Preconditions.assertTrue(connections > 1, "connections must be > 1"); + final List>> tmpPool = new ArrayList<>(connections); + for (int i = 0; i < connections; i++) { + tmpPool.add(MemoizedSupplier.valueOf(() -> new Stub<>(address, sslContext, stubFactory, maxInflightPerConn))); + } + this.pool = Collections.unmodifiableList(tmpPool); + } + + Stub getStub(int i) { + return pool.get(i).get(); + } + + Stub acquire() throws InterruptedException { + final int size = pool.size(); + final int start = ThreadLocalRandom.current().nextInt(size); + for (int k = 0; k < size; k++) { + Stub p = getStub((start + k) % size); + if (p.permits.tryAcquire()) { + return p; + } + } + final Stub p = getStub(start); + p.permits.acquire(); + return p; + } + + public void close() { + for (MemoizedSupplier> p : pool) { + if (p.isInitialized()) { + p.get().shutdown(); + } + } + } +} From f06d8c36c1efa1750d37b84f8d38f3b5a0b2b9a1 Mon Sep 17 00:00:00 2001 From: "Doroszlai, Attila" <6454655+adoroszlai@users.noreply.github.com> Date: Wed, 19 Nov 2025 07:02:51 +0100 Subject: [PATCH 230/397] RATIS-2362. Use maven.compiler.release for target Java version (#1317) --- pom.xml | 66 +++++---------------------------------------------------- 1 file changed, 5 insertions(+), 61 deletions(-) diff --git a/pom.xml b/pom.xml index 718a1a3e27..b702cbcd3a 100644 --- a/pom.xml +++ b/pom.xml @@ -177,10 +177,9 @@ bash - 8 - ${javac.version} - ${javac.version} - 3.3.9 + 8 + + ${maven.compiler.release} 1.0.10 @@ -470,16 +469,6 @@ org.apache.maven.plugins maven-enforcer-plugin - - - - [${maven.min.version},) - - - [${java.min.version},) - - - de.skuzzle.enforcer @@ -654,49 +643,6 @@ com.github.spotbugs spotbugs-maven-plugin - - org.apache.maven.plugins - maven-enforcer-plugin - false - - - clean - - enforce - - pre-clean - - - default - - enforce - - validate - - - - - [${maven.min.version},) - Maven is out of date. - Ratis requires at least version ${maven.min.version} of Maven to properly build from source. - You appear to be using an older version. You can use either "mvn -version" or - "mvn enforcer:display-info" to verify what version is active. - - - - - [${java.min.version},) - Java is out of date. - Ratis requires at least version ${java.min.version} of the JDK to properly build from source. - You appear to be using an older version. You can use either "mvn -version" or - "mvn enforcer:display-info" to verify what version is active. - - - - - - - org.apache.felix maven-bundle-plugin @@ -919,13 +865,11 @@ - ${java.min.version} + ${maven.compiler.release} Ratis has unsupported dependencies. - Ratis requires that all dependencies be compiled with version ${java.min.version} or earlier + Ratis requires that all dependencies be compiled with version ${maven.compiler.release} or earlier of the JDK to properly build from source. You appear to be using a newer dependency. You can use either "mvn -version" or "mvn enforcer:display-info" to verify what version is active. - Non-release builds can temporarily build with a newer JDK version by setting the - 'javac.source' property (eg. mvn -Djavac.source=1.8 clean package). From db7ced016050fc77d84ed98e9a228be54c84b943 Mon Sep 17 00:00:00 2001 From: Tsz-Wo Nicholas Sze Date: Thu, 20 Nov 2025 07:43:41 -0800 Subject: [PATCH 231/397] RATIS-2361. Change MD5Hash to value-based. (#1316) --- .../dev-support/findbugsExcludeFile.xml | 4 - .../java/org/apache/ratis/io/MD5Hash.java | 209 +++++------------- .../org/apache/ratis/util/MD5FileUtil.java | 29 ++- .../java/org/apache/ratis/io/TestMD5.java | 69 ++++++ .../ratis/server/storage/FileChunkReader.java | 4 +- .../ratis/server/storage/SnapshotManager.java | 13 +- 6 files changed, 150 insertions(+), 178 deletions(-) create mode 100644 ratis-common/src/test/java/org/apache/ratis/io/TestMD5.java diff --git a/ratis-common/dev-support/findbugsExcludeFile.xml b/ratis-common/dev-support/findbugsExcludeFile.xml index 3148752344..9267e763fa 100644 --- a/ratis-common/dev-support/findbugsExcludeFile.xml +++ b/ratis-common/dev-support/findbugsExcludeFile.xml @@ -19,10 +19,6 @@ - - - - diff --git a/ratis-common/src/main/java/org/apache/ratis/io/MD5Hash.java b/ratis-common/src/main/java/org/apache/ratis/io/MD5Hash.java index 0d19feb938..71fd39f34b 100644 --- a/ratis-common/src/main/java/org/apache/ratis/io/MD5Hash.java +++ b/ratis-common/src/main/java/org/apache/ratis/io/MD5Hash.java @@ -1,4 +1,4 @@ -/** +/* * 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 @@ -18,157 +18,73 @@ package org.apache.ratis.io; -import java.io.DataInput; -import java.io.DataOutput; -import java.io.IOException; -import java.io.InputStream; -import java.security.MessageDigest; -import java.security.NoSuchAlgorithmException; -import java.util.Arrays; - -public class MD5Hash { - public static final int MD5_LEN = 16; - - private static final ThreadLocal DIGESTER_FACTORY = - ThreadLocal.withInitial(MD5Hash::newDigester); - - public static MessageDigest newDigester() { - try { - return MessageDigest.getInstance("MD5"); - } catch (NoSuchAlgorithmException e) { - throw new IllegalStateException("Failed to create MessageDigest for MD5", e); - } - } - - private byte[] digest; - - /** Constructs an MD5Hash. */ - public MD5Hash() { - this.digest = new byte[MD5_LEN]; - } - - /** Constructs an MD5Hash from a hex string. */ - public MD5Hash(String hex) { - setDigest(hex); - } - - /** Constructs an MD5Hash with a specified value. */ - public MD5Hash(byte[] digest) { - if (digest.length != MD5_LEN) { - throw new IllegalArgumentException("Wrong length: " + digest.length); - } - this.digest = digest.clone(); - } - - public void readFields(DataInput in) throws IOException { - in.readFully(digest); - } - - /** Constructs, reads and returns an instance. */ - public static MD5Hash read(DataInput in) throws IOException { - MD5Hash result = new MD5Hash(); - result.readFields(in); - return result; - } - - public void write(DataOutput out) throws IOException { - out.write(digest); - } - - /** Copy the contents of another instance into this instance. */ - public void set(MD5Hash that) { - System.arraycopy(that.digest, 0, this.digest, 0, MD5_LEN); - } - - /** Returns the digest bytes. */ - public byte[] getDigest() { - return digest.clone(); - } - - /** Construct a hash value for a byte array. */ - public static MD5Hash digest(byte[] data) { - return digest(data, 0, data.length); - } - - /** - * Create a thread local MD5 digester - */ - public static MessageDigest getDigester() { - MessageDigest digester = DIGESTER_FACTORY.get(); - digester.reset(); - return digester; - } +import org.apache.ratis.util.MemoizedSupplier; +import org.apache.ratis.util.Preconditions; - /** Construct a hash value for the content from the InputStream. */ - public static MD5Hash digest(InputStream in) throws IOException { - final byte[] buffer = new byte[4*1024]; +import java.nio.ByteBuffer; +import java.util.Arrays; +import java.util.Objects; +import java.util.function.Supplier; - final MessageDigest digester = getDigester(); - for(int n; (n = in.read(buffer)) != -1; ) { - digester.update(buffer, 0, n); +/** + * A MD5 hash value. + *

+ * This is a value-based class. + */ +public final class MD5Hash { + public static final int MD5_LENGTH = 16; + + /** @return an instance with the given digest in a (case-insensitive) hexadecimals. */ + public static MD5Hash newInstance(String digestHexadecimals) { + Objects.requireNonNull(digestHexadecimals, "digestHexadecimals == null"); + Preconditions.assertSame(2 * MD5_LENGTH, digestHexadecimals.length(), "digestHexadecimals"); + + final byte[] digest = new byte[MD5_LENGTH]; + for (int i = 0; i < MD5_LENGTH; i++) { + final int j = i << 1; + digest[i] = (byte) (charToNibble(digestHexadecimals, j) << 4 | + charToNibble(digestHexadecimals, j + 1)); } - - return new MD5Hash(digester.digest()); - } - - /** Construct a hash value for a byte array. */ - public static MD5Hash digest(byte[] data, int start, int len) { - byte[] digest; - MessageDigest digester = getDigester(); - digester.update(data, start, len); - digest = digester.digest(); return new MD5Hash(digest); } - /** Construct a hash value for an array of byte array. */ - public static MD5Hash digest(byte[][] dataArr, int start, int len) { - byte[] digest; - MessageDigest digester = getDigester(); - for (byte[] data : dataArr) { - digester.update(data, start, len); - } - digest = digester.digest(); - return new MD5Hash(digest); + /** @return an instance with the given digest. */ + public static MD5Hash newInstance(byte[] digest) { + Objects.requireNonNull(digest, "digest == null"); + Preconditions.assertSame(MD5_LENGTH, digest.length, "digest"); + return new MD5Hash(digest.clone()); } - /** Construct a half-sized version of this MD5. Fits in a long **/ - public long halfDigest() { - long value = 0; - for (int i = 0; i < 8; i++) { - value |= ((digest[i] & 0xffL) << (8*(7-i))); - } - return value; + private final byte[] digest; + private final Supplier digestString; + + private MD5Hash(byte[] digest) { + this.digest = digest; + this.digestString = MemoizedSupplier.valueOf(() -> digestToString(digest)); } - /** - * Return a 32-bit digest of the MD5. - * @return the first 4 bytes of the md5 - */ - public int quarterDigest() { - int value = 0; - for (int i = 0; i < 4; i++) { - value |= ((digest[i] & 0xff) << (8*(3-i))); - } - return value; + /** @return the digest wrapped by a read-only {@link ByteBuffer}. */ + public ByteBuffer getDigest() { + return ByteBuffer.wrap(digest).asReadOnlyBuffer(); } - /** Returns true iff o is an MD5Hash whose digest contains the - * same values. */ @Override - public boolean equals(Object o) { - if (!(o instanceof MD5Hash)) { + public boolean equals(Object object) { + if (this == object) { + return true; + } else if (!(object instanceof MD5Hash)) { return false; } - MD5Hash other = (MD5Hash)o; - return Arrays.equals(this.digest, other.digest); + final MD5Hash that = (MD5Hash) object; + return Arrays.equals(this.digest, that.digest); } - /** Returns a hash code value for this object. - * Only uses the first 4 bytes, since md5s are evenly distributed. - */ @Override public int hashCode() { - return quarterDigest(); + return ((digest[0] & 0xFF) << 24) + | ((digest[1] & 0xFF) << 16) + | ((digest[2] & 0xFF) << 8) + | (digest[3] & 0xFF); } private static final char[] HEX_DIGITS = @@ -177,8 +93,12 @@ public int hashCode() { /** Returns a string representation of this object. */ @Override public String toString() { - StringBuilder buf = new StringBuilder(MD5_LEN*2); - for (int i = 0; i < MD5_LEN; i++) { + return digestString.get(); + } + + static String digestToString(byte[] digest) { + StringBuilder buf = new StringBuilder(MD5_LENGTH *2); + for (int i = 0; i < MD5_LENGTH; i++) { int b = digest[i]; buf.append(HEX_DIGITS[(b >> 4) & 0xf]); buf.append(HEX_DIGITS[b & 0xf]); @@ -186,20 +106,8 @@ public String toString() { return buf.toString(); } - /** Sets the digest value from a hex string. */ - public void setDigest(String hex) { - if (hex.length() != MD5_LEN*2) { - throw new IllegalArgumentException("Wrong length: " + hex.length()); - } - this.digest = new byte[MD5_LEN]; - for (int i = 0; i < MD5_LEN; i++) { - int j = i << 1; - this.digest[i] = (byte)(charToNibble(hex.charAt(j)) << 4 | - charToNibble(hex.charAt(j+1))); - } - } - - private static int charToNibble(char c) { + private static int charToNibble(String hexadecimals, int i) { + final char c = hexadecimals.charAt(i); if (c >= '0' && c <= '9') { return c - '0'; } else if (c >= 'a' && c <= 'f') { @@ -207,7 +115,8 @@ private static int charToNibble(char c) { } else if (c >= 'A' && c <= 'F') { return 0xA + (c - 'A'); } else { - throw new RuntimeException("Not a hex character: " + c); + throw new IllegalArgumentException( + "Found a non-hexadecimal character '" + c + "' at index " + i + " in \"" + hexadecimals + "\""); } } } diff --git a/ratis-common/src/main/java/org/apache/ratis/util/MD5FileUtil.java b/ratis-common/src/main/java/org/apache/ratis/util/MD5FileUtil.java index 8a38f45e67..2c217b27d4 100644 --- a/ratis-common/src/main/java/org/apache/ratis/util/MD5FileUtil.java +++ b/ratis-common/src/main/java/org/apache/ratis/util/MD5FileUtil.java @@ -18,8 +18,6 @@ package org.apache.ratis.util; import org.apache.ratis.io.MD5Hash; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; import java.io.BufferedReader; import java.io.File; @@ -29,16 +27,21 @@ import java.nio.charset.StandardCharsets; import java.nio.file.StandardOpenOption; import java.security.MessageDigest; +import java.security.NoSuchAlgorithmException; import java.util.Optional; import java.util.regex.Matcher; import java.util.regex.Pattern; -public abstract class MD5FileUtil { - public static final Logger LOG = LoggerFactory.getLogger(MD5FileUtil.class); +public final class MD5FileUtil { + private MD5FileUtil() {} - // TODO: we should provide something like Hadoop's checksum fs for the local filesystem - // so that individual state machines do not have to deal with checksumming/corruption prevention. - // Keep the checksum and data in the same block format instead of individual files. + public static MessageDigest newMD5() { + try { + return MessageDigest.getInstance("MD5"); + } catch (NoSuchAlgorithmException e) { + throw new IllegalStateException("Failed to create MessageDigest for MD5", e); + } + } public static final String MD5_SUFFIX = ".md5"; private static final String LINE_REGEX = "([0-9a-f]{32}) [ *](.+)"; @@ -105,7 +108,7 @@ public static MD5Hash readStoredMd5ForFile(File dataFile) throws IOException { referencedFile.getName() + " but we expected it to reference " + dataFile); } - return new MD5Hash(storedHash); + return MD5Hash.newInstance(storedHash); } /** @@ -113,7 +116,7 @@ public static MD5Hash readStoredMd5ForFile(File dataFile) throws IOException { */ public static MD5Hash computeMd5ForFile(File dataFile) throws IOException { final int bufferSize = SizeInBytes.ONE_MB.getSizeInt(); - final MessageDigest digester = MD5Hash.getDigester(); + final MessageDigest digester = newMD5(); try (FileChannel in = FileUtils.newFileChannel(dataFile, StandardOpenOption.READ)) { final long fileSize = in.size(); for (int offset = 0; offset < fileSize; ) { @@ -122,7 +125,7 @@ public static MD5Hash computeMd5ForFile(File dataFile) throws IOException { offset += readSize; } } - return new MD5Hash(digester.digest()); + return MD5Hash.newInstance(digester.digest()); } public static MD5Hash computeAndSaveMd5ForFile(File dataFile) { @@ -147,7 +150,7 @@ public static MD5Hash computeAndSaveMd5ForFile(File dataFile) { */ public static void saveMD5File(File dataFile, MD5Hash digest) throws IOException { - final String digestString = StringUtils.bytes2HexString(digest.getDigest()); + final String digestString = digest.toString(); saveMD5File(dataFile, digestString); } @@ -162,10 +165,6 @@ private static void saveMD5File(File dataFile, String digestString) try (AtomicFileOutputStream afos = new AtomicFileOutputStream(md5File)) { afos.write(md5Line.getBytes(StandardCharsets.UTF_8)); } - - if (LOG.isDebugEnabled()) { - LOG.debug("Saved MD5 " + digestString + " to " + md5File); - } } /** diff --git a/ratis-common/src/test/java/org/apache/ratis/io/TestMD5.java b/ratis-common/src/test/java/org/apache/ratis/io/TestMD5.java new file mode 100644 index 0000000000..7e7c29b642 --- /dev/null +++ b/ratis-common/src/test/java/org/apache/ratis/io/TestMD5.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.ratis.io; + +import org.apache.ratis.util.StringUtils; +import org.junit.jupiter.api.Test; + +import java.nio.ByteBuffer; +import java.util.concurrent.ThreadLocalRandom; + +import static org.junit.jupiter.api.Assertions.assertEquals; + +class TestMD5 { + + @Test + void testMD5Hash() { + final byte[] digest = new byte[MD5Hash.MD5_LENGTH]; + final ThreadLocalRandom random = ThreadLocalRandom.current(); + + for(int i = 0; i < 1000; i++) { + random.nextBytes(digest); + final MD5Hash md5 = MD5Hash.newInstance(digest); + + // test hashCode + final int expectedHashCode = oldQuarterDigest(digest); + assertEquals(expectedHashCode, md5.hashCode()); + + // test toString + final String expectedString = StringUtils.bytes2HexString(digest); + assertEquals(expectedString, md5.toString()); + assertEquals(expectedString, MD5Hash.digestToString(digest)); + + // test newInstance(String) + assertEquals(md5, MD5Hash.newInstance(expectedString.toLowerCase())); + assertEquals(md5, MD5Hash.newInstance(expectedString.toUpperCase())); + + // test getDigest + final ByteBuffer expectedByteBuffer = ByteBuffer.wrap(digest); + assertEquals(expectedByteBuffer, md5.getDigest()); + } + } + + /** + * Return a 32-bit digest of the MD5. + * @return the first 4 bytes of the md5 + */ + private static int oldQuarterDigest(byte[] digest) { + int value = 0; + for (int i = 0; i < 4; i++) { + value |= ((digest[i] & 0xff) << (8*(3-i))); + } + return value; + } +} \ No newline at end of file diff --git a/ratis-server/src/main/java/org/apache/ratis/server/storage/FileChunkReader.java b/ratis-server/src/main/java/org/apache/ratis/server/storage/FileChunkReader.java index a12818443e..b80924eef1 100644 --- a/ratis-server/src/main/java/org/apache/ratis/server/storage/FileChunkReader.java +++ b/ratis-server/src/main/java/org/apache/ratis/server/storage/FileChunkReader.java @@ -17,13 +17,13 @@ */ package org.apache.ratis.server.storage; -import org.apache.ratis.io.MD5Hash; import org.apache.ratis.proto.RaftProtos.FileChunkProto; import org.apache.ratis.thirdparty.com.google.protobuf.ByteString; import org.apache.ratis.thirdparty.com.google.protobuf.UnsafeByteOperations; import org.apache.ratis.util.FileUtils; import org.apache.ratis.util.IOUtils; import org.apache.ratis.util.JavaUtils; +import org.apache.ratis.util.MD5FileUtil; import java.io.Closeable; import java.io.File; @@ -56,7 +56,7 @@ public FileChunkReader(FileInfo info, Path relativePath) throws IOException { this.relativePath = relativePath; final File f = info.getPath().toFile(); if (info.getFileDigest() == null) { - digester = MD5Hash.newDigester(); + digester = MD5FileUtil.newMD5(); this.in = new DigestInputStream(FileUtils.newInputStream(f), digester); } else { digester = null; diff --git a/ratis-server/src/main/java/org/apache/ratis/server/storage/SnapshotManager.java b/ratis-server/src/main/java/org/apache/ratis/server/storage/SnapshotManager.java index 2d10c53a45..a96001b598 100644 --- a/ratis-server/src/main/java/org/apache/ratis/server/storage/SnapshotManager.java +++ b/ratis-server/src/main/java/org/apache/ratis/server/storage/SnapshotManager.java @@ -87,7 +87,7 @@ private FileChannel open(FileChunkProto chunk, File tmpSnapshotFile) throws IOEx } // create the temp snapshot file and put padding inside out = FileUtils.newFileChannel(tmpSnapshotFile, StandardOpenOption.WRITE, StandardOpenOption.CREATE); - digester = MD5Hash.newDigester(); + digester = MD5FileUtil.newMD5(); } else { if (!exists) { throw new FileNotFoundException("Chunk offset is non-zero but file is not found: " + tmpSnapshotFile @@ -138,11 +138,10 @@ public void installSnapshot(InstallSnapshotRequestProto request, StateMachine st // rename the temp snapshot file if this is the last chunk. also verify // the md5 digest and create the md5 meta-file. if (chunk.getDone()) { - final MD5Hash expectedDigest = - new MD5Hash(chunk.getFileDigest().toByteArray()); + final MD5Hash expectedDigest = MD5Hash.newInstance(chunk.getFileDigest().toByteArray()); // calculate the checksum of the snapshot file and compare it with the // file digest in the request - final MD5Hash digest = new MD5Hash(digester.digest()); + final MD5Hash digest = MD5Hash.newInstance(digester.digest()); if (!digest.equals(expectedDigest)) { LOG.warn("The snapshot md5 digest {} does not match expected {}", digest, expectedDigest); @@ -180,8 +179,8 @@ private static void rename(File tmpDir, File stateMachineDir) throws IOException try { moved = FileUtils.move(stateMachineDir, TMP + StringUtils.currentDateTime()); } catch(IOException e) { - LOG.warn("Failed to rename state machine directory " + stateMachineDir.getAbsolutePath() - + " to a " + TMP + " directory. Try deleting it directly.", e); + LOG.warn("Failed to rename state machine directory {} to a " + TMP + " directory. Try deleting it directly.", + stateMachineDir.getAbsolutePath(), e); FileUtils.deleteFully(stateMachineDir); } existingDir = moved; @@ -202,7 +201,7 @@ private static void rename(File tmpDir, File stateMachineDir) throws IOException try { FileUtils.deleteFully(existingDir); } catch (IOException e) { - LOG.warn("Failed to delete existing directory " + existingDir.getAbsolutePath(), e); + LOG.warn("Failed to delete existing directory {}", existingDir.getAbsolutePath(), e); } } } From d393e33ad5ba0a5d9f33e3374f339370fe3a2cfd Mon Sep 17 00:00:00 2001 From: "Doroszlai, Attila" <6454655+adoroszlai@users.noreply.github.com> Date: Thu, 20 Nov 2025 20:24:00 +0100 Subject: [PATCH 232/397] RATIS-2363. Bump plugin versions (#1318) --- pom.xml | 18 +++++++++--------- 1 file changed, 9 insertions(+), 9 deletions(-) diff --git a/pom.xml b/pom.xml index b702cbcd3a..27bdd2352b 100644 --- a/pom.xml +++ b/pom.xml @@ -151,18 +151,18 @@ 5.1.8 3.5.3 - 3.4.0 - 5.0.0.4389 + 3.4.2 + 5.3.0.6276 - 3.3.0 - 3.1.0 - 1.6.1 - 2.4.0 - 2.2.0 + 3.6.1 + 3.6.2 + 1.11.0 + 3.0.0 + 2.7.0 0.6.1 - 2.8.0 + 2.9.1 4.8.6 4.8.6.2 @@ -473,7 +473,7 @@ de.skuzzle.enforcer restrict-imports-enforcer-rule - ${restrict-imports-enforcer-rules.version} + ${restrict-imports-enforcer-rule.version} From f909fa7e7ef4b758be1319a3766e4a256a33da9c Mon Sep 17 00:00:00 2001 From: Tsz-Wo Nicholas Sze Date: Tue, 25 Nov 2025 23:45:02 -0800 Subject: [PATCH 233/397] RATIS-2364. MultipleLinearRandomRetry should throw exceptions for illegal arguments. (#1319) --- .../dev-support/findbugsExcludeFile.xml | 8 --- .../retry/MultipleLinearRandomRetry.java | 56 ++++++------------- .../retry/TestMultipleLinearRandomRetry.java | 10 +++- 3 files changed, 24 insertions(+), 50 deletions(-) diff --git a/ratis-common/dev-support/findbugsExcludeFile.xml b/ratis-common/dev-support/findbugsExcludeFile.xml index 9267e763fa..787621f17f 100644 --- a/ratis-common/dev-support/findbugsExcludeFile.xml +++ b/ratis-common/dev-support/findbugsExcludeFile.xml @@ -23,10 +23,6 @@ - - - - @@ -35,10 +31,6 @@ - - - - diff --git a/ratis-common/src/main/java/org/apache/ratis/retry/MultipleLinearRandomRetry.java b/ratis-common/src/main/java/org/apache/ratis/retry/MultipleLinearRandomRetry.java index bc453f5bea..9cceb6bc5b 100644 --- a/ratis-common/src/main/java/org/apache/ratis/retry/MultipleLinearRandomRetry.java +++ b/ratis-common/src/main/java/org/apache/ratis/retry/MultipleLinearRandomRetry.java @@ -19,8 +19,6 @@ import org.apache.ratis.util.JavaUtils; import org.apache.ratis.util.TimeDuration; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; import java.util.ArrayList; import java.util.Collections; @@ -34,28 +32,19 @@ * Given pairs of number of retries and sleep time (n0, t0), (n1, t1), ..., * the first n0 retries sleep t0 milliseconds on average, * the following n1 retries sleep t1 milliseconds on average, and so on. - * + *

* For all the sleep, the actual sleep time is randomly uniform distributed * in the close interval [0.5t, 1.5t], where t is the sleep time specified. - * + *

* The objects of this class are immutable. */ public final class MultipleLinearRandomRetry implements RetryPolicy { - static final Logger LOG = LoggerFactory.getLogger(MultipleLinearRandomRetry.class); - /** Pairs of numRetries and sleepSeconds */ - private static class Pair { + private static final class Pair { private final int numRetries; private final TimeDuration sleepTime; Pair(int numRetries, TimeDuration sleepTime) { - if (numRetries < 0) { - throw new IllegalArgumentException("numRetries = " + numRetries+" < 0"); - } - if (sleepTime.isNegative()) { - throw new IllegalArgumentException("sleepTime = " + sleepTime + " < 0"); - } - this.numRetries = numRetries; this.sleepTime = sleepTime; } @@ -76,9 +65,6 @@ public String toString() { private final Supplier myString; private MultipleLinearRandomRetry(List pairs) { - if (pairs == null || pairs.isEmpty()) { - throw new IllegalArgumentException("pairs must be neither null nor empty."); - } this.pairs = Collections.unmodifiableList(pairs); this.myString = JavaUtils.memoize(() -> JavaUtils.getClassSimpleName(getClass()) + pairs); } @@ -131,30 +117,22 @@ public String toString() { * @return the parsed object, or null if the parsing fails. */ public static MultipleLinearRandomRetry parseCommaSeparated(String input) { - final String[] elements = input.split(","); - if (elements.length == 0) { - LOG.warn("Illegal value: there is no element in \"{}\".", input); - return null; + input = input.trim(); + if (input.isEmpty()) { + throw new IllegalArgumentException("Failed to parse \"" + input + "\": no elements found"); } + final String[] elements = input.split(","); if (elements.length % 2 != 0) { - LOG.warn("Illegal value: the number of elements in \"{}\" is {} but an even number of elements is expected.", - input, elements.length); - return null; + throw new IllegalArgumentException("Failed to parse \"" + input + + "\": number of elements (" + elements.length + ") is old"); } final List pairs = new ArrayList<>(); for(int i = 0; i < elements.length; ) { - //parse the i-th sleep-time - final TimeDuration sleep = parseElement(elements, i++, input, MultipleLinearRandomRetry::parsePositiveTime); - if (sleep == null) { - return null; //parse fails - } - //parse the i-th number-of-retries - final Integer retries = parseElement(elements, i++, input, MultipleLinearRandomRetry::parsePositiveInt); - if (retries == null) { - return null; //parse fails - } - + final TimeDuration sleep = parseElement("sleep-time", elements, i++, input, + MultipleLinearRandomRetry::parsePositiveTime); + final Integer retries = parseElement("retries", elements, i++, input, + MultipleLinearRandomRetry::parsePositiveInt); pairs.add(new Pair(retries, sleep)); } return new MultipleLinearRandomRetry(pairs); @@ -176,13 +154,13 @@ private static int parsePositiveInt(String trimmed) { return n; } - private static E parseElement(String[] elements, int i, String input, Function parser) { + private static E parseElement(String name, String[] elements, int i, String input, Function parser) { final String s = elements[i].trim().replace("_", ""); try { return parser.apply(s); - } catch(Exception t) { - LOG.warn("Failed to parse \"{}\", which is the index {} element in \"{}\"", s, i, input, t); - return null; + } catch (Exception e) { + throw new IllegalArgumentException( + "Failed to parse \"" + s + "\" as " + name + " (element " + i + " in \"" + input + "\")", e); } } } diff --git a/ratis-test/src/test/java/org/apache/ratis/retry/TestMultipleLinearRandomRetry.java b/ratis-test/src/test/java/org/apache/ratis/retry/TestMultipleLinearRandomRetry.java index 77bcb70f74..621d46b5b3 100644 --- a/ratis-test/src/test/java/org/apache/ratis/retry/TestMultipleLinearRandomRetry.java +++ b/ratis-test/src/test/java/org/apache/ratis/retry/TestMultipleLinearRandomRetry.java @@ -44,10 +44,14 @@ public void testParseCommaSeparated() { assertLegalInput("[10x100ms, 20x1s, 30x5s]", "100,10, 1s,20, 5s,30"); } - private static void assertIllegalInput(String input) { - final MultipleLinearRandomRetry computed = MultipleLinearRandomRetry.parseCommaSeparated(input); - Assertions.assertNull(computed); + private void assertIllegalInput(String input) { + try { + MultipleLinearRandomRetry.parseCommaSeparated(input); + } catch (IllegalArgumentException e) { + LOG.info("Expected to catch: {}", String.valueOf(e)); + } } + private static MultipleLinearRandomRetry assertLegalInput(String expected, String input) { final MultipleLinearRandomRetry computed = MultipleLinearRandomRetry.parseCommaSeparated(input); Assertions.assertNotNull(computed); From eefff9443ba85215dbec1602fd98ab941a01696c Mon Sep 17 00:00:00 2001 From: Potato Date: Wed, 17 Dec 2025 03:03:32 +0800 Subject: [PATCH 234/397] RATIS-2371. Fix LeaderElection/SegmentedRaftLogReader/FileChunkReader CT_CONSTRUCTOR_THROW spotbugs (#1327) --- .../dev-support/findbugsExcludeFile.xml | 12 -------- .../ratis/server/impl/LeaderElection.java | 30 ++++++++++++------- .../apache/ratis/server/impl/RoleInfo.java | 2 +- .../leader/InstallSnapshotRequests.java | 2 +- .../SegmentedRaftLogInputStream.java | 2 +- .../segmented/SegmentedRaftLogReader.java | 16 +++++++--- .../ratis/server/storage/FileChunkReader.java | 22 ++++++++++---- .../server/impl/LeaderElectionTests.java | 4 +-- .../TestLeaderElectionServerInterface.java | 2 +- 9 files changed, 53 insertions(+), 39 deletions(-) diff --git a/ratis-server/dev-support/findbugsExcludeFile.xml b/ratis-server/dev-support/findbugsExcludeFile.xml index 9180856202..0161c226bb 100644 --- a/ratis-server/dev-support/findbugsExcludeFile.xml +++ b/ratis-server/dev-support/findbugsExcludeFile.xml @@ -15,10 +15,6 @@ limitations under the License. --> - - - - @@ -51,10 +47,6 @@ - - - - @@ -67,10 +59,6 @@ - - - - diff --git a/ratis-server/src/main/java/org/apache/ratis/server/impl/LeaderElection.java b/ratis-server/src/main/java/org/apache/ratis/server/impl/LeaderElection.java index 9953e12aff..385d338335 100644 --- a/ratis-server/src/main/java/org/apache/ratis/server/impl/LeaderElection.java +++ b/ratis-server/src/main/java/org/apache/ratis/server/impl/LeaderElection.java @@ -77,7 +77,7 @@ * Ongaro, D. Consensus: Bridging Theory and Practice. PhD thesis, Stanford University, 2014. * Available at https://github.com/ongardie/dissertation */ -class LeaderElection implements Runnable { +final class LeaderElection implements Runnable { public static final Logger LOG = LoggerFactory.getLogger(LeaderElection.class); interface ServerInterface { @@ -306,25 +306,33 @@ public String toString() { private final boolean skipPreVote; private final ConfAndTerm round0; - LeaderElection(RaftServerImpl server, boolean force) { - this(ServerInterface.get(server), force); + static LeaderElection newInstance(RaftServerImpl server, boolean force) { + return newInstance(ServerInterface.get(server), force); } - LeaderElection(ServerInterface server, boolean force) { - this.name = ServerStringUtils.generateUnifiedName(server.getMemberId(), getClass()) + COUNT.incrementAndGet(); - this.lifeCycle = new LifeCycle(this); - this.daemon = Daemon.newBuilder().setName(name).setRunnable(this) - .setThreadGroup(server.getThreadGroup()).build(); - this.server = server; - this.skipPreVote = force || !server.isPreVoteEnabled(); + static LeaderElection newInstance(ServerInterface server, boolean force) { + String name = ServerStringUtils.generateUnifiedName(server.getMemberId(), LeaderElection.class) + + COUNT.incrementAndGet(); try { // increase term of the candidate in advance if it's forced to election - this.round0 = force ? server.initElection(Phase.ELECTION) : null; + final ConfAndTerm round0 = force ? server.initElection(Phase.ELECTION) : null; + return new LeaderElection(name, server, force, round0); } catch (IOException e) { throw new IllegalStateException(name + ": Failed to initialize election", e); } } + + private LeaderElection(String name, ServerInterface server, boolean force, ConfAndTerm round0) { + this.name = name; + this.lifeCycle = new LifeCycle(this); + this.daemon = Daemon.newBuilder().setName(name).setRunnable(this) + .setThreadGroup(server.getThreadGroup()).build(); + this.server = server; + this.skipPreVote = force || !server.isPreVoteEnabled(); + this.round0 = round0; + } + void start() { startIfNew(daemon::start); } diff --git a/ratis-server/src/main/java/org/apache/ratis/server/impl/RoleInfo.java b/ratis-server/src/main/java/org/apache/ratis/server/impl/RoleInfo.java index a5cd7da665..409d7a06bd 100644 --- a/ratis-server/src/main/java/org/apache/ratis/server/impl/RoleInfo.java +++ b/ratis-server/src/main/java/org/apache/ratis/server/impl/RoleInfo.java @@ -126,7 +126,7 @@ void startLeaderElection(RaftServerImpl server, boolean force) { if (pauseLeaderElection.get()) { return; } - updateAndGet(leaderElection, new LeaderElection(server, force)).start(); + updateAndGet(leaderElection, LeaderElection.newInstance(server, force)).start(); } void setLeaderElectionPause(boolean pause) { diff --git a/ratis-server/src/main/java/org/apache/ratis/server/leader/InstallSnapshotRequests.java b/ratis-server/src/main/java/org/apache/ratis/server/leader/InstallSnapshotRequests.java index 6300ea483c..218c864e69 100644 --- a/ratis-server/src/main/java/org/apache/ratis/server/leader/InstallSnapshotRequests.java +++ b/ratis-server/src/main/java/org/apache/ratis/server/leader/InstallSnapshotRequests.java @@ -119,7 +119,7 @@ public InstallSnapshotRequestProto next() { final FileInfo info = snapshot.getFiles().get(fileIndex); try { if (current == null) { - current = new FileChunkReader(info, getRelativePath.apply(info)); + current = FileChunkReader.newInstance(info, getRelativePath.apply(info)); } final FileChunkProto chunk = current.readFileChunk(snapshotChunkMaxSize); if (chunk.getDone()) { diff --git a/ratis-server/src/main/java/org/apache/ratis/server/raftlog/segmented/SegmentedRaftLogInputStream.java b/ratis-server/src/main/java/org/apache/ratis/server/raftlog/segmented/SegmentedRaftLogInputStream.java index 3cc8767fa4..c302d1f9aa 100644 --- a/ratis-server/src/main/java/org/apache/ratis/server/raftlog/segmented/SegmentedRaftLogInputStream.java +++ b/ratis-server/src/main/java/org/apache/ratis/server/raftlog/segmented/SegmentedRaftLogInputStream.java @@ -83,7 +83,7 @@ private void init() throws IOException { state.open(); boolean initSuccess = false; try { - reader = new SegmentedRaftLogReader(logFile, maxOpSize, raftLogMetrics); + reader = SegmentedRaftLogReader.newInstance(logFile, maxOpSize, raftLogMetrics); initSuccess = reader.verifyHeader(); } finally { if (!initSuccess) { diff --git a/ratis-server/src/main/java/org/apache/ratis/server/raftlog/segmented/SegmentedRaftLogReader.java b/ratis-server/src/main/java/org/apache/ratis/server/raftlog/segmented/SegmentedRaftLogReader.java index 57baffb2fd..b8c906366a 100644 --- a/ratis-server/src/main/java/org/apache/ratis/server/raftlog/segmented/SegmentedRaftLogReader.java +++ b/ratis-server/src/main/java/org/apache/ratis/server/raftlog/segmented/SegmentedRaftLogReader.java @@ -46,7 +46,7 @@ import java.util.Optional; import java.util.zip.Checksum; -class SegmentedRaftLogReader implements Closeable { +final class SegmentedRaftLogReader implements Closeable { static final Logger LOG = LoggerFactory.getLogger(SegmentedRaftLogReader.class); /** * InputStream wrapper that keeps track of the current stream position. @@ -150,10 +150,18 @@ public long skip(long amt) throws IOException { private final SegmentedRaftLogMetrics raftLogMetrics; private final SizeInBytes maxOpSize; - SegmentedRaftLogReader(File file, SizeInBytes maxOpSize, SegmentedRaftLogMetrics raftLogMetrics) throws IOException { + static SegmentedRaftLogReader newInstance(File file, SizeInBytes maxOpSize, SegmentedRaftLogMetrics raftLogMetrics) + throws IOException { + final LimitedInputStream limiter = new LimitedInputStream(new BufferedInputStream(FileUtils.newInputStream(file))); + final DataInputStream in = new DataInputStream(limiter); + return new SegmentedRaftLogReader(file, maxOpSize, raftLogMetrics, limiter, in); + } + + private SegmentedRaftLogReader(File file, SizeInBytes maxOpSize, SegmentedRaftLogMetrics raftLogMetrics, + LimitedInputStream limiter, DataInputStream in) { this.file = file; - this.limiter = new LimitedInputStream(new BufferedInputStream(FileUtils.newInputStream(file))); - in = new DataInputStream(limiter); + this.limiter = limiter; + this.in = in; checksum = new PureJavaCrc32C(); this.maxOpSize = maxOpSize; this.raftLogMetrics = raftLogMetrics; diff --git a/ratis-server/src/main/java/org/apache/ratis/server/storage/FileChunkReader.java b/ratis-server/src/main/java/org/apache/ratis/server/storage/FileChunkReader.java index b80924eef1..6c45412096 100644 --- a/ratis-server/src/main/java/org/apache/ratis/server/storage/FileChunkReader.java +++ b/ratis-server/src/main/java/org/apache/ratis/server/storage/FileChunkReader.java @@ -34,7 +34,7 @@ import java.security.MessageDigest; /** Read {@link FileChunkProto}s from a file. */ -public class FileChunkReader implements Closeable { +public final class FileChunkReader implements Closeable { private final FileInfo info; private final Path relativePath; private final InputStream in; @@ -51,17 +51,27 @@ public class FileChunkReader implements Closeable { * @param relativePath the relative path of the file. * @throws IOException if it failed to open the file. */ - public FileChunkReader(FileInfo info, Path relativePath) throws IOException { - this.info = info; - this.relativePath = relativePath; + public static FileChunkReader newInstance(FileInfo info, Path relativePath) throws IOException { final File f = info.getPath().toFile(); + final InputStream in; + final MessageDigest digester; + if (info.getFileDigest() == null) { digester = MD5FileUtil.newMD5(); - this.in = new DigestInputStream(FileUtils.newInputStream(f), digester); + in = new DigestInputStream(FileUtils.newInputStream(f), digester); } else { digester = null; - this.in = FileUtils.newInputStream(f); + in = FileUtils.newInputStream(f); } + + return new FileChunkReader(info, relativePath, in, digester); + } + + private FileChunkReader(FileInfo info, Path relativePath, InputStream in, MessageDigest digester) { + this.info = info; + this.relativePath = relativePath; + this.in = in; + this.digester = digester; } static ByteString readFileChunk(int chunkLength, InputStream in) throws IOException { diff --git a/ratis-server/src/test/java/org/apache/ratis/server/impl/LeaderElectionTests.java b/ratis-server/src/test/java/org/apache/ratis/server/impl/LeaderElectionTests.java index 724a066430..456d2ad2a2 100644 --- a/ratis-server/src/test/java/org/apache/ratis/server/impl/LeaderElectionTests.java +++ b/ratis-server/src/test/java/org/apache/ratis/server/impl/LeaderElectionTests.java @@ -592,7 +592,7 @@ public void testLeaderElectionMetrics() throws IOException, InterruptedException @Test public void testImmediatelyRevertedToFollower() { RaftServerImpl server = createMockServer(true); - LeaderElection subject = new LeaderElection(server, false); + LeaderElection subject = LeaderElection.newInstance(server, false); try { subject.startInForeground(); @@ -606,7 +606,7 @@ public void testImmediatelyRevertedToFollower() { @Test public void testShutdownBeforeStart() { RaftServerImpl server = createMockServer(false); - LeaderElection subject = new LeaderElection(server, false); + LeaderElection subject = LeaderElection.newInstance(server, false); try { subject.shutdown(); diff --git a/ratis-test/src/test/java/org/apache/ratis/server/impl/TestLeaderElectionServerInterface.java b/ratis-test/src/test/java/org/apache/ratis/server/impl/TestLeaderElectionServerInterface.java index 876633db1d..3a91f9a343 100644 --- a/ratis-test/src/test/java/org/apache/ratis/server/impl/TestLeaderElectionServerInterface.java +++ b/ratis-test/src/test/java/org/apache/ratis/server/impl/TestLeaderElectionServerInterface.java @@ -186,7 +186,7 @@ void runTestVoterWithEmptyLog(boolean expectToPass, TermIndex... lastEntries) { for(int i = 0; i < lastEntries.length; i++) { map.put(peers.get(i).getId(), lastEntries[i]); } - final LeaderElection election = new LeaderElection(newServerInterface(expectToPass, map), false); + final LeaderElection election = LeaderElection.newInstance(newServerInterface(expectToPass, map), false); election.startInForeground(); } From 4db27b011a392b168982d4b0bf598afcf648bacd Mon Sep 17 00:00:00 2001 From: "Doroszlai, Attila" <6454655+adoroszlai@users.noreply.github.com> Date: Tue, 16 Dec 2025 20:17:51 +0100 Subject: [PATCH 235/397] RATIS-2373. Use workflow inputs in run command via environment variable (#1329) --- .github/workflows/check.yaml | 8 +++++--- .github/workflows/repeat-test.yaml | 3 ++- 2 files changed, 7 insertions(+), 4 deletions(-) diff --git a/.github/workflows/check.yaml b/.github/workflows/check.yaml index 81651c1fa7..996277d75d 100644 --- a/.github/workflows/check.yaml +++ b/.github/workflows/check.yaml @@ -79,6 +79,7 @@ on: env: MAVEN_ARGS: --batch-mode --show-version MAVEN_OPTS: -Dhttp.keepAlive=false -Dmaven.wagon.http.pool=false -Dmaven.wagon.http.retryHandler.class=standard -Dmaven.wagon.http.retryHandler.count=3 + SCRIPT: ${{ inputs.script }} WITH_COVERAGE: ${{ github.event_name == 'push' }} jobs: @@ -154,15 +155,16 @@ jobs: - name: Execute tests run: | - dev-support/checks/${{ inputs.script }}.sh ${{ inputs.script-args }} + $COMMAND env: + COMMAND: dev-support/checks/${{ inputs.script }}.sh ${{ inputs.script-args }} DEVELOCITY_ACCESS_KEY: ${{ secrets.DEVELOCITY_ACCESS_KEY }} - name: Summary of failures if: ${{ failure() }} run: | - if [[ -s "target/${{ inputs.script }}/summary.txt" ]]; then - cat target/${{ inputs.script }}/summary.txt + if [[ -s "target/$SCRIPT/summary.txt" ]]; then + cat target/$SCRIPT/summary.txt fi - name: Archive build results diff --git a/.github/workflows/repeat-test.yaml b/.github/workflows/repeat-test.yaml index 66fec90f16..de787d9291 100644 --- a/.github/workflows/repeat-test.yaml +++ b/.github/workflows/repeat-test.yaml @@ -47,6 +47,7 @@ env: TEST_METHOD: ${{ github.event.inputs.test-method }} ITERATIONS: ${{ github.event.inputs.iterations }} FAIL_FAST: ${{ github.event.inputs.fail-fast }} + SPLITS: ${{ github.event.inputs.splits }} run-name: ${{ github.event_name == 'workflow_dispatch' && format('{0}#{1}[{2}]-{3}x{4}', inputs.test-class, inputs.test-method, inputs.ref, inputs.splits, inputs.iterations) || '' }} jobs: prepare: @@ -59,7 +60,7 @@ jobs: name: Generate test matrix run: | splits=() - for ((i = 1; i <= ${{ github.event.inputs.splits }}; i++)); do + for ((i = 1; i <= $SPLITS; i++)); do splits+=("$i") done printf -v x "%s," "${splits[@]}" From a43511fb08f9b5bbabacf9e6becbd881e66064e7 Mon Sep 17 00:00:00 2001 From: Symious <14933944+symious@users.noreply.github.com> Date: Fri, 19 Dec 2025 01:26:52 +0800 Subject: [PATCH 236/397] RATIS-2129. Low replication performance because of lock contention on RaftLog (#1322) Co-authored-by: Tsz-Wo Nicholas Sze --- .../ratis/server/RaftServerConfigKeys.java | 10 +++ .../server/raftlog/segmented/LogSegment.java | 83 ++++++++++++++----- .../raftlog/segmented/SegmentedRaftLog.java | 10 ++- .../segmented/SegmentedRaftLogCache.java | 13 ++- .../segmented/TestSegmentedRaftLogCache.java | 5 +- 5 files changed, 90 insertions(+), 31 deletions(-) diff --git a/ratis-server-api/src/main/java/org/apache/ratis/server/RaftServerConfigKeys.java b/ratis-server-api/src/main/java/org/apache/ratis/server/RaftServerConfigKeys.java index 849597433a..002286c4ca 100644 --- a/ratis-server-api/src/main/java/org/apache/ratis/server/RaftServerConfigKeys.java +++ b/ratis-server-api/src/main/java/org/apache/ratis/server/RaftServerConfigKeys.java @@ -415,6 +415,16 @@ static void setSegmentSizeMax(RaftProperties properties, SizeInBytes segmentSize setSizeInBytes(properties::set, SEGMENT_SIZE_MAX_KEY, segmentSizeMax); } + String READ_LOCK_ENABLED_KEY = PREFIX + ".read.lock.enabled"; + boolean READ_LOCK_ENABLED_DEFAULT = true; + static boolean readLockEnabled(RaftProperties properties) { + return getBoolean(properties::getBoolean, + READ_LOCK_ENABLED_KEY, READ_LOCK_ENABLED_DEFAULT, getDefaultLog()); + } + static void setReadLockEnabled(RaftProperties properties, boolean readLockEnabled) { + setBoolean(properties::setBoolean, READ_LOCK_ENABLED_KEY, readLockEnabled); + } + /** * Besides the open segment, the max number of segments caching log entries. */ diff --git a/ratis-server/src/main/java/org/apache/ratis/server/raftlog/segmented/LogSegment.java b/ratis-server/src/main/java/org/apache/ratis/server/raftlog/segmented/LogSegment.java index 444d417ba5..c40b91f708 100644 --- a/ratis-server/src/main/java/org/apache/ratis/server/raftlog/segmented/LogSegment.java +++ b/ratis-server/src/main/java/org/apache/ratis/server/raftlog/segmented/LogSegment.java @@ -36,13 +36,13 @@ import java.io.File; import java.io.IOException; -import java.util.ArrayList; import java.nio.file.Path; import java.util.Comparator; -import java.util.List; import java.util.Map; import java.util.Objects; import java.util.concurrent.ConcurrentHashMap; +import java.util.concurrent.ConcurrentNavigableMap; +import java.util.concurrent.ConcurrentSkipListMap; import java.util.concurrent.atomic.AtomicInteger; import java.util.concurrent.atomic.AtomicLong; import java.util.concurrent.atomic.AtomicReference; @@ -105,6 +105,44 @@ long getOffset() { } } + private static class Records { + private final ConcurrentNavigableMap map = new ConcurrentSkipListMap<>(); + + int size() { + return map.size(); + } + + LogRecord getFirst() { + final Map.Entry first = map.firstEntry(); + return first != null? first.getValue() : null; + } + + LogRecord getLast() { + final Map.Entry last = map.lastEntry(); + return last != null? last.getValue() : null; + } + + LogRecord get(long i) { + return map.get(i); + } + + long append(LogRecord record) { + final long index = record.getTermIndex().getIndex(); + final LogRecord previous = map.put(index, record); + Preconditions.assertNull(previous, "previous"); + return index; + } + + LogRecord removeLast() { + final Map.Entry last = map.pollLastEntry(); + return Objects.requireNonNull(last, "last == null").getValue(); + } + + void clear() { + map.clear(); + } + } + static LogSegment newOpenSegment(RaftStorage storage, long start, SizeInBytes maxOpSize, SegmentedRaftLogMetrics raftLogMetrics) { Preconditions.assertTrue(start >= 0); @@ -204,10 +242,12 @@ private void assertSegment(long expectedStart, int expectedEntryCount, boolean c final long expectedLastIndex = expectedStart + expectedEntryCount - 1; Preconditions.assertSame(expectedLastIndex, getEndIndex(), "Segment end index"); - final LogRecord last = getLastRecord(); + final LogRecord last = records.getLast(); if (last != null) { Preconditions.assertSame(expectedLastIndex, last.getTermIndex().getIndex(), "Index at the last record"); - Preconditions.assertSame(expectedStart, records.get(0).getTermIndex().getIndex(), "Index at the first record"); + final LogRecord first = records.getFirst(); + Objects.requireNonNull(first, "first record"); + Preconditions.assertSame(expectedStart, first.getTermIndex().getIndex(), "Index at the first record"); } if (!corrupted) { Preconditions.assertSame(expectedEnd, expectedLastIndex, "End/last Index"); @@ -272,7 +312,7 @@ File getFile() { /** * the list of records is more like the index of a segment */ - private final List records = new ArrayList<>(); + private final Records records = new Records(); /** * the entryCache caches the content of log entries. */ @@ -293,7 +333,11 @@ long getStartIndex() { } long getEndIndex() { - return endIndex; + if (!isOpen) { + return endIndex; + } + final LogRecord last = records.getLast(); + return last == null ? getStartIndex() - 1 : last.getTermIndex().getIndex(); } boolean isOpen() { @@ -301,7 +345,7 @@ boolean isOpen() { } int numOfEntries() { - return Math.toIntExact(endIndex - startIndex + 1); + return Math.toIntExact(getEndIndex() - startIndex + 1); } CorruptionPolicy getLogCorruptionPolicy() { @@ -315,14 +359,12 @@ void appendToOpenSegment(LogEntryProto entry, Op op) { private void append(boolean keepEntryInCache, LogEntryProto entry, Op op) { Objects.requireNonNull(entry, "entry == null"); - if (records.isEmpty()) { + final LogRecord currentLast = records.getLast(); + if (currentLast == null) { Preconditions.assertTrue(entry.getIndex() == startIndex, "gap between start index %s and first entry to append %s", startIndex, entry.getIndex()); - } - - final LogRecord currentLast = getLastRecord(); - if (currentLast != null) { + } else { Preconditions.assertTrue(entry.getIndex() == currentLast.getTermIndex().getIndex() + 1, "gap between entries %s and %s", entry.getIndex(), currentLast.getTermIndex().getIndex()); } @@ -331,7 +373,7 @@ private void append(boolean keepEntryInCache, LogEntryProto entry, Op op) { if (keepEntryInCache) { putEntryCache(record.getTermIndex(), entry, op); } - records.add(record); + records.append(record); totalFileSize += getEntrySize(entry, op); endIndex = entry.getIndex(); } @@ -358,18 +400,14 @@ synchronized LogEntryProto loadCache(LogRecord record) throws RaftLogIOException } LogRecord getLogRecord(long index) { - if (index >= startIndex && index <= endIndex) { - return records.get(Math.toIntExact(index - startIndex)); + if (index >= startIndex && index <= getEndIndex()) { + return records.get(index); } return null; } - private LogRecord getLastRecord() { - return records.isEmpty() ? null : records.get(records.size() - 1); - } - TermIndex getLastTermIndex() { - LogRecord last = getLastRecord(); + final LogRecord last = records.getLast(); return last == null ? null : last.getTermIndex(); } @@ -387,7 +425,8 @@ long getTotalCacheSize() { synchronized void truncate(long fromIndex) { Preconditions.assertTrue(fromIndex >= startIndex && fromIndex <= endIndex); for (long index = endIndex; index >= fromIndex; index--) { - LogRecord removed = records.remove(Math.toIntExact(index - startIndex)); + final LogRecord removed = records.removeLast(); + Preconditions.assertSame(index, removed.getTermIndex().getIndex(), "removedIndex"); removeEntryCache(removed.getTermIndex(), Op.REMOVE_CACHE); totalFileSize = removed.offset; } @@ -458,7 +497,7 @@ boolean hasCache() { } boolean containsIndex(long index) { - return startIndex <= index && endIndex >= index; + return startIndex <= index && getEndIndex() >= index; } boolean hasEntries() { diff --git a/ratis-server/src/main/java/org/apache/ratis/server/raftlog/segmented/SegmentedRaftLog.java b/ratis-server/src/main/java/org/apache/ratis/server/raftlog/segmented/SegmentedRaftLog.java index 6dc3d7961c..6bcc3f8e1c 100644 --- a/ratis-server/src/main/java/org/apache/ratis/server/raftlog/segmented/SegmentedRaftLog.java +++ b/ratis-server/src/main/java/org/apache/ratis/server/raftlog/segmented/SegmentedRaftLog.java @@ -202,6 +202,7 @@ public TransactionContext getTransactionContext(LogEntryProto entry, boolean cre private final long segmentMaxSize; private final boolean stateMachineCachingEnabled; private final SegmentedRaftLogMetrics metrics; + private final boolean readLockEnabled; @SuppressWarnings({"squid:S2095"}) // Suppress closeable warning private SegmentedRaftLog(Builder b) { @@ -217,6 +218,12 @@ private SegmentedRaftLog(Builder b) { this.fileLogWorker = new SegmentedRaftLogWorker(b.memberId, stateMachine, b.submitUpdateCommitEvent, b.server, storage, b.properties, getRaftLogMetrics()); stateMachineCachingEnabled = RaftServerConfigKeys.Log.StateMachineData.cachingEnabled(b.properties); + this.readLockEnabled = RaftServerConfigKeys.Log.readLockEnabled(b.properties); + } + + @Override + public AutoCloseableLock readLock() { + return readLockEnabled ? super.readLock() : null; } @Override @@ -338,8 +345,7 @@ private void checkAndEvictCache() { public TermIndex getTermIndex(long index) { checkLogState(); try(AutoCloseableLock readLock = readLock()) { - LogRecord record = cache.getLogRecord(index); - return record != null ? record.getTermIndex() : null; + return cache.getTermIndex(index); } } diff --git a/ratis-server/src/main/java/org/apache/ratis/server/raftlog/segmented/SegmentedRaftLogCache.java b/ratis-server/src/main/java/org/apache/ratis/server/raftlog/segmented/SegmentedRaftLogCache.java index a1f0cdd8ab..46acbcc3d8 100644 --- a/ratis-server/src/main/java/org/apache/ratis/server/raftlog/segmented/SegmentedRaftLogCache.java +++ b/ratis-server/src/main/java/org/apache/ratis/server/raftlog/segmented/SegmentedRaftLogCache.java @@ -547,9 +547,13 @@ LogSegment getSegment(long index) { } } - LogRecord getLogRecord(long index) { + TermIndex getTermIndex(long index) { LogSegment segment = getSegment(index); - return segment == null ? null : segment.getLogRecord(index); + if (segment == null) { + return null; + } + final LogRecord record = segment.getLogRecord(index); + return record != null ? record.getTermIndex() : null; } /** @@ -610,8 +614,9 @@ long getLastIndexInClosedSegments() { TermIndex getLastTermIndex() { try (AutoCloseableLock readLock = closedSegments.readLock()) { - return (openSegment != null && openSegment.numOfEntries() > 0) ? - openSegment.getLastTermIndex() : + LogSegment tmpSegment = openSegment; + return (tmpSegment != null && tmpSegment.getLastTermIndex() != null) ? + tmpSegment.getLastTermIndex() : (closedSegments.isEmpty() ? null : closedSegments.get(closedSegments.size() - 1).getLastTermIndex()); } diff --git a/ratis-test/src/test/java/org/apache/ratis/server/raftlog/segmented/TestSegmentedRaftLogCache.java b/ratis-test/src/test/java/org/apache/ratis/server/raftlog/segmented/TestSegmentedRaftLogCache.java index 7c2dbac912..532e32c87d 100644 --- a/ratis-test/src/test/java/org/apache/ratis/server/raftlog/segmented/TestSegmentedRaftLogCache.java +++ b/ratis-test/src/test/java/org/apache/ratis/server/raftlog/segmented/TestSegmentedRaftLogCache.java @@ -20,7 +20,6 @@ import static org.apache.ratis.server.metrics.SegmentedRaftLogMetrics.*; import static org.apache.ratis.server.raftlog.segmented.SegmentedRaftLogTestUtils.MAX_OP_SIZE; -import java.io.IOException; import java.util.Iterator; import java.util.stream.IntStream; @@ -282,12 +281,12 @@ private void populatedSegment(int start, int end, int segmentSize, boolean isOpe }); } - private void testIterator(long startIndex) throws IOException { + private void testIterator(long startIndex) { Iterator iterator = cache.iterator(startIndex); TermIndex prev = null; while (iterator.hasNext()) { TermIndex termIndex = iterator.next(); - Assertions.assertEquals(cache.getLogRecord(termIndex.getIndex()).getTermIndex(), termIndex); + Assertions.assertEquals(cache.getTermIndex(termIndex.getIndex()), termIndex); if (prev != null) { Assertions.assertEquals(prev.getIndex() + 1, termIndex.getIndex()); } From 422c96cf198b40a039df8afa17c4a94ea7311359 Mon Sep 17 00:00:00 2001 From: Tsz-Wo Nicholas Sze Date: Wed, 24 Dec 2025 05:41:41 -0800 Subject: [PATCH 237/397] RATIS-2374. Do not print stack trace for known exceptions in OrderedAsync (#1330) --- .../ratis/client/impl/OrderedAsync.java | 27 +++++++++++++++---- .../java/org/apache/ratis/util/JavaUtils.java | 15 ++++++++--- 2 files changed, 34 insertions(+), 8 deletions(-) diff --git a/ratis-client/src/main/java/org/apache/ratis/client/impl/OrderedAsync.java b/ratis-client/src/main/java/org/apache/ratis/client/impl/OrderedAsync.java index 1e21b171b3..fbeb4b992a 100644 --- a/ratis-client/src/main/java/org/apache/ratis/client/impl/OrderedAsync.java +++ b/ratis-client/src/main/java/org/apache/ratis/client/impl/OrderedAsync.java @@ -51,7 +51,6 @@ import java.util.concurrent.ConcurrentMap; import java.util.concurrent.Semaphore; import java.util.concurrent.atomic.AtomicReference; -import java.util.function.Consumer; import java.util.function.Function; import java.util.function.LongFunction; @@ -176,9 +175,9 @@ CompletableFuture send(RaftClientRequest.Type type, Message mes ).whenComplete((r, e) -> { if (e != null) { if (e.getCause() instanceof AlreadyClosedException) { - LOG.error("Failed to send request, message=" + message + " due to " + e); + LOG.error("Failed to send request, message={} due to {}", message, e.toString()); } else { - LOG.error("Failed to send request, message=" + message, e); + LOG.error("Failed to send request, message={}", message, e); } } requestSemaphore.release(); @@ -212,13 +211,31 @@ private void sendRequestWithRetry(PendingOrderedRequest pending) { }).exceptionally(e -> { final Throwable exception = e; final String key = client.getId() + "-" + request.getCallId() + "-" + exception; - final Consumer op = suffix -> LOG.error("{} {}: Failed* {}", suffix, client.getId(), request, exception); - BatchLogger.print(BatchLogKey.SEND_REQUEST_EXCEPTION, key, op); + BatchLogger.print(BatchLogKey.SEND_REQUEST_EXCEPTION, key, prefix -> logError(prefix, request, exception)); handleException(pending, request, e); return null; }); } + private void logError(String prefix, RaftClientRequest request, Throwable e) { + final Class[] knownExceptionClasses = {AlreadyClosedException.class, NotLeaderException.class}; + for(Class known : knownExceptionClasses) { + if (logError(prefix, request, e, known)) { + return; + } + } + LOG.error("{} {}: Failed* {}", prefix, client.getId(), request, e); + } + + private boolean logError(String prefix, RaftClientRequest request, Throwable e, Class cause) { + if (JavaUtils.isCausedBy(e, cause)) { + LOG.error("{} {}: Failed* {} due to {} caused by {}", + prefix, client.getId(), request, e, cause.getSimpleName()); + return true; + } + return false; + } + private void handleException(PendingOrderedRequest pending, RaftClientRequest request, Throwable e) { final RetryPolicy retryPolicy = client.getRetryPolicy(); if (client.isClosed()) { diff --git a/ratis-common/src/main/java/org/apache/ratis/util/JavaUtils.java b/ratis-common/src/main/java/org/apache/ratis/util/JavaUtils.java index 958e88cee5..c7f8f76298 100644 --- a/ratis-common/src/main/java/org/apache/ratis/util/JavaUtils.java +++ b/ratis-common/src/main/java/org/apache/ratis/util/JavaUtils.java @@ -134,7 +134,7 @@ static T doPrivileged(Supplier action, Function exc static T doPrivileged(Supplier action, Supplier name) { return doPrivileged(action, e -> { - LOG.warn("Failed to " + name.get(), e); + LOG.warn("Failed to {}", name.get(), e); return null; }); } @@ -228,8 +228,8 @@ static RETURN attempt( throw t; } if (log != null && log.isWarnEnabled()) { - log.warn("FAILED \"" + name.get() + "\", attempt #" + i + "/" + numAttempts - + ", sleep " + sleepTime + " and then retry: " + t); + log.warn("FAILED \"{}\", attempt #{}/{}, sleep {} and then retry: {}", + name.get(), i, numAttempts, sleepTime, t.toString()); } } @@ -291,6 +291,15 @@ static Throwable unwrapCompletionException(Throwable t) { return t instanceof CompletionException && t.getCause() != null? t.getCause(): t; } + static boolean isCausedBy(Throwable t, Class cause) { + for(; t != null; t = t.getCause()) { + if (cause.isInstance(t)) { + return true; + } + } + return false; + } + static CompletableFuture allOf(Collection> futures) { if (futures == null || futures.isEmpty()) { return CompletableFuture.completedFuture(null); From e92a8e4b52a02e8081c9e8d769de306b1b084682 Mon Sep 17 00:00:00 2001 From: Potato Date: Mon, 29 Dec 2025 10:08:10 +0800 Subject: [PATCH 238/397] RATIS-2372. Add weekly CVE vulnerability check workflow for Apache Ratis (#1328) Signed-off-by: OneSizeFitsQuorum Co-authored-by: Copilot <175728472+Copilot@users.noreply.github.com> --- .github/workflows/vulnerability-check.yaml | 64 ++++++++++++++++++++++ pom.xml | 5 ++ 2 files changed, 69 insertions(+) create mode 100644 .github/workflows/vulnerability-check.yaml diff --git a/.github/workflows/vulnerability-check.yaml b/.github/workflows/vulnerability-check.yaml new file mode 100644 index 0000000000..49d4fa80ac --- /dev/null +++ b/.github/workflows/vulnerability-check.yaml @@ -0,0 +1,64 @@ +# 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. + +name: vulnerability-check + +on: + schedule: + # Run at 16:00 UTC every Sunday (Monday 00:00 CST) + - cron: "0 16 * * 0" + workflow_dispatch: + +concurrency: + group: ${{ github.workflow }}-${{ github.ref }} + cancel-in-progress: true + +env: + MAVEN_OPTS: -Dhttp.keepAlive=false -Dmaven.wagon.http.pool=false -Dmaven.wagon.http.retryHandler.class=standard -Dmaven.wagon.http.retryHandler.count=3 + MAVEN_ARGS: --batch-mode --no-transfer-progress + +jobs: + dependency-check: + if: ${{ github.event_name == 'workflow_dispatch' || github.repository == 'apache/ratis' }} + runs-on: ubuntu-latest + + steps: + - uses: actions/checkout@v4 + - name: Set up JDK 11 + uses: actions/setup-java@v4 + with: + distribution: corretto + java-version: 11 + + - name: Do Maven install + shell: bash + run: mvn $MAVEN_ARGS clean install -DskipTests + + - name: Do the dependency-check:aggregate + shell: bash + run: mvn $MAVEN_ARGS org.owasp:dependency-check-maven:aggregate -DossIndexUsername=${{ secrets.OSS_INDEX_USER }} -DossIndexPassword=${{ secrets.OSS_INDEX_TOKEN }} -DnvdApiKey=${{ secrets.NVD_API_KEY }} + + - name: Generate report date for artifact name + run: | + utc_time="${{ github.run_started_at }}" + target_time=$(TZ=Asia/Shanghai date -d "$utc_time" +"%Y-%m-%d") + echo "REPORT_DATE=$target_time" >> $GITHUB_ENV + + - name: Upload Artifact + uses: actions/upload-artifact@v4 + with: + name: vulnerability-check-result-${{ env.REPORT_DATE }} + path: target/dependency-check-report.html + retention-days: 15 \ No newline at end of file diff --git a/pom.xml b/pom.xml index 27bdd2352b..41c32c7c24 100644 --- a/pom.xml +++ b/pom.xml @@ -726,6 +726,11 @@ + + org.owasp + dependency-check-maven + 12.1.9 + From d46b1705f1d75414f1bff84f37318f297c52e49a Mon Sep 17 00:00:00 2001 From: jiangyuan Date: Sun, 4 Jan 2026 05:02:01 +0800 Subject: [PATCH 239/397] RATIS-2378. fix listener role transition (#1331) --- .../ratis/server/impl/RaftServerImpl.java | 12 +++++++++-- .../apache/ratis/server/impl/ServerState.java | 21 ++++++++++++++----- .../impl/SnapshotInstallationHandler.java | 3 ++- .../server/impl/LeaderElectionTests.java | 4 ++++ 4 files changed, 32 insertions(+), 8 deletions(-) diff --git a/ratis-server/src/main/java/org/apache/ratis/server/impl/RaftServerImpl.java b/ratis-server/src/main/java/org/apache/ratis/server/impl/RaftServerImpl.java index 689bb8cefd..60f72e001e 100644 --- a/ratis-server/src/main/java/org/apache/ratis/server/impl/RaftServerImpl.java +++ b/ratis-server/src/main/java/org/apache/ratis/server/impl/RaftServerImpl.java @@ -591,7 +591,7 @@ private synchronized CompletableFuture changeToFollower( throw new IllegalStateException("Unexpected role " + old); } CompletableFuture future = CompletableFuture.completedFuture(null); - if ((old != RaftPeerRole.FOLLOWER || force) && old != RaftPeerRole.LISTENER) { + if (shouldSetFollower(old, force)) { setRole(RaftPeerRole.FOLLOWER, reason); if (old == RaftPeerRole.LEADER) { future = role.shutdownLeaderState(false) @@ -607,7 +607,7 @@ private synchronized CompletableFuture changeToFollower( state.setLeader(null, reason); } else if (old == RaftPeerRole.CANDIDATE) { future = role.shutdownLeaderElection(); - } else if (old == RaftPeerRole.FOLLOWER) { + } else if (old == RaftPeerRole.FOLLOWER || old == RaftPeerRole.LISTENER) { future = role.shutdownFollowerState(); } @@ -620,6 +620,14 @@ private synchronized CompletableFuture changeToFollower( return future; } + private boolean shouldSetFollower(RaftPeerRole old, boolean force) { + if (old == RaftPeerRole.LISTENER) { + final RaftConfigurationImpl conf = state.getRaftConf(); + return conf.isStable() && conf.containsInConf(getId()); + } + return old != RaftPeerRole.FOLLOWER || force; + } + synchronized CompletableFuture changeToFollowerAndPersistMetadata( long newTerm, boolean allowListener, diff --git a/ratis-server/src/main/java/org/apache/ratis/server/impl/ServerState.java b/ratis-server/src/main/java/org/apache/ratis/server/impl/ServerState.java index ee1b7d37b7..bcf11baf7a 100644 --- a/ratis-server/src/main/java/org/apache/ratis/server/impl/ServerState.java +++ b/ratis-server/src/main/java/org/apache/ratis/server/impl/ServerState.java @@ -376,10 +376,12 @@ boolean isConfCommitted() { return getLog().getLastCommittedIndex() >= getRaftConf().getLogEntryIndex(); } - void setRaftConf(LogEntryProto entry) { + private boolean setRaftConf(LogEntryProto entry) { if (entry.hasConfigurationEntry()) { setRaftConf(LogProtoUtils.toRaftConfiguration(entry)); + return true; } + return false; } void setRaftConf(RaftConfiguration conf) { @@ -397,10 +399,19 @@ void truncate(long logIndex) { configurationManager.removeConfigurations(logIndex); } - void updateConfiguration(List entries) { - if (entries != null && !entries.isEmpty()) { - configurationManager.removeConfigurations(entries.get(0).getIndex()); - entries.forEach(this::setRaftConf); + void updateConfiguration(List entries) throws IOException { + if (entries == null || entries.isEmpty()) { + return; + } + configurationManager.removeConfigurations(entries.get(0).getIndex()); + + boolean changed = false; + for(LogEntryProto entry : entries) { + changed |= setRaftConf(entry); + } + + if (changed && server.getRole().getCurrentRole() == RaftPeerRole.LISTENER) { + server.changeToFollowerAndPersistMetadata(getCurrentTerm(), true, "setRaftConf").join(); } } diff --git a/ratis-server/src/main/java/org/apache/ratis/server/impl/SnapshotInstallationHandler.java b/ratis-server/src/main/java/org/apache/ratis/server/impl/SnapshotInstallationHandler.java index eac690feb2..46b6aaf87f 100644 --- a/ratis-server/src/main/java/org/apache/ratis/server/impl/SnapshotInstallationHandler.java +++ b/ratis-server/src/main/java/org/apache/ratis/server/impl/SnapshotInstallationHandler.java @@ -46,6 +46,7 @@ import org.slf4j.LoggerFactory; import java.io.IOException; +import java.util.Collections; import java.util.Optional; import java.util.concurrent.CompletableFuture; import java.util.concurrent.atomic.AtomicBoolean; @@ -145,7 +146,7 @@ private InstallSnapshotReplyProto installSnapshotImpl(InstallSnapshotRequestProt state.truncate(proto.getIndex()); if (!state.getRaftConf().equals(LogProtoUtils.toRaftConfiguration(proto))) { LOG.info("{}: set new configuration {} from snapshot", getMemberId(), ProtoUtils.shortDebugString(proto)); - state.setRaftConf(proto); + state.updateConfiguration(Collections.singletonList(proto)); state.writeRaftConfiguration(proto); server.getStateMachine().event().notifyConfigurationChanged( proto.getTerm(), proto.getIndex(), proto.getConfigurationEntry()); diff --git a/ratis-server/src/test/java/org/apache/ratis/server/impl/LeaderElectionTests.java b/ratis-server/src/test/java/org/apache/ratis/server/impl/LeaderElectionTests.java index 456d2ad2a2..6959bd3422 100644 --- a/ratis-server/src/test/java/org/apache/ratis/server/impl/LeaderElectionTests.java +++ b/ratis-server/src/test/java/org/apache/ratis/server/impl/LeaderElectionTests.java @@ -556,6 +556,10 @@ public void testChangeListenerToFollower() throws Exception { assertTrue(reply.isSuccess()); Collection peer = leader.getRaftConf().getAllPeers(RaftProtos.RaftPeerRole.LISTENER); assertEquals(0, peer.size()); + + listeners = cluster.getListeners() + .stream().map(RaftServer.Division::getPeer).collect(Collectors.toList()); + assertEquals(0, listeners.size()); } cluster.shutdown(); } From f82804e13c1b5f644817c6a9c5017bb1148b3ee6 Mon Sep 17 00:00:00 2001 From: Ivan Andika Date: Sat, 10 Jan 2026 00:07:58 +0800 Subject: [PATCH 240/397] RATIS-2379. Support returning applied index for ReadIndex (#1332) --- .../src/site/markdown/configurations.md | 7 +++++ .../ratis/server/RaftServerConfigKeys.java | 15 +++++++++++ .../ratis/server/impl/LeaderStateImpl.java | 17 +++++++----- .../apache/ratis/LinearizableReadTests.java | 3 +++ ...stLinearizableLeaderLeaseReadWithGrpc.java | 5 ++++ ...adAppliedIndexLeaderLeaseReadWithGrpc.java | 27 +++++++++++++++++++ ...tLinearizableReadAppliedIndexWithGrpc.java | 27 +++++++++++++++++++ .../grpc/TestLinearizableReadWithGrpc.java | 5 ++++ 8 files changed, 100 insertions(+), 6 deletions(-) create mode 100644 ratis-test/src/test/java/org/apache/ratis/grpc/TestLinearizableReadAppliedIndexLeaderLeaseReadWithGrpc.java create mode 100644 ratis-test/src/test/java/org/apache/ratis/grpc/TestLinearizableReadAppliedIndexWithGrpc.java diff --git a/ratis-docs/src/site/markdown/configurations.md b/ratis-docs/src/site/markdown/configurations.md index 4caf11b30e..acd1cb9f9e 100644 --- a/ratis-docs/src/site/markdown/configurations.md +++ b/ratis-docs/src/site/markdown/configurations.md @@ -218,6 +218,13 @@ if it fails to receive any RPC responses from this peer within this specified ti | **Type** | TimeDuration | | **Default** | 60s | +### Read Index - Configurations related to ReadIndex used in linearizable read + +| **Property** | `raft.server.read.read-index.applied-index.enabled` | +|:----------------|:----------------------------------------------------------------------| +| **Description** | whether applied index (instead of commit index) is used for ReadIndex | +| **Type** | boolean | +| **Default** | false | ### Write - Configurations related to write requests. diff --git a/ratis-server-api/src/main/java/org/apache/ratis/server/RaftServerConfigKeys.java b/ratis-server-api/src/main/java/org/apache/ratis/server/RaftServerConfigKeys.java index 002286c4ca..2538a472a8 100644 --- a/ratis-server-api/src/main/java/org/apache/ratis/server/RaftServerConfigKeys.java +++ b/ratis-server-api/src/main/java/org/apache/ratis/server/RaftServerConfigKeys.java @@ -265,6 +265,21 @@ static void setWriteIndexCacheExpiryTime(RaftProperties properties, TimeDuration setTimeDuration(properties::setTimeDuration, WRITE_INDEX_CACHE_EXPIRY_TIME_KEY, expiryTime); } } + + interface ReadIndex { + String PREFIX = Read.PREFIX + ".read-index"; + + String APPLIED_INDEX_ENABLED_KEY = PREFIX + ".applied-index.enabled"; + boolean APPLIED_INDEX_ENABLED_DEFAULT = false; + static boolean appliedIndexEnabled(RaftProperties properties) { + return getBoolean(properties::getBoolean, APPLIED_INDEX_ENABLED_KEY, + APPLIED_INDEX_ENABLED_DEFAULT, getDefaultLog()); + } + + static void setAppliedIndexEnabled(RaftProperties properties, boolean enabled) { + setBoolean(properties::setBoolean, APPLIED_INDEX_ENABLED_KEY, enabled); + } + } } interface Write { diff --git a/ratis-server/src/main/java/org/apache/ratis/server/impl/LeaderStateImpl.java b/ratis-server/src/main/java/org/apache/ratis/server/impl/LeaderStateImpl.java index 836b15bcdb..90d0b76df5 100644 --- a/ratis-server/src/main/java/org/apache/ratis/server/impl/LeaderStateImpl.java +++ b/ratis-server/src/main/java/org/apache/ratis/server/impl/LeaderStateImpl.java @@ -353,6 +353,7 @@ boolean isApplied() { private final PendingStepDown pendingStepDown; private final ReadIndexHeartbeats readIndexHeartbeats; + private final boolean readIndexAppliedIndexEnabled; private final LeaderLease lease; LeaderStateImpl(RaftServerImpl server) { @@ -389,6 +390,8 @@ boolean isApplied() { } else { this.followerMaxGapThreshold = (long) (followerGapRatioMax * maxPendingRequests); } + this.readIndexAppliedIndexEnabled = RaftServerConfigKeys.Read.ReadIndex + .appliedIndexEnabled(properties); final RaftConfigurationImpl conf = state.getRaftConf(); Collection others = conf.getOtherPeers(server.getId()); @@ -1134,21 +1137,23 @@ public boolean checkLeadership() { /** * Obtain the current readIndex for read only requests. See Raft paper section 6.4. * 1. Leader makes sure at least one log from current term is committed. - * 2. Leader record last committed index as readIndex. + * 2. Leader record last committed index or applied index (depending on configuration) as readIndex. * 3. Leader broadcast heartbeats to followers and waits for acknowledgements. * 4. If majority respond success, returns readIndex. * @return current readIndex. */ CompletableFuture getReadIndex(Long readAfterWriteConsistentIndex) { - final long commitIndex = server.getRaftLog().getLastCommittedIndex(); + final long index = readIndexAppliedIndexEnabled ? + server.getState().getLastAppliedIndex() : server.getRaftLog().getLastCommittedIndex(); final long readIndex; - if (readAfterWriteConsistentIndex != null && readAfterWriteConsistentIndex > commitIndex) { + if (readAfterWriteConsistentIndex != null && readAfterWriteConsistentIndex > index) { readIndex = readAfterWriteConsistentIndex; } else { - readIndex = commitIndex; + readIndex = index; } - LOG.debug("readIndex={} (commitIndex={}, readAfterWriteConsistentIndex={})", - readIndex, commitIndex, readAfterWriteConsistentIndex); + LOG.debug("readIndex={} ({}Index={}, readAfterWriteConsistentIndex={})", + readIndex, readIndexAppliedIndexEnabled ? "applied" : "commit", + index, readAfterWriteConsistentIndex); // if group contains only one member, fast path if (server.getRaftConf().isSingleton()) { diff --git a/ratis-server/src/test/java/org/apache/ratis/LinearizableReadTests.java b/ratis-server/src/test/java/org/apache/ratis/LinearizableReadTests.java index 91bd2f28d6..b15ae3067f 100644 --- a/ratis-server/src/test/java/org/apache/ratis/LinearizableReadTests.java +++ b/ratis-server/src/test/java/org/apache/ratis/LinearizableReadTests.java @@ -60,6 +60,8 @@ public abstract class LinearizableReadTests public abstract boolean isLeaderLeaseEnabled(); + public abstract boolean readIndexAppliedIndexEnabled(); + public abstract void assertRaftProperties(RaftProperties properties); void runWithNewCluster(CheckedConsumer testCase) throws Exception { @@ -75,6 +77,7 @@ public void setup() { CounterStateMachine.setProperties(p); RaftServerConfigKeys.Read.setOption(p, LINEARIZABLE); RaftServerConfigKeys.Read.setLeaderLeaseEnabled(p, isLeaderLeaseEnabled()); + RaftServerConfigKeys.Read.ReadIndex.setAppliedIndexEnabled(p, readIndexAppliedIndexEnabled()); } @Test diff --git a/ratis-test/src/test/java/org/apache/ratis/grpc/TestLinearizableLeaderLeaseReadWithGrpc.java b/ratis-test/src/test/java/org/apache/ratis/grpc/TestLinearizableLeaderLeaseReadWithGrpc.java index e45d8f4ff4..d637498d73 100644 --- a/ratis-test/src/test/java/org/apache/ratis/grpc/TestLinearizableLeaderLeaseReadWithGrpc.java +++ b/ratis-test/src/test/java/org/apache/ratis/grpc/TestLinearizableLeaderLeaseReadWithGrpc.java @@ -34,6 +34,11 @@ public boolean isLeaderLeaseEnabled() { return true; } + @Override + public boolean readIndexAppliedIndexEnabled() { + return false; + } + @Override public void assertRaftProperties(RaftProperties p) { assertOption(LINEARIZABLE, p); diff --git a/ratis-test/src/test/java/org/apache/ratis/grpc/TestLinearizableReadAppliedIndexLeaderLeaseReadWithGrpc.java b/ratis-test/src/test/java/org/apache/ratis/grpc/TestLinearizableReadAppliedIndexLeaderLeaseReadWithGrpc.java new file mode 100644 index 0000000000..9bf3e307be --- /dev/null +++ b/ratis-test/src/test/java/org/apache/ratis/grpc/TestLinearizableReadAppliedIndexLeaderLeaseReadWithGrpc.java @@ -0,0 +1,27 @@ +/* + * 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.ratis.grpc; + +public class TestLinearizableReadAppliedIndexLeaderLeaseReadWithGrpc + extends TestLinearizableLeaderLeaseReadWithGrpc { + + @Override + public boolean readIndexAppliedIndexEnabled() { + return true; + } +} diff --git a/ratis-test/src/test/java/org/apache/ratis/grpc/TestLinearizableReadAppliedIndexWithGrpc.java b/ratis-test/src/test/java/org/apache/ratis/grpc/TestLinearizableReadAppliedIndexWithGrpc.java new file mode 100644 index 0000000000..c019aac166 --- /dev/null +++ b/ratis-test/src/test/java/org/apache/ratis/grpc/TestLinearizableReadAppliedIndexWithGrpc.java @@ -0,0 +1,27 @@ +/* + * 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.ratis.grpc; + +public class TestLinearizableReadAppliedIndexWithGrpc + extends TestLinearizableReadWithGrpc { + + @Override + public boolean readIndexAppliedIndexEnabled() { + return true; + } +} diff --git a/ratis-test/src/test/java/org/apache/ratis/grpc/TestLinearizableReadWithGrpc.java b/ratis-test/src/test/java/org/apache/ratis/grpc/TestLinearizableReadWithGrpc.java index a434fe0003..3e8860dd19 100644 --- a/ratis-test/src/test/java/org/apache/ratis/grpc/TestLinearizableReadWithGrpc.java +++ b/ratis-test/src/test/java/org/apache/ratis/grpc/TestLinearizableReadWithGrpc.java @@ -34,6 +34,11 @@ public boolean isLeaderLeaseEnabled() { return false; } + @Override + public boolean readIndexAppliedIndexEnabled() { + return false; + } + @Override public void assertRaftProperties(RaftProperties p) { assertOption(LINEARIZABLE, p); From 519581d49ddc0a7fa56d2b9e0773678892d849bd Mon Sep 17 00:00:00 2001 From: "Doroszlai, Attila" <6454655+adoroszlai@users.noreply.github.com> Date: Tue, 13 Jan 2026 10:05:57 +0100 Subject: [PATCH 241/397] RATIS-2380. repeat-test.yaml is vulnerable to parameter injection attacks (#1333) --- .github/workflows/repeat-test.yaml | 9 ++++++++- 1 file changed, 8 insertions(+), 1 deletion(-) diff --git a/.github/workflows/repeat-test.yaml b/.github/workflows/repeat-test.yaml index de787d9291..11b77b1d35 100644 --- a/.github/workflows/repeat-test.yaml +++ b/.github/workflows/repeat-test.yaml @@ -55,6 +55,7 @@ jobs: outputs: matrix: ${{ steps.generate.outputs.matrix }} test-spec: ${{ steps.test-spec.outputs.test-spec }} + ref: ${{ steps.ref.outputs.ref }} steps: - id: generate name: Generate test matrix @@ -76,6 +77,12 @@ jobs: fi echo "Test to be run: $test_spec" echo "test-spec=$test_spec" >> $GITHUB_OUTPUT + - name: Define checkout ref + id: ref + run: | + echo "ref=$REF" >> $GITHUB_OUTPUT + env: + REF: ${{ github.event.inputs.ref }} test: if: ${{ always() }} needs: @@ -90,7 +97,7 @@ jobs: steps: - uses: actions/checkout@v4 with: - ref: ${{ github.event.inputs.ref }} + ref: ${{ needs.prepare.outputs.ref }} - name: Cache for maven dependencies uses: actions/cache@v4 with: From a63ea32ca9f9481e3d9def93aac2e07c6adaff06 Mon Sep 17 00:00:00 2001 From: Ivan Andika Date: Wed, 21 Jan 2026 02:27:05 +0800 Subject: [PATCH 242/397] RATIS-2383. AtomicFileOutputStream should override FilterOutputStream batch write API (#1336) --- .../java/org/apache/ratis/util/AtomicFileOutputStream.java | 5 +++++ 1 file changed, 5 insertions(+) diff --git a/ratis-common/src/main/java/org/apache/ratis/util/AtomicFileOutputStream.java b/ratis-common/src/main/java/org/apache/ratis/util/AtomicFileOutputStream.java index b54427066f..3961d7336b 100644 --- a/ratis-common/src/main/java/org/apache/ratis/util/AtomicFileOutputStream.java +++ b/ratis-common/src/main/java/org/apache/ratis/util/AtomicFileOutputStream.java @@ -68,6 +68,11 @@ public AtomicFileOutputStream(File outFile, File tmpFile) throws IOException { this.tmpFile = tmpFile.getAbsoluteFile(); } + @Override + public void write(byte[] b, int off, int len) throws IOException { + out.write(b, off, len); + } + public boolean isClosed() { return isClosed.get(); } From cceeae9f4eaa18a187bb1a9860c87f4edacfdda7 Mon Sep 17 00:00:00 2001 From: "Doroszlai, Attila" <6454655+adoroszlai@users.noreply.github.com> Date: Thu, 22 Jan 2026 10:27:35 +0100 Subject: [PATCH 243/397] RATIS-2384. Add Bill of Materials module (#1335) --- pom.xml | 3 +- ratis-assembly/src/main/assembly/src.xml | 1 + ratis-bom/pom.xml | 128 +++++++++++++++++++++++ 3 files changed, 131 insertions(+), 1 deletion(-) create mode 100644 ratis-bom/pom.xml diff --git a/pom.xml b/pom.xml index 41c32c7c24..e3b7fcb9f0 100644 --- a/pom.xml +++ b/pom.xml @@ -73,6 +73,7 @@ ratis-tools ratis-shell ratis-assembly + ratis-bom @@ -181,7 +182,7 @@ ${maven.compiler.release} - + 1.0.10 diff --git a/ratis-assembly/src/main/assembly/src.xml b/ratis-assembly/src/main/assembly/src.xml index fdc64ed2ad..e20770d54d 100644 --- a/ratis-assembly/src/main/assembly/src.xml +++ b/ratis-assembly/src/main/assembly/src.xml @@ -31,6 +31,7 @@ true org.apache.ratis:ratis-assembly + org.apache.ratis:ratis-bom org.apache.ratis:ratis-client org.apache.ratis:ratis-common org.apache.ratis:ratis-examples diff --git a/ratis-bom/pom.xml b/ratis-bom/pom.xml new file mode 100644 index 0000000000..c023790cf5 --- /dev/null +++ b/ratis-bom/pom.xml @@ -0,0 +1,128 @@ + + + + 4.0.0 + org.apache.ratis + ratis-bom + 3.3.0-SNAPSHOT + Apache Ratis BOM + Apache Ratis Bill of Materials (BOM) + + pom + + + 1.0.10 + + true + + + + + + org.apache.ratis + ratis-client + ${project.version} + + + org.apache.ratis + ratis-common + ${project.version} + + + org.apache.ratis + ratis-docs + ${project.version} + + + org.apache.ratis + ratis-examples + ${project.version} + + + org.apache.ratis + ratis-experiments + ${project.version} + + + org.apache.ratis + ratis-grpc + ${project.version} + + + org.apache.ratis + ratis-metrics-api + ${project.version} + + + org.apache.ratis + ratis-metrics-default + ${project.version} + + + org.apache.ratis + ratis-metrics-dropwizard3 + ${project.version} + + + org.apache.ratis + ratis-netty + ${project.version} + + + org.apache.ratis + ratis-proto + ${project.version} + + + org.apache.ratis + ratis-server + ${project.version} + + + org.apache.ratis + ratis-server-api + ${project.version} + + + org.apache.ratis + ratis-shell + ${project.version} + + + org.apache.ratis + ratis-thirdparty-misc + ${ratis.thirdparty.version} + + + org.apache.ratis + ratis-tools + ${project.version} + + + + + + + + com.github.spotbugs + spotbugs-maven-plugin + 4.8.6.2 + + true + + + + + From eaff520f4dff4ac3da198ecbee7563d118030245 Mon Sep 17 00:00:00 2001 From: Sergey Soldatov Date: Wed, 28 Jan 2026 23:49:11 -0800 Subject: [PATCH 244/397] RATIS-2387. Performance degradation after RATIS-2235 (#1337) --- .../ratis/server/RaftServerConfigKeys.java | 10 +++++++ .../ratis/server/impl/RaftServerImpl.java | 29 +++++++------------ .../ratis/server/impl/ServerImplUtils.java | 19 ++++++++++-- 3 files changed, 37 insertions(+), 21 deletions(-) diff --git a/ratis-server-api/src/main/java/org/apache/ratis/server/RaftServerConfigKeys.java b/ratis-server-api/src/main/java/org/apache/ratis/server/RaftServerConfigKeys.java index 2538a472a8..efb3c67963 100644 --- a/ratis-server-api/src/main/java/org/apache/ratis/server/RaftServerConfigKeys.java +++ b/ratis-server-api/src/main/java/org/apache/ratis/server/RaftServerConfigKeys.java @@ -440,6 +440,16 @@ static void setReadLockEnabled(RaftProperties properties, boolean readLockEnable setBoolean(properties::setBoolean, READ_LOCK_ENABLED_KEY, readLockEnabled); } + String APPEND_ENTRIES_COMPOSE_ENABLED_KEY = PREFIX + ".append-entries.compose.enabled"; + boolean APPEND_ENTRIES_COMPOSE_ENABLED_DEFAULT = true; + static boolean appendEntriesComposeEnabled(RaftProperties properties) { + return getBoolean(properties::getBoolean, + APPEND_ENTRIES_COMPOSE_ENABLED_KEY, APPEND_ENTRIES_COMPOSE_ENABLED_DEFAULT, getDefaultLog()); + } + static void setAppendEntriesComposeEnabled(RaftProperties properties, boolean enabled) { + setBoolean(properties::setBoolean, APPEND_ENTRIES_COMPOSE_ENABLED_KEY, enabled); + } + /** * Besides the open segment, the max number of segments caching log entries. */ diff --git a/ratis-server/src/main/java/org/apache/ratis/server/impl/RaftServerImpl.java b/ratis-server/src/main/java/org/apache/ratis/server/impl/RaftServerImpl.java index 60f72e001e..846b87702f 100644 --- a/ratis-server/src/main/java/org/apache/ratis/server/impl/RaftServerImpl.java +++ b/ratis-server/src/main/java/org/apache/ratis/server/impl/RaftServerImpl.java @@ -81,7 +81,6 @@ import org.apache.ratis.server.RaftServerRpc; import org.apache.ratis.server.impl.LeaderElection.Phase; import org.apache.ratis.server.impl.RetryCacheImpl.CacheEntry; -import org.apache.ratis.server.impl.ServerImplUtils.ConsecutiveIndices; import org.apache.ratis.server.impl.ServerImplUtils.NavigableIndices; import org.apache.ratis.server.leader.LeaderState.StepDownReason; import org.apache.ratis.server.metrics.LeaderElectionMetrics; @@ -133,7 +132,6 @@ import java.util.concurrent.ThreadLocalRandom; import java.util.concurrent.TimeUnit; import java.util.concurrent.atomic.AtomicBoolean; -import java.util.concurrent.atomic.AtomicReference; import java.util.function.Function; import java.util.function.Supplier; import java.util.stream.Collectors; @@ -260,8 +258,7 @@ public long[] getFollowerMatchIndices() { private final AtomicBoolean firstElectionSinceStartup = new AtomicBoolean(true); private final ThreadGroup threadGroup; - private final AtomicReference> appendLogFuture; - private final NavigableIndices appendLogTermIndices = new NavigableIndices(); + private final NavigableIndices appendLogTermIndices; RaftServerImpl(RaftGroup group, StateMachine stateMachine, RaftServerProxy proxy, RaftStorage.StartupOption option) throws IOException { @@ -296,7 +293,8 @@ public long[] getFollowerMatchIndices() { this.transferLeadership = new TransferLeadership(this, properties); this.snapshotRequestHandler = new SnapshotManagementRequestHandler(this); this.snapshotInstallationHandler = new SnapshotInstallationHandler(this, properties); - this.appendLogFuture = new AtomicReference<>(CompletableFuture.completedFuture(null)); + this.appendLogTermIndices = RaftServerConfigKeys.Log.appendEntriesComposeEnabled(properties) ? + new NavigableIndices() : null; this.serverExecutor = ConcurrentUtils.newThreadPoolWithMax( RaftServerConfigKeys.ThreadPool.serverCached(properties), @@ -1620,7 +1618,8 @@ leaderId, getMemberId(), currentTerm, followerCommit, inconsistencyReplyNextInde state.updateConfiguration(entries); } future.join(); - final CompletableFuture appendLog = entries.isEmpty()? CompletableFuture.completedFuture(null) + final CompletableFuture appendFuture = entries.isEmpty()? CompletableFuture.completedFuture(null) + : appendLogTermIndices != null ? appendLogTermIndices.append(entries, this::appendLog) : appendLog(entries); proto.getCommitInfosList().forEach(commitInfoCache::update); @@ -1636,7 +1635,7 @@ leaderId, getMemberId(), currentTerm, followerCommit, inconsistencyReplyNextInde final long commitIndex = effectiveCommitIndex(proto.getLeaderCommit(), previous, entries.size()); final long matchIndex = isHeartbeat? RaftLog.INVALID_LOG_INDEX: entries.get(entries.size() - 1).getIndex(); - return appendLog.whenCompleteAsync((r, t) -> { + return appendFuture.whenCompleteAsync((r, t) -> { followerState.ifPresent(fs -> fs.updateLastRpcTime(FollowerState.UpdateType.APPEND_COMPLETE)); timer.stop(); }, getServerExecutor()).thenApply(v -> { @@ -1654,16 +1653,8 @@ leaderId, getMemberId(), currentTerm, followerCommit, inconsistencyReplyNextInde }); } private CompletableFuture appendLog(List entries) { - final List entriesTermIndices = ConsecutiveIndices.convert(entries); - if (!appendLogTermIndices.append(entriesTermIndices)) { - // index already exists, return the last future - return appendLogFuture.get(); - } - - - return appendLogFuture.updateAndGet(f -> f.thenComposeAsync( - ignored -> JavaUtils.allOf(state.getLog().append(entries)), serverExecutor)) - .whenComplete((v, e) -> appendLogTermIndices.removeExisting(entriesTermIndices)); + return CompletableFuture.completedFuture(null) + .thenComposeAsync(dummy -> JavaUtils.allOf(state.getLog().append(entries)), serverExecutor); } private long checkInconsistentAppendEntries(TermIndex previous, List entries) { @@ -1690,7 +1681,9 @@ private long checkInconsistentAppendEntries(TermIndex previous, List map = new TreeMap<>(); + private final AtomicReference> future + = new AtomicReference<>(CompletableFuture.completedFuture(null)); boolean contains(TermIndex ti) { final Long term = getTerm(ti.getIndex()); @@ -137,7 +142,15 @@ synchronized Long getTerm(long index) { return floorEntry.getValue().getTerm(index); } - synchronized boolean append(List entriesTermIndices) { + CompletableFuture append(List entries, + Function, CompletableFuture> appendLog) { + final List entriesTermIndices = ConsecutiveIndices.convert(entries); + return alreadyExists(entriesTermIndices) ? future.get() + : future.updateAndGet(f -> f.thenComposeAsync(ignored -> appendLog.apply(entries))) + .whenComplete((v, e) -> removeExisting(entriesTermIndices)); + } + + private synchronized boolean alreadyExists(List entriesTermIndices) { for(int i = 0; i < entriesTermIndices.size(); i++) { final ConsecutiveIndices indices = entriesTermIndices.get(i); final ConsecutiveIndices previous = map.put(indices.startIndex, indices); @@ -147,10 +160,10 @@ synchronized boolean append(List entriesTermIndices) { for(int j = 0; j < i; j++) { map.remove(entriesTermIndices.get(j).startIndex); } - return false; + return true; } } - return true; + return false; } synchronized void removeExisting(List entriesTermIndices) { From 1022fc413cc484f02e8c0107387c730ed1d479f5 Mon Sep 17 00:00:00 2001 From: Joel Date: Tue, 3 Feb 2026 19:33:18 +0000 Subject: [PATCH 245/397] RATIS-2388 (Further) Enhancing content for concept in ratis-docs (#1338) --- .../src/site/markdown/concept/advanced.md | 85 ++++++++ .../site/markdown/concept/core-concepts.md | 200 ++++++++++++++++++ ratis-docs/src/site/markdown/concept/index.md | 166 ++++++++------- .../src/site/markdown/concept/integration.md | 107 ++++++++++ .../src/site/markdown/concept/operations.md | 137 ++++++++++++ 5 files changed, 621 insertions(+), 74 deletions(-) create mode 100644 ratis-docs/src/site/markdown/concept/advanced.md create mode 100644 ratis-docs/src/site/markdown/concept/core-concepts.md create mode 100644 ratis-docs/src/site/markdown/concept/integration.md create mode 100644 ratis-docs/src/site/markdown/concept/operations.md diff --git a/ratis-docs/src/site/markdown/concept/advanced.md b/ratis-docs/src/site/markdown/concept/advanced.md new file mode 100644 index 0000000000..58b4f8e3a5 --- /dev/null +++ b/ratis-docs/src/site/markdown/concept/advanced.md @@ -0,0 +1,85 @@ + +# Introduction to Apache Ratis + +Previous: [Operations and Management](operations.md) | Top:[Overview of Raft and Ratis](index.md) + +## Section 5: Advanced Topics + +* [Scaling with Multi-Raft Groups](#scaling-with-multi-raft-groups) + +### Scaling with Multi-Raft Groups + +As your application grows, you may find that a single Raft group becomes a bottleneck. This is +where Ratis's multi-group capability becomes valuable. + +#### Understanding Multi-Raft + +Multi-Raft is an implementation pattern that Ratis supports for scaling beyond the limits of a +single Raft group. In a multi-Raft setup, you run multiple independent Raft groups, each +handling a subset of your application's operations. Each group operates independently with its +own leader election, consensus, log, and state machine. + +#### What is a Raft Group in Ratis? + +In Ratis terminology, a "Raft Group" is a collection of servers that participate in a single +Raft cluster. Each group has a unique RaftGroupId (a UUID) that distinguishes it from other groups. +Each group consists of a set of RaftPeer objects representing the servers that participate in that +group's consensus. + +#### When to Use Multiple Groups + +Consider using multiple Raft groups when a single group cannot handle the required throughput, +when you can logically partition your data or operations (such as having one group per geographic +region, per customer tenant, or per data type), when you need better fault isolation (if one +group fails, other groups can continue operating), or when you need different operational +characteristics for different parts of your system. + +#### Implementation Considerations + +A single RaftServer instance can participate in multiple groups simultaneously. Each group gets +its own "Division" within the server, with its own state machine and storage. Since groups don't +coordinate at the Raft level, your application must handle any cross-group consistency +requirements through distributed transactions, saga patterns, or eventual consistency approaches. + +To use multi-Raft effectively, you need to partition your application state. Horizontal +partitioning involves partitioning data across groups based on some key (e.g., user ID hash, +geographic region). Functional partitioning assigns different groups to handle different types +of operations or services. Hierarchical partitioning uses a tree-like structure where +higher-level groups coordinate lower-level groups. + +Clients need to know which group to send requests to through client-side routing logic, a proxy +layer that routes requests, or consistent hashing schemes. + +#### Trade-offs and Limitations + +Multi-group setups are significantly more complex than single-group setups. Maintaining +consistency across groups requires application-level coordination, which can be complex and +error-prone. More groups mean more leaders to monitor, more logs to manage, and more complex +failure scenarios. Each group consumes resources, so there's a practical limit to the number of +groups per server. + +#### Best Practices + +Begin with a single group and only move to multiple groups when you have a clear scalability +need. Design your data model and operations to be partition-friendly from the start if you +anticipate needing multiple groups. Implement comprehensive monitoring for all groups, including +leader stability, replication lag, and resource usage. + +Multi-Raft groups are a powerful scaling tool, but they should be used judiciously. The added +complexity is only worthwhile when you have clear scalability requirements that cannot be met +with a single Raft cluster. diff --git a/ratis-docs/src/site/markdown/concept/core-concepts.md b/ratis-docs/src/site/markdown/concept/core-concepts.md new file mode 100644 index 0000000000..a97e6e91ca --- /dev/null +++ b/ratis-docs/src/site/markdown/concept/core-concepts.md @@ -0,0 +1,200 @@ + +# Introduction to Apache Ratis + +Previous: [Overview of Raft and Ratis](index.md) | Top:[Overview of Raft and Ratis](index.md) + +## Section 2: Core Concepts + +* [The Raft Log](#the-raft-log---foundation-of-consensus) +* [The State Machine](#the-state-machine---your-applications-heart) +* [Consistency Models and Read Patterns](#consistency-models-and-read-patterns) + +### The Raft Log - Foundation of Consensus + +The Raft log is the central data structure that makes distributed consensus possible. Each server +in a Raft group maintains its own copy of this append-only ledger that records every operation +in the exact order they should be applied to the state machine. + +Each entry in the log contains three key pieces of information: the operation itself (what should +be done), a log index (a sequential number indicating the entry's position), and a term number +(the period during which a leader created this entry). Terms represent periods of leadership and +increase each time a new leader is elected, preventing old leaders from overwriting newer entries. +The combination of the term and log index is referred to as a term-index (`TermIndex`) and +establishes the ordering of entries in the log. + +The log serves as both the mechanism for replication (leaders send log entries to followers) and +the source of truth for recovery (servers can rebuild their state by replaying the log). When we +talk about "committing" an operation, we mean that a majority of servers have acknowledged +storing that log entry, making it safe to apply to the state machine. + +### The State Machine - Your Application's Heart + +In Ratis, the state machine is your application's primary integration point. Your business logic +or data storage operations are implemented by the state machine. + +The state machine is a deterministic computation engine that processes a sequence of operations +and maintains some internal state. The state machine must be deterministic: given the same +sequence of operations, it must always produce the same results and end up in the same final state. +Operations are processed sequentially, one at a time, in the order they appear in the Raft log. + +#### State Machine Responsibilities + +Your state machine has three primary responsibilities. First, it processes Raft transactions by +validating incoming requests before they're replicated and applying committed operations to your +application state. Second, it maintains your application's actual data, which might be an +in-memory data structure, a local database, files on disk, or any combination of these. Third, +it creates point-in-time representations of its state (snapshots) and can restore its state from +snapshots during recovery. + +#### The State Machine Lifecycle + +The state machine operates at two different lifecycle levels: an overall peer lifecycle and a +per-transaction processing lifecycle. + +##### Peer Lifecycle + +During initialization, when a peer starts up, the state machine loads any existing snapshots and +prepares its internal data structures. The Raft layer then replays any log entries that occurred +after the snapshot, bringing the peer up to the current state of the group. + +During normal operation, the state machine continuously processes transactions as they're +committed by the Raft group, handles read-only queries, and may respond to changes in the node's +status as a leader or follower. For read-only operations, the state machine can answer queries +directly without going through the Raft log, providing better performance for reads but with +[consistency trade-offs](#consistency-models-and-read-patterns). + +Periodically, the state machine creates snapshots of its current state. This happens either +automatically based on configuration (like log size thresholds) or manually through +administrative commands. + +##### Transaction Processing Lifecycle + +For each individual transaction, the state machine follows a multistep processing sequence. In +the validation phase, the leader's state machine examines incoming requests through the +`startTransaction` method. This is where you validate that the operation is properly structured +and valid in the current context. + +In the pre-append phase, just before the operation is written to the log, the state machine can +perform any final preparations through the `preAppendTransaction` method. After the operation is +committed by the Raft group, the state machine is notified via `applyTransactionSerial` and can +handle any order-sensitive logic that must happen before the main application logic is invoked. + +Finally, in the application phase, the operation is applied to the actual application state +through the `applyTransaction` method. This is where your business logic executes and where the +operation's effects become visible to future queries. + +#### Designing Your State Machine + +When designing your state machine, ensure your operations are deterministic and can be efficiently +serialized for replication. Operations are not required to be idempotent because the Raft protocol +ensures that each operation is applied exactly once on each peer, however idempotent operations may +make it easier to reason about your application. + +Plan how you'll represent your application's state for both runtime efficiency and snapshot +serialization. If your state machine maintains state in external systems (databases, files), +ensure your snapshot process captures this external state consistently. + +Robust error handling is crucial. Server-side errors require distinguishing between recoverable +errors (like validation failures) and fatal errors (like storage failures). Errors in +`startTransaction` prevent operations from being committed and replicated. Errors in +`applyTransaction` are considered fatal since they indicate the state machine cannot process +already-committed operations. + +### Consistency Models and Read Patterns + +In a distributed system, consistency refers to the guarantees you have about seeing the effects +of write operations when you read data. For write operations, Raft and Ratis provide strong +consistency: once a write operation is acknowledged as committed, all subsequent reads will see +the effects of that write. Read operations are more complex because Ratis offers several +different approaches with different consistency and performance characteristics. + +#### Write Consistency + +Write operations in Ratis follow a straightforward path that provides strong consistency. Clients +send write requests to the leader, which validates the operation through the state machine's +`startTransaction` method, then replicates it to a majority of followers. Once a majority +acknowledges, the operation is committed. The leader applies the operation to its state machine +and returns the result to the client, while followers eventually apply the same operation in the +same order. + +#### Read Consistency Options + +Ratis provides several read patterns with different consistency and performance characteristics. + +Read requests query the state machine of a server directly without going through the Raft consensus +protocol. The `sendReadOnly()` API sends a read request to the leader. If a non-leader server +receives such request, it throws a `NotLeaderException` and then the client will retry other +servers. In contrast, the `sendReadOnly(message, serverId)` API sends the request to a particular +server, which may be a leader or a follower. + +The server's `raft.server.read.option` configuration affects read consistency behavior: + +* **DEFAULT (default setting)**: `sendReadOnly()` performs leader reads for efficiency. It provides +strong consistency under normal conditions. However, In case that an old leader has been +partitioned from the majority and a new leader has been elected, reading from the old leader can +return stale data since the old leader does not have the new transactions committed by the new +leader (referred to as the "split-brain problem"). +* **LINEARIZABLE**: both `sendReadOnly()` and `sendReadOnly(message, serverId)` use the ReadIndex +protocol to provide linearizable consistency, ensuring you always read the most up-to-date committed +data and won't read stale data as described in the "Split-brain Problem" above. + * Non-linearizable API: Clients may use `sendReadOnlyNonLinearizable()` to read from leader's + state machine directly without a linearizable guarantee. + +Server-side configuration allows operators to choose between performance (leader reads) and strong +consistency guarantees (linearizable reads) for their entire cluster. + +Stale reads with minimum index let you specify a minimum log index that the peer must have +applied before serving the read. Call `sendStaleRead()`: if the peer hasn't caught up to your +minimum index, it will throw a `StaleReadException`. + +In summary: +* **Leader reads** query the current leader's state machine directly without going through the Raft +consensus protocol. Call `sendReadOnly()` for the strongest consistency supported by the server. +* Use`sendReadOnlyNonLinearizable()` for leader reads without a linearizable guarantee. +* Use `sendReadOnly(message, serverId)` with a specific follower's server ID for **follower reads**, +which offer better performance but may return stale data. +* Use `sendStaleRead()` to specify the minimum log index that the server must have applied. +* Use `sendReadAfterWrite()` to ensure the read reflects the latest successful write by the +same client, for **read-after-write consistency**. + +Note that all of these operations may be performing as blocking or async operations. See +[Client API Patterns](integration.md#client-api-patterns) for more information. + +#### The Query Method and Read-Only Operations + +The state machine's `query` method enables you to handle read-only operations without going +through the Raft protocol. This provides significant performance benefits but requires careful +consideration of consistency requirements. Your state machine's `query` method will be called +for explicit read-only requests from clients, queries that need to read state without modifying +it, and health checks or monitoring queries. + +#### Choosing the Right Read Pattern + +Use **linearizable reads** when correctness is more important than performance, you need to read +your own writes immediately, or the application cannot tolerate any stale data. Use **leader +reads** when you need strong consistency but can tolerate very brief staleness during network +partitions, or when building interactive applications where users expect to see their recent +changes. + +Use **follower reads** when you can tolerate stale data in exchange for better performance and +availability, you're implementing read replicas for scaling read-heavy workloads, or the data +being read doesn't change frequently. Use **stale reads** when you need fine-grained control +over the consistency/performance trade-off. + +--- +Next: [Integration](integration.md) \ No newline at end of file diff --git a/ratis-docs/src/site/markdown/concept/index.md b/ratis-docs/src/site/markdown/concept/index.md index 6ff77b2203..3843221a69 100644 --- a/ratis-docs/src/site/markdown/concept/index.md +++ b/ratis-docs/src/site/markdown/concept/index.md @@ -14,77 +14,95 @@ See the License for the specific language governing permissions and limitations under the License. --> -# Concepts - -## RaftServer - -The `RaftServer` is a core component of Apache Ratis, -responsible for handling all operations related to the RAFT protocol. -A `RaftServer` manages all the `RaftGroup`(s) within the current process. -For each group, a corresponding `RaftServer.Division` handles the core functions -such as replication of log entries, voting, and leader election within that group. -Each division can independently act as a Leader, Candidate, Follower or Listener, -with the specific role depending on the outcome of elections and the state of the protocol. - -## RaftPeer and RaftPeerId - -`RaftPeer` represents a participant node in the cluster, -including the node's unique identifier, IP address, and port number. -The unique identifier is represented by the `RaftPeerId` class, -which is crucial for distinguishing different nodes within a cluster. - -## RaftGroup - -A `RaftGroup` represents a collection of `RaftPeer`(s) in a Raft protocol cluster. -Each group has a unique identifier represented by the `RaftGroupId` class. -Multiple groups can operate independently within a physical network, -while each group managing its own consistency and state replication. - -## Transport (gRPC, Netty, etc.) - -Ratis supports various network transport protocols for node communication, -including gRPC (default) and Netty. -These transport layers in Ratis are used for data serialization and deserialization, -as well as ensuring safe and efficient data transmission between different nodes. - -## RaftLog - -The `RaftLog` is a core component of the Raft algorithm, -used to record all state change transactions. -Once a log entry has been acknowledged by a majority of peers, -the entry becomes committed. -The Raft log is key to achieving distributed data consistency. - -## Snapshot - -A `Snapshot` is a point-in-time copy of the current state of the `StateMachine`. -It can be used for quick recovery of the state after system restarts, -and for transferring the state to newly joined nodes. -When a snapshot has been taken, -the log entries earlier than the snapshot can be purged -in order to free up the storage space. - -## TermIndex - -`TermIndex` is an order pair of `long` integers (term, index) as defined in the Raft protocol. -Term is the logical clock in Raft. -A newly elected leader starts a new term and remains the leader for the rest of the term. -Index is the position of log entries in the Raft log. - -## StateMachine - -In Ratis, `StateMachine` is the abstraction point for user-defined code. -Developers implement specific business logic or data storage operations at this layer. -The transactions committed through the Raft protocol will be applied to it. - -### The `applyTransaction` method - -In Ratis, transaction processing is implemented by the `StateMachine` -through the `applyTransaction` method. -A transaction usually changes the state of the `StateMachine`. - -### StateMachineStorage - -`StateMachineStorage` is a component for storing data related to the `StateMachine`. -It is for persisting the Raft log and the snapshots -such that the state can be fully recovered even after system failures. +# Introduction to Apache Ratis + +## Sections +1. [Overview](index.md#section-1) +2. [Core Concepts](core-concepts.md) +3. [Integration Guide](integration.md) +4. [Operations and Management](operations.md) +5. [Advanced Topics](advanced.md) + + + +## Section 1: Overview of Raft and Apache Ratis + +* [Introduction to Raft and Apache Ratis](#raft-and-apache-ratis) +* [Raft Cluster Topology](#raft-cluster-topology) + +### Raft and Apache Ratis + +The Raft consensus algorithm solves a fundamental problem in distributed systems: how do you get +multiple computers to agree on a sequence of operations, even when some might fail or become +unreachable? This problem, known as distributed consensus, is at the heart of building reliable +distributed systems. + +Raft ensures that a cluster of servers maintains an identical, ordered log of operations. Each +server applies these operations to its local state machine in the same order, guaranteeing that +all servers end up with identical state. This approach, called state machine replication, +provides both consistency and fault tolerance. + +You should consider using Raft when your system needs strong consistency guarantees across +multiple servers. This typically applies to systems where correctness is more important than +absolute performance, such as distributed databases, configuration management systems, or any +application where split-brain scenarios would be unacceptable. + +Apache Ratis is a Java library that implements the Raft consensus protocol. The key word here +is "library" - Ratis is not a standalone service that you communicate with over the network. +Instead, you embed Ratis directly into your Java application, and it becomes part of your +application's runtime. + +This embedded approach creates tight integration between your application and the consensus +mechanism. Your application and Ratis run in the same JVM, sharing memory and computational +resources. Your application provides the business logic (the "state machine" in Raft terminology), +while Ratis handles the distributed consensus mechanics needed to keep multiple instances of your +application synchronized. + +### Raft Cluster Topology + +Understanding the basic building blocks of a Raft deployment affects both the correctness and +performance of your system. + +#### Servers, Clusters, and Groups + +A Raft server (also known as a "peer" or "member") is a single running instance of your application +with Ratis embedded. Each server runs your state machine and participates in the consensus +protocol. + +A Raft cluster is a physical collection of servers that can participate in consensus. A Raft +group is a logical consensus domain that runs across a specific subset of peers in the cluster. +One of the peers in a group acts as the "leader" while the others are "followers" or "listeners". +The leader handles all write requests and replicates operations to other peers in the group. Both +leaders and followers can service read requests, with different consistency guarantees. A single +cluster can host multiple independent Raft groups, each with its own leader election, consistency +and state replication. + +#### Majority-Based Decision-Making + +Raft's safety guarantees depend on majority agreement within each group. The leader replicates +each operation to the followers in its group, and operations are committed when at least +$\lfloor N/2 + 1 \rfloor$ peers in that group acknowledge them. This means a group of 3 peers can +tolerate 1 failure, a group of five peers can tolerate 2 failures, and so on. Since a group of +$N$ peers for an even $N$ can tolerate the same number of failures as a group of $(N-1)$ peers, +groups typically consist of an odd number of peers (3, 5, or 7 are common) to ensure clear +majority decisions. + +This majority requirement affects both availability and performance. A group remains available as +long as a majority of its peers are reachable and functioning. However, every transaction must +wait for majority acknowledgment, so the slowest server in the majority determines your write +latency. + +#### Server Placement and Network Considerations + +The physical and network placement of your servers impacts both availability and performance. +Placing all servers in the same rack or data center provides the lowest latency but risks +creating a single point of failure. Distributing servers across multiple availability zones or +data centers improves fault tolerance but can increase latency. + +A common approach is to place servers across multiple availability zones within a single region +for a balance of fault tolerance and performance. For applications requiring geographic +distribution, you might place servers in different regions, accepting higher latency in exchange +for better disaster recovery capabilities. + +--- +Next: [Core Concepts](core-concepts.md) \ No newline at end of file diff --git a/ratis-docs/src/site/markdown/concept/integration.md b/ratis-docs/src/site/markdown/concept/integration.md new file mode 100644 index 0000000000..352f1308df --- /dev/null +++ b/ratis-docs/src/site/markdown/concept/integration.md @@ -0,0 +1,107 @@ + +# Introduction to Apache Ratis + +Previous: [Core Concepts](core-concepts.md) | Top:[Overview of Raft and Ratis](index.md) + +## Section 3: Integration + +* [Logical Organization of Ratis](#logical-organization-of-ratis) +* [Server Configuration and Lifecycle](#server-configuration-and-lifecycle) + +### Logical Organization of Ratis + +Rather than focusing on package structure, let's examine the logical components and their +relationships, understanding how they work together to provide the Raft consensus functionality. + +#### Primary Integration Points + +When integrating with Ratis, you'll work with a small set of key classes and interfaces. + +`StateMachine` Interface - This is where you'll spend most of your development time. Your +application implements this interface to define what operations mean and how they affect your +data. Key methods include `startTransaction()` to validate requests, `applyTransaction()` to +process committed operations, `query()` to handle reads, and `takeSnapshot()` to checkpoint your +application state. + +`RaftClient` - Your application uses this to send requests to the Raft cluster. It handles +leader discovery, retries, and connection management automatically. You'll primarily use `send()` +for writes, `sendReadOnly()` for consistent reads, and `sendStaleRead()` for performance- +optimized reads. + +`RaftServer` - This hosts your `StateMachine` and handles the Raft protocol. You'll configure +and start it, but most interaction happens through your `StateMachine` implementation. One +server can participate in multiple Raft groups simultaneously. + +`RaftGroup` and `RaftPeer` - These define your cluster topology. `RaftGroup` represents a +consensus domain (which peers participate in a group), while `RaftPeer` represents individual +servers (their IDs and network addresses). + +Configuration Classes - `RaftProperties` and related classes control behavior like timeouts, +storage locations, and transport settings. + +Message and Request Types - Your operations flow through the system as `Message` objects. These +are serializable containers that carry your application's operations from clients to the +`StateMachine`. The `Message` interface is simple but designing your message types thoughtfully +affects both performance and maintainability. + +#### Client API Patterns + +`RaftClient` provides several API styles to match different application patterns. The +`BlockingApi`, accessed through `RaftClient.io()`, offers traditional synchronous operations: +simple to use and understand, ideal when simplicity matters more than maximum throughput. The +`AsyncApi`, accessed through `RaftClient.async()` provides non-blocking operations that return +`CompletableFuture` objects, allowing your application to send multiple requests concurrently. + +For applications that need to transfer large amounts of data, the `DataStreamApi` provides +efficient streaming that bypasses the normal Raft log for the data payload itself. Instead of +sending large payloads through the consensus mechanism, you stream data directly to peers while +still maintaining ordering and consistency guarantees through the Raft protocol. + +The `AdminApi` handles cluster management operations like adding or removing peers, triggering +snapshots, and querying cluster status. + +#### Request Flow Through the System + +When your application calls `RaftClient.send(message)`, the `RaftClient` first determines which +server to contact, handling leader discovery automatically. If the contacted server isn't the +current leader, it returns a `NotLeaderException` with information about the actual leader. + +Once the message reaches the leader, your `StateMachine`'s `startTransaction(message)` method +validates the request. If validation succeeds, the leader replicates the operation through the +Raft protocol to a majority of followers. After the operation is committed, the leader calls +your `StateMachine`'s `applyTransaction(message)` method to execute the business logic. + +The result flows back to the client, while followers eventually receive and apply the same +operation through their own `applyTransaction` calls. Read-only operations can bypass this flow +by going directly to the `query` method, trading consistency guarantees for better performance. + + +### Server Configuration and Lifecycle + +`RaftServer` is the main server-side entry point, but it requires several configuration decisions +before startup. You'll need to choose a transport implementation (gRPC works well for most +deployments, while Netty provides more control), storage configuration including directories for +logs and snapshots, and key configuration like timeout and retry policies, snapshot policies, +and security settings. + +A single `RaftServer` instance can participate in [multiple Raft groups](advanced.md) +simultaneously through Ratis's `Division` concept. Each group gets its own state machine instance +and storage within the server. + +--- +Next: [Operations and Management](operations.md) \ No newline at end of file diff --git a/ratis-docs/src/site/markdown/concept/operations.md b/ratis-docs/src/site/markdown/concept/operations.md new file mode 100644 index 0000000000..11e774d715 --- /dev/null +++ b/ratis-docs/src/site/markdown/concept/operations.md @@ -0,0 +1,137 @@ + +# Introduction to Apache Ratis + +Previous: [Integration](integration.md) | Top:[Overview of Raft and Ratis](index.md) + +## Section 4: Operations and Management + +* [Snapshots](#snapshots---managing-growth-and-recovery) +* [Leadership and Fault Tolerance](#leadership-and-fault-tolerance) + +### Snapshots - Managing Growth and Recovery + +Snapshots are a point-in-time representation of your state machine's complete state, along with +metadata about which log entries are included in that state. They prevent the log from growing +without bound and enable efficient recovery and catch-up for peers that have fallen behind. + +The snapshot includes the actual application state, the term-index of the last log entry that +contributed to this state, and the Raft group configuration at the time the snapshot was taken. + +Without snapshots, the Raft log would grow indefinitely, eventually consuming all available +storage. Crashed peers would need to replay potentially millions of log entries to catch up, +dramatically slowing recovery. New peers joining an established group would need to process the +entire history of the group, which could take hours or days for active systems. + +#### Creating Snapshots + +Snapshots can be created automatically when the log grows beyond a certain size, manually +triggered through the admin API, or sent by the leader to peers that are far behind instead of +replaying thousands of log entries. + +When your state machine's `takeSnapshot` method is called, it needs to create a consistent view +of your application state. This might involve pausing writes, creating a database transaction, +or using copy-on-write data structures. The method must serialize state by writing it to durable +storage in a format that can be read back later, record which term-index the snapshot represents, +and return the log index so Ratis can safely discard older log entries. + +Different applications will have different strategies for snapshot creation. A stop-the-world +approach pauses all operations while creating the snapshot: simple but impacts availability. +Copy-on-write uses data structures that support efficient point-in-time copies. Database +transactions can create consistent snapshots if your state is in a database. Some storage +engines support checkpointing to leverage native snapshot capabilities. + +#### Snapshot Installation and Recovery + +When a peer needs to catch up using a snapshot, it receives the snapshot data from the leader or +loads it from local storage. The state machine is paused to prevent conflicts during restoration, +the snapshot data is loaded replacing any existing state, and the state machine resumes normal +operation by replaying any log entries that occurred after the snapshot. + +Your state machine's `reinitialize` method is responsible for loading snapshots during startup by +loading the latest snapshot if available, with the Raft layer replaying any log entries after +the snapshot. + +#### Designing Snapshot-Friendly State Machines + +When designing your state machine, ensure your state can be efficiently serialized and +deserialized, avoiding complex object graphs that are difficult to serialize. For very large +state machines, consider whether you can implement incremental snapshots that only capture +changes since the last snapshot. + +If your state machine maintains state in external systems, ensure your snapshot process captures +this external state consistently. Regularly test your snapshot and recovery process to ensure it +works correctly under various failure scenarios. + +### Leadership and Fault Tolerance + +Ratis handles the mechanics of leader election and failover automatically. If your application does +not care about whether a specific server is a leader or follower, then it does not need to do +anything when leadership changes. Otherwise, your application can optionally observe leadership +changes and react accordingly: see [State Machine Leadership Events](#State-Machine-Leadership-Events). + +#### Leadership and Automatic Election + +In Raft, the leader is the only server that can accept write requests and decide the order of +operations in the log. This centralized decision-making enables Raft to provide strong +consistency guarantees. Leadership is temporary and can change at any time due to failures, +network partitions, or normal operational events. + +When a Raft group starts up, or when the current leader fails, the remaining servers +automatically elect a new leader through a voting process. This process uses randomized timeouts +to prevent split votes and ensures that only servers with up-to-date logs can become leaders. +This happens entirely within Ratis without any intervention from your application code. + +#### Leadership and Client Behavior + +From a client perspective, leadership changes are largely transparent. Clients can send requests +to any server in the group, and if that server is not the leader, it returns a +`NotLeaderException` with information about the current leader. If the leader fails while +processing a request, the client's retry logic will eventually find the new leader and retry. + +Leadership changes can cause temporary performance degradation as the new leader establishes +itself and catches up any lagging followers. Applications should be designed to handle these +temporary slowdowns gracefully. + +#### State Machine Leadership Events + +Your `StateMachine` can observe and react to leadership changes through several event +notification methods exposed through the `StateMachine.EventApi` interface. The +`notifyLeaderChanged` method is called whenever leadership changes. The `notifyLeaderReady` +method is called when this server becomes leader and is ready to serve requests: the +appropriate place to start any leader-specific background tasks. The `notifyNotLeader` method +is called when this server is nolonger the leader: where you should clean up any leader-specific +resources. + +#### Handling Network Partitions + +When a network partition occurs, the Raft group may split into multiple subgroups that cannot +communicate with each other. Raft's majority-based approach ensures that at most one subgroup (that +contains a majority of servers) can continue processing writes. Any minority subgroup will be +unable to elect a leader and will reject write requests. + +This behavior prevents split-brain scenarios where different parts of the system make conflicting +decisions. However, it also means that your application may become unavailable for writes if no +subgroups have a majority of servers. + +Consider the implications of different partition scenarios when designing your Raft deployment. +If you're deploying across multiple data centers, consider how network partitions between data +centers might affect availability. You may need to choose between consistency and availability +based on your application's requirements. + +--- +Next: [Advanced Topics](advanced.md) \ No newline at end of file From 9d63777e2cdbe5abb787614fcb11c1184ab638de Mon Sep 17 00:00:00 2001 From: slfan1989 <55643692+slfan1989@users.noreply.github.com> Date: Sun, 8 Feb 2026 11:09:22 +0800 Subject: [PATCH 246/397] RATIS-2400. Support timeout and interrupt handling in GrpcClientRpc. (#1342) --- .../ratis/grpc/client/GrpcClientRpc.java | 66 ++++++++++++++----- .../ratis/grpc/TestRaftServerWithGrpc.java | 54 +++++++++++++++ 2 files changed, 104 insertions(+), 16 deletions(-) diff --git a/ratis-grpc/src/main/java/org/apache/ratis/grpc/client/GrpcClientRpc.java b/ratis-grpc/src/main/java/org/apache/ratis/grpc/client/GrpcClientRpc.java index 4010ade27b..65175dc2a1 100644 --- a/ratis-grpc/src/main/java/org/apache/ratis/grpc/client/GrpcClientRpc.java +++ b/ratis-grpc/src/main/java/org/apache/ratis/grpc/client/GrpcClientRpc.java @@ -17,6 +17,7 @@ */ package org.apache.ratis.grpc.client; +import org.apache.ratis.client.RaftClientConfigKeys; import org.apache.ratis.client.impl.ClientProtoUtils; import org.apache.ratis.client.impl.RaftClientRpcWithProxy; import org.apache.ratis.conf.RaftProperties; @@ -24,6 +25,8 @@ import org.apache.ratis.grpc.GrpcUtil; import org.apache.ratis.protocol.*; import org.apache.ratis.protocol.exceptions.AlreadyClosedException; +import org.apache.ratis.protocol.exceptions.TimeoutIOException; +import org.apache.ratis.thirdparty.io.grpc.Status; import org.apache.ratis.thirdparty.io.grpc.StatusRuntimeException; import org.apache.ratis.thirdparty.io.grpc.stub.StreamObserver; import org.apache.ratis.proto.RaftProtos.GroupInfoRequestProto; @@ -39,6 +42,7 @@ import org.apache.ratis.util.IOUtils; import org.apache.ratis.util.JavaUtils; import org.apache.ratis.util.PeerProxyMap; +import org.apache.ratis.util.TimeDuration; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -46,12 +50,16 @@ import java.io.InterruptedIOException; import java.util.concurrent.CompletableFuture; import java.util.concurrent.ExecutionException; +import java.util.concurrent.TimeUnit; +import java.util.concurrent.TimeoutException; public class GrpcClientRpc extends RaftClientRpcWithProxy { public static final Logger LOG = LoggerFactory.getLogger(GrpcClientRpc.class); private final ClientId clientId; private final int maxMessageSize; + private final TimeDuration requestTimeoutDuration; + private final TimeDuration watchRequestTimeoutDuration; public GrpcClientRpc(ClientId clientId, RaftProperties properties, SslContext adminSslContext, SslContext clientSslContext) { @@ -59,6 +67,8 @@ public GrpcClientRpc(ClientId clientId, RaftProperties properties, p -> new GrpcClientProtocolClient(clientId, p, properties, adminSslContext, clientSslContext))); this.clientId = clientId; this.maxMessageSize = GrpcConfigKeys.messageSizeMax(properties, LOG::debug).getSizeInt(); + this.requestTimeoutDuration = RaftClientConfigKeys.Rpc.requestTimeout(properties); + this.watchRequestTimeoutDuration = RaftClientConfigKeys.Rpc.watchRequestTimeout(properties); } @Override @@ -121,24 +131,11 @@ public RaftClientReply sendRequest(RaftClientRequest request) ((LeaderElectionManagementRequest) request); return ClientProtoUtils.toRaftClientReply(proxy.leaderElectionManagement(proto)); } else { - final CompletableFuture f = sendRequest(request, proxy); - // TODO: timeout support - try { - return f.get(); - } catch (InterruptedException e) { - Thread.currentThread().interrupt(); - throw new InterruptedIOException( - "Interrupted while waiting for response of request " + request); - } catch (ExecutionException e) { - if (LOG.isTraceEnabled()) { - LOG.trace(clientId + ": failed " + request, e); - } - throw IOUtils.toIOException(e); - } + return sendRequest(request, proxy); } } - private CompletableFuture sendRequest( + private RaftClientReply sendRequest( RaftClientRequest request, GrpcClientProtocolClient proxy) throws IOException { final RaftClientRequestProto requestProto = toRaftClientRequestProto(request); @@ -167,7 +164,44 @@ public void onCompleted() { requestObserver.onNext(requestProto); requestObserver.onCompleted(); - return replyFuture.thenApply(ClientProtoUtils::toRaftClientReply); + final TimeDuration timeout = getTimeoutDuration(request); + try { + return replyFuture.thenApply(ClientProtoUtils::toRaftClientReply) + .get(timeout.getDuration(), timeout.getUnit()); + } catch (InterruptedException e) { + Thread.currentThread().interrupt(); + replyFuture.cancel(true); + final InterruptedIOException ioe = new InterruptedIOException(clientId + ": Interrupted " + request); + sendOnError(requestObserver, Status.CANCELLED, ioe.getMessage()); + throw ioe; + } catch (TimeoutException e) { + replyFuture.cancel(true); + final TimeoutIOException ioe = + new TimeoutIOException(clientId + ": Timed out " + timeout + " for " + request, e); + sendOnError(requestObserver, Status.DEADLINE_EXCEEDED, ioe.getMessage()); + throw ioe; + } catch (ExecutionException e) { + if (LOG.isTraceEnabled()) { + LOG.trace("{} : failed {}", clientId, request, e); + } + throw IOUtils.toIOException(e); + } + } + + private void sendOnError(StreamObserver requestObserver, Status status, String message) { + try { + requestObserver.onError(status.withDescription(message).asException()); + } catch (Exception ignored) { + // the stream already closed. + } + } + + private TimeDuration getTimeoutDuration(RaftClientRequest request) { + final long timeoutMs = request.getTimeoutMs(); + if (timeoutMs > 0) { + return TimeDuration.valueOf(timeoutMs, TimeUnit.MILLISECONDS); + } + return request.is(RaftClientRequestProto.TypeCase.WATCH) ? watchRequestTimeoutDuration : requestTimeoutDuration; } private RaftClientRequestProto toRaftClientRequestProto(RaftClientRequest request) throws IOException { diff --git a/ratis-test/src/test/java/org/apache/ratis/grpc/TestRaftServerWithGrpc.java b/ratis-test/src/test/java/org/apache/ratis/grpc/TestRaftServerWithGrpc.java index 322eb52287..100fb8d5d2 100644 --- a/ratis-test/src/test/java/org/apache/ratis/grpc/TestRaftServerWithGrpc.java +++ b/ratis-test/src/test/java/org/apache/ratis/grpc/TestRaftServerWithGrpc.java @@ -69,6 +69,7 @@ import javax.net.ssl.KeyManager; import javax.net.ssl.TrustManager; import java.io.IOException; +import java.io.InterruptedIOException; import java.nio.channels.OverlappingFileLockException; import java.util.ArrayList; import java.util.Arrays; @@ -81,6 +82,7 @@ import java.util.concurrent.ThreadLocalRandom; import java.util.concurrent.TimeUnit; import java.util.concurrent.atomic.AtomicLong; +import java.util.concurrent.atomic.AtomicReference; public class TestRaftServerWithGrpc extends BaseTest implements MiniRaftClusterWithGrpc.FactoryGet { { @@ -238,6 +240,58 @@ public void testRaftClientMetrics(Boolean separateHeartbeat) throws Exception { runWithNewCluster(3, this::testRaftClientRequestMetrics); } + @ParameterizedTest + @MethodSource("data") + public void testGrpcClientRpcSyncTimeout(Boolean separateHeartbeat) throws Exception { + GrpcConfigKeys.Server.setHeartbeatChannel(getProperties(), separateHeartbeat); + runWithNewCluster(3, cluster -> { + final RaftPeerId leaderId = RaftTestUtil.waitForLeader(cluster).getId(); + try (RaftClient client = cluster.createClient(leaderId, RetryPolicies.noRetry())) { + final SimpleStateMachine4Testing stateMachine = SimpleStateMachine4Testing.get(cluster.getLeader()); + stateMachine.blockStartTransaction(); + try { + Assertions.assertThrows(TimeoutIOException.class, + () -> client.io().send(new SimpleMessage("sync-timeout"))); + } finally { + stateMachine.unblockStartTransaction(); + } + } + }); + } + + @ParameterizedTest + @MethodSource("data") + public void testGrpcClientRpcSyncCancelOnInterrupt(Boolean separateHeartbeat) throws Exception { + RaftClientConfigKeys.Rpc.setRequestTimeout(getProperties(), TimeDuration.valueOf(10, TimeUnit.SECONDS)); + GrpcConfigKeys.Server.setHeartbeatChannel(getProperties(), separateHeartbeat); + runWithNewCluster(3, cluster -> { + final RaftPeerId leaderId = RaftTestUtil.waitForLeader(cluster).getId(); + try (RaftClient client = cluster.createClient(leaderId, RetryPolicies.noRetry())) { + final SimpleStateMachine4Testing stateMachine = SimpleStateMachine4Testing.get(cluster.getLeader()); + stateMachine.blockStartTransaction(); + try { + final AtomicReference error = new AtomicReference<>(); + final Thread t = new Thread(() -> { + try { + client.io().send(new SimpleMessage("sync-cancel")); + } catch (Throwable e) { + error.set(e); + } + }); + t.start(); + Thread.sleep(200); + t.interrupt(); + t.join(5000); + Assertions.assertFalse(t.isAlive(), "request thread should exit after interrupt"); + Assertions.assertTrue(error.get() instanceof InterruptedIOException, + "expected InterruptedIOException but got " + error.get()); + } finally { + stateMachine.unblockStartTransaction(); + } + } + }); + } + @ParameterizedTest @MethodSource("data") public void testRaftServerMetrics(Boolean separateHeartbeat) throws Exception { From 1e154c825a9add024ffa2335105db2f9d6491ce7 Mon Sep 17 00:00:00 2001 From: slfan1989 <55643692+slfan1989@users.noreply.github.com> Date: Sun, 8 Feb 2026 15:19:09 +0800 Subject: [PATCH 247/397] RATIS-2402. Fix CallId overflow when parsing gRPC metadata. (#1344) --- .../java/org/apache/ratis/grpc/GrpcUtil.java | 2 +- .../ratis/grpc/TestCustomGrpcServices.java | 20 +++++++++++++++++++ 2 files changed, 21 insertions(+), 1 deletion(-) diff --git a/ratis-grpc/src/main/java/org/apache/ratis/grpc/GrpcUtil.java b/ratis-grpc/src/main/java/org/apache/ratis/grpc/GrpcUtil.java index 8dcfb65443..e7bb2b1693 100644 --- a/ratis-grpc/src/main/java/org/apache/ratis/grpc/GrpcUtil.java +++ b/ratis-grpc/src/main/java/org/apache/ratis/grpc/GrpcUtil.java @@ -146,7 +146,7 @@ static long getCallId(Throwable t) { final Metadata trailers = ((StatusRuntimeException)t).getTrailers(); if (trailers != null) { final String callId = trailers.get(CALL_ID); - return callId != null ? Integer.parseInt(callId) : -1; + return callId != null ? Long.parseUnsignedLong(callId) : -1; } } return -1; diff --git a/ratis-test/src/test/java/org/apache/ratis/grpc/TestCustomGrpcServices.java b/ratis-test/src/test/java/org/apache/ratis/grpc/TestCustomGrpcServices.java index 13c4a59fba..7784e4d3dd 100644 --- a/ratis-test/src/test/java/org/apache/ratis/grpc/TestCustomGrpcServices.java +++ b/ratis-test/src/test/java/org/apache/ratis/grpc/TestCustomGrpcServices.java @@ -28,6 +28,7 @@ import org.apache.ratis.test.proto.GreeterGrpc; import org.apache.ratis.test.proto.HelloReply; import org.apache.ratis.test.proto.HelloRequest; +import org.apache.ratis.thirdparty.io.grpc.StatusRuntimeException; import org.apache.ratis.thirdparty.io.grpc.ManagedChannel; import org.apache.ratis.thirdparty.io.grpc.ManagedChannelBuilder; import org.apache.ratis.thirdparty.io.grpc.netty.NettyServerBuilder; @@ -202,4 +203,23 @@ static void sendAndAssertReply(String name, GreeterClient client, GreeterImpl gr final String expected = greeter.toReply(name); Assertions.assertEquals(expected, computed); } + + @Test + public void testGetCallIdWithLargeValue() { + long largeCallId = (long) Integer.MAX_VALUE + 1L; + StatusRuntimeException ex = GrpcUtil.wrapException(new IOException("test"), largeCallId); + Assertions.assertEquals(largeCallId, GrpcUtil.getCallId(ex)); + } + + @Test + public void testGetCallIdWithMissingValue() { + StatusRuntimeException ex = GrpcUtil.wrapException(new IOException("test")); + Assertions.assertEquals(-1L, GrpcUtil.getCallId(ex)); + } + + @Test + public void testGetCallIdWithZeroValue() { + StatusRuntimeException ex = GrpcUtil.wrapException(new IOException("test"), 0L); + Assertions.assertEquals(-1L, GrpcUtil.getCallId(ex)); + } } From 2152d356ef84ef1f4ecd121dfcdb5e30062de068 Mon Sep 17 00:00:00 2001 From: Tsz-Wo Nicholas Sze Date: Sat, 7 Feb 2026 23:19:56 -0800 Subject: [PATCH 248/397] RATIS-2401. Add memory info to VersionInfo. (#1343) --- .../org/apache/ratis/util/VersionInfo.java | 19 ++++++++++++++++++- 1 file changed, 18 insertions(+), 1 deletion(-) diff --git a/ratis-common/src/main/java/org/apache/ratis/util/VersionInfo.java b/ratis-common/src/main/java/org/apache/ratis/util/VersionInfo.java index 98f662f373..07136e9a40 100644 --- a/ratis-common/src/main/java/org/apache/ratis/util/VersionInfo.java +++ b/ratis-common/src/main/java/org/apache/ratis/util/VersionInfo.java @@ -28,6 +28,9 @@ import java.util.Map; import java.util.Objects; import java.util.Properties; +import java.util.SortedMap; +import java.util.TreeMap; +import java.util.function.BiConsumer; import java.util.function.Consumer; /** @@ -58,7 +61,7 @@ static SoftwareInfo parse(String key) { private enum RuntimeInfo { // the ordering is the output ordering - JAVA, USER; + JAVA, USER, MEMORY; static final InfoMap MAP; @@ -66,6 +69,10 @@ private enum RuntimeInfo { final EnumMap map = new EnumMap<>(RuntimeInfo.class); final Properties properties = System.getProperties(); map.put(JAVA, properties.getProperty("java.vm.name") + " " + properties.getProperty("java.runtime.version")); + final Runtime r = Runtime.getRuntime(); + map.put(MEMORY, "max: " + TraditionalBinaryPrefix.long2String(r.maxMemory()) + + ", total: " + TraditionalBinaryPrefix.long2String(r.totalMemory()) + + ", free: " + TraditionalBinaryPrefix.long2String(r.freeMemory())); map.put(USER, properties.getProperty("user.name")); MAP = new InfoMap<>(map); } @@ -143,7 +150,17 @@ public void printStartupMessages(Object name, Consumer log) { } } + static void printSystemProperties(BiConsumer out) { + final SortedMap sortedMap = new TreeMap<>(); + for(Map.Entry e : System.getProperties().entrySet()) { + sortedMap.put(e.getKey().toString(), e.getValue()); + } + sortedMap.forEach(out); + } + public static void main(String[] args) { + printSystemProperties((key, value) -> System.out.printf("%-40s = %s%n", key, value)); + VersionInfo.load(VersionInfo.class).printStartupMessages(":", System.out::println); } } From b182b31c60a78843df8de8211b6da1a35e3a48e9 Mon Sep 17 00:00:00 2001 From: Ivan Andika Date: Mon, 9 Feb 2026 11:35:39 +0800 Subject: [PATCH 249/397] RATIS-2392. Leader should trigger heartbeat immediately after ReadIndex (#1340) --- .../apache/ratis/server/impl/RaftServerImpl.java | 14 ++++++++++++++ 1 file changed, 14 insertions(+) diff --git a/ratis-server/src/main/java/org/apache/ratis/server/impl/RaftServerImpl.java b/ratis-server/src/main/java/org/apache/ratis/server/impl/RaftServerImpl.java index 846b87702f..e5ebfafb37 100644 --- a/ratis-server/src/main/java/org/apache/ratis/server/impl/RaftServerImpl.java +++ b/ratis-server/src/main/java/org/apache/ratis/server/impl/RaftServerImpl.java @@ -83,6 +83,7 @@ import org.apache.ratis.server.impl.RetryCacheImpl.CacheEntry; import org.apache.ratis.server.impl.ServerImplUtils.NavigableIndices; import org.apache.ratis.server.leader.LeaderState.StepDownReason; +import org.apache.ratis.server.leader.LogAppender; import org.apache.ratis.server.metrics.LeaderElectionMetrics; import org.apache.ratis.server.metrics.RaftServerMetricsImpl; import org.apache.ratis.server.protocol.RaftServerAsynchronousProtocol; @@ -1530,6 +1531,19 @@ public CompletableFuture readIndexAsync(ReadIndexRequestPro return getReadIndex(ClientProtoUtils.toRaftClientRequest(request.getClientRequest()), leader) .thenApply(index -> toReadIndexReplyProto(peerId, getMemberId(), true, index)) + .whenComplete((reply, exception) -> { + if (exception == null) { + // Leader should try to trigger heartbeat immediately after leader replies the ReadIndex to the follower + // so that the follower's commitIndex can be updated to the leader's commitIndex and the follower + // can start applying the logs up until the leader's commitIndex (instead of waiting for the next + // AppendEntries to happen through heartbeat or new transactions (which might increase the latency + // considerably)). + // Note that if the follower commitIndex is already equal to the leader's commitIndex, no heartbeat + // will be triggered, see GrpcLogAppender#isFollowerCommitBehindLastCommitIndex. + RaftPeerId requestorId = RaftPeerId.valueOf(reply.getServerReply().getRequestorId()); + leader.getLogAppender(requestorId).ifPresent(LogAppender::triggerHeartbeat); + } + }) .exceptionally(throwable -> toReadIndexReplyProto(peerId, getMemberId())); } From d56ae223bdbd34c8a6ebb5eab4ba2cee9df7251c Mon Sep 17 00:00:00 2001 From: Ivan Andika Date: Mon, 9 Feb 2026 16:04:45 +0800 Subject: [PATCH 250/397] RATIS-2382. Support skip leadership check during ReadIndex (#1334) --- .../src/site/markdown/configurations.md | 21 +++++++++++++++++++ .../ratis/server/RaftServerConfigKeys.java | 11 ++++++++++ .../ratis/server/impl/LeaderStateImpl.java | 6 +++++- 3 files changed, 37 insertions(+), 1 deletion(-) diff --git a/ratis-docs/src/site/markdown/configurations.md b/ratis-docs/src/site/markdown/configurations.md index acd1cb9f9e..0f20540a20 100644 --- a/ratis-docs/src/site/markdown/configurations.md +++ b/ratis-docs/src/site/markdown/configurations.md @@ -226,6 +226,27 @@ if it fails to receive any RPC responses from this peer within this specified ti | **Type** | boolean | | **Default** | false | +| **Property** | `raft.server.read.leader.heartbeat-check.enabled` | +|:----------------|:--------------------------------------------------| +| **Description** | whether to check heartbeat for read index. | +| **Type** | boolean | +| **Default** | true | + +Note that the original read index algorithm requires heartbeat check +in order to guarantee linearizable read. +By setting this property to false, +it reduces the RTT by eliminating the heartbeat check. +However, it might cause the reads not to be linearizable in a split-brain case. +Without the heartbeat check, a leader might not be the latest leader +and, as a result, it might serve stale reads. +When there is a split brain, there might be a small period of time +that the (old) leader has lost majority heartbeats but have not yet detected it. +As the same time, a new leader is elected by a majority of peers. +Then, the old leader might serve stale data +since it does not have the transactions committed by the new leaders. +Since such split-brain case is supposed to be rare, +it might be an acceptable tradeoff for applications that +seek to improve the linearizable read performance. ### Write - Configurations related to write requests. diff --git a/ratis-server-api/src/main/java/org/apache/ratis/server/RaftServerConfigKeys.java b/ratis-server-api/src/main/java/org/apache/ratis/server/RaftServerConfigKeys.java index efb3c67963..ef16f67f67 100644 --- a/ratis-server-api/src/main/java/org/apache/ratis/server/RaftServerConfigKeys.java +++ b/ratis-server-api/src/main/java/org/apache/ratis/server/RaftServerConfigKeys.java @@ -249,6 +249,17 @@ static void setLeaderLeaseTimeoutRatio(RaftProperties properties, double ratio) setDouble(properties::setDouble, LEADER_LEASE_TIMEOUT_RATIO_KEY, ratio); } + String LEADER_HEARTBEAT_CHECK_ENABLED_KEY = PREFIX + ".leader.heartbeat-check.enabled"; + boolean LEADER_HEARTBEAT_CHECK_ENABLED_DEFAULT = true; + static boolean leaderHeartbeatCheckEnabled(RaftProperties properties) { + return getBoolean(properties::getBoolean, LEADER_HEARTBEAT_CHECK_ENABLED_KEY, + LEADER_HEARTBEAT_CHECK_ENABLED_DEFAULT, getDefaultLog()); + } + + static void setLeaderHeartbeatCheckEnabled(RaftProperties properties, boolean enabled) { + setBoolean(properties::setBoolean, LEADER_HEARTBEAT_CHECK_ENABLED_KEY, enabled); + } + interface ReadAfterWriteConsistent { String PREFIX = Read.PREFIX + ".read-after-write-consistent"; diff --git a/ratis-server/src/main/java/org/apache/ratis/server/impl/LeaderStateImpl.java b/ratis-server/src/main/java/org/apache/ratis/server/impl/LeaderStateImpl.java index 90d0b76df5..ef0bb6b700 100644 --- a/ratis-server/src/main/java/org/apache/ratis/server/impl/LeaderStateImpl.java +++ b/ratis-server/src/main/java/org/apache/ratis/server/impl/LeaderStateImpl.java @@ -354,6 +354,7 @@ boolean isApplied() { private final ReadIndexHeartbeats readIndexHeartbeats; private final boolean readIndexAppliedIndexEnabled; + private final boolean leaderHeartbeatCheckEnabled; private final LeaderLease lease; LeaderStateImpl(RaftServerImpl server) { @@ -392,6 +393,8 @@ boolean isApplied() { } this.readIndexAppliedIndexEnabled = RaftServerConfigKeys.Read.ReadIndex .appliedIndexEnabled(properties); + this.leaderHeartbeatCheckEnabled = RaftServerConfigKeys.Read + .leaderHeartbeatCheckEnabled(properties); final RaftConfigurationImpl conf = state.getRaftConf(); Collection others = conf.getOtherPeers(server.getId()); @@ -1166,7 +1169,8 @@ CompletableFuture getReadIndex(Long readAfterWriteConsistentIndex) { } // if lease is enabled, check lease first - if (hasLease()) { + // if we allow leader to skip the leadership check heartbeat, we can return immediately + if (!leaderHeartbeatCheckEnabled || hasLease()) { return CompletableFuture.completedFuture(readIndex); } From 39cb97535fb33c08528b5c741cd0c95f89ea36f6 Mon Sep 17 00:00:00 2001 From: slfan1989 <55643692+slfan1989@users.noreply.github.com> Date: Wed, 11 Feb 2026 08:10:35 +0800 Subject: [PATCH 251/397] RATIS-2404. Validate message size before sending async requests. (#1345) --- .../grpc/client/GrpcClientProtocolClient.java | 18 +++++++--- .../ratis/grpc/TestRaftServerWithGrpc.java | 35 +++++++++++++++++++ 2 files changed, 48 insertions(+), 5 deletions(-) diff --git a/ratis-grpc/src/main/java/org/apache/ratis/grpc/client/GrpcClientProtocolClient.java b/ratis-grpc/src/main/java/org/apache/ratis/grpc/client/GrpcClientProtocolClient.java index 159919fabc..0eaec6b962 100644 --- a/ratis-grpc/src/main/java/org/apache/ratis/grpc/client/GrpcClientProtocolClient.java +++ b/ratis-grpc/src/main/java/org/apache/ratis/grpc/client/GrpcClientProtocolClient.java @@ -82,6 +82,7 @@ public class GrpcClientProtocolClient implements Closeable { private final ManagedChannel clientChannel; private final ManagedChannel adminChannel; + private final SizeInBytes maxMessageSize; private final TimeDuration requestTimeoutDuration; private final TimeDuration watchRequestTimeoutDuration; private final TimeoutExecutor scheduler = TimeoutExecutor.getInstance(); @@ -99,7 +100,7 @@ public class GrpcClientProtocolClient implements Closeable { this.name = JavaUtils.memoize(() -> id + "->" + target.getId()); this.target = target; final SizeInBytes flowControlWindow = GrpcConfigKeys.flowControlWindow(properties, LOG::debug); - final SizeInBytes maxMessageSize = GrpcConfigKeys.messageSizeMax(properties, LOG::debug); + this.maxMessageSize = GrpcConfigKeys.messageSizeMax(properties, LOG::debug); metricClientInterceptor = new MetricClientInterceptor(getName()); final String clientAddress = Optional.ofNullable(target.getClientAddress()) @@ -108,9 +109,9 @@ public class GrpcClientProtocolClient implements Closeable { .filter(x -> !x.isEmpty()).orElse(target.getAddress()); final boolean separateAdminChannel = !Objects.equals(clientAddress, adminAddress); - clientChannel = buildChannel(clientAddress, clientSslContext, flowControlWindow, maxMessageSize); + clientChannel = buildChannel(clientAddress, clientSslContext, flowControlWindow); adminChannel = separateAdminChannel - ? buildChannel(adminAddress, adminSslContext, flowControlWindow, maxMessageSize) + ? buildChannel(adminAddress, adminSslContext, flowControlWindow) : clientChannel; asyncStub = RaftClientProtocolServiceGrpc.newStub(clientChannel); @@ -121,7 +122,7 @@ public class GrpcClientProtocolClient implements Closeable { } private ManagedChannel buildChannel(String address, SslContext sslContext, - SizeInBytes flowControlWindow, SizeInBytes maxMessageSize) { + SizeInBytes flowControlWindow) { NettyChannelBuilder channelBuilder = NettyChannelBuilder.forTarget(address); // ignore any http proxy for grpc @@ -332,13 +333,20 @@ public void onCompleted() { } CompletableFuture onNext(RaftClientRequest request) { + final RaftClientRequestProto proto = ClientProtoUtils.toRaftClientRequestProto(request); + if (proto.getSerializedSize() > maxMessageSize.getSizeInt()) { + return JavaUtils.completeExceptionally(new IllegalArgumentException(getName() + + ": request serialized size " + proto.getSerializedSize() + + " exceeds maximum " + maxMessageSize + " for " + request)); + } + final long callId = request.getCallId(); final CompletableFuture f = replies.putNew(callId); if (f == null) { return JavaUtils.completeExceptionally(new AlreadyClosedException(getName() + " is closed.")); } try { - if (!requestStreamer.onNext(ClientProtoUtils.toRaftClientRequestProto(request))) { + if (!requestStreamer.onNext(proto)) { return JavaUtils.completeExceptionally(new AlreadyClosedException(getName() + ": the stream is closed.")); } } catch(Exception t) { diff --git a/ratis-test/src/test/java/org/apache/ratis/grpc/TestRaftServerWithGrpc.java b/ratis-test/src/test/java/org/apache/ratis/grpc/TestRaftServerWithGrpc.java index 100fb8d5d2..b5247cf63d 100644 --- a/ratis-test/src/test/java/org/apache/ratis/grpc/TestRaftServerWithGrpc.java +++ b/ratis-test/src/test/java/org/apache/ratis/grpc/TestRaftServerWithGrpc.java @@ -63,6 +63,8 @@ import org.apache.ratis.util.TimeDuration; import org.junit.jupiter.api.Assertions; import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.Test; +import org.junit.jupiter.api.Timeout; import org.junit.jupiter.params.ParameterizedTest; import org.junit.jupiter.params.provider.MethodSource; import org.slf4j.event.Level; @@ -100,6 +102,39 @@ public void setup() { RaftClientConfigKeys.Rpc.setRequestTimeout(p, TimeDuration.valueOf(1, TimeUnit.SECONDS)); } + @Test + @Timeout(value = 30, unit = TimeUnit.SECONDS) + public void testAsyncRequestExceedsMaxMessageSize() throws Exception { + final RaftProperties properties = getProperties(); + final SizeInBytes originalMessageSize = GrpcConfigKeys.messageSizeMax(properties, s -> {}); + final SizeInBytes originalBufferLimit = RaftServerConfigKeys.Log.Appender.bufferByteLimit(properties); + final boolean originalSendDummyRequest = + RaftClientConfigKeys.Async.Experimental.sendDummyRequest(properties); + + RaftServerConfigKeys.Log.Appender.setBufferByteLimit(properties, SizeInBytes.valueOf("16KB")); + final SizeInBytes testMessageSizeMax = SizeInBytes.valueOf("1040KB"); + GrpcConfigKeys.setMessageSizeMax(properties, testMessageSizeMax); + RaftClientConfigKeys.Async.Experimental.setSendDummyRequest(properties, false); + + try { + runWithNewCluster(1, cluster -> { + try (RaftClient client = cluster.createClient(RetryPolicies.noRetry())) { + final int oversizedKb = 1200; + final byte[] bytes = new byte[oversizedKb * 1024]; // > 1040KB + final SimpleMessage message = new SimpleMessage("oversized", ByteString.copyFrom(bytes)); + + testFailureCaseAsync("async oversized request", + () -> client.async().send(message), + IllegalArgumentException.class); + } + }); + } finally { + GrpcConfigKeys.setMessageSizeMax(properties, originalMessageSize); + RaftServerConfigKeys.Log.Appender.setBufferByteLimit(properties, originalBufferLimit); + RaftClientConfigKeys.Async.Experimental.setSendDummyRequest(properties, originalSendDummyRequest); + } + } + @ParameterizedTest @MethodSource("data") public void testServerRestartOnException(Boolean separateHeartbeat) throws Exception { From 7382059e8717cfcb10135a97d63e7197709a9bf0 Mon Sep 17 00:00:00 2001 From: slfan1989 <55643692+slfan1989@users.noreply.github.com> Date: Thu, 12 Feb 2026 06:02:56 +0800 Subject: [PATCH 252/397] RATIS-2405. Remove duplicate computeIfAbsent call in MessageMetrics.inc() method. (#1346) --- .../org/apache/ratis/grpc/metrics/MessageMetrics.java | 8 -------- .../ratis/grpc/server/TestGrpcMessageMetrics.java | 10 +++++++--- 2 files changed, 7 insertions(+), 11 deletions(-) diff --git a/ratis-grpc/src/main/java/org/apache/ratis/grpc/metrics/MessageMetrics.java b/ratis-grpc/src/main/java/org/apache/ratis/grpc/metrics/MessageMetrics.java index 2a211aae80..b152c67098 100644 --- a/ratis-grpc/src/main/java/org/apache/ratis/grpc/metrics/MessageMetrics.java +++ b/ratis-grpc/src/main/java/org/apache/ratis/grpc/metrics/MessageMetrics.java @@ -61,14 +61,6 @@ private void inc(String metricNamePrefix, Type t) { types.get(t) .computeIfAbsent(metricNamePrefix, prefix -> getRegistry().counter(prefix + t.getSuffix())) .inc(); - final Map counters = types.get(t); - LongCounter c = counters.get(metricNamePrefix); - if (c == null) { - synchronized (counters) { - c = counters.computeIfAbsent(metricNamePrefix, prefix -> getRegistry().counter(prefix + t.getSuffix())); - } - } - c.inc(); } /** diff --git a/ratis-test/src/test/java/org/apache/ratis/grpc/server/TestGrpcMessageMetrics.java b/ratis-test/src/test/java/org/apache/ratis/grpc/server/TestGrpcMessageMetrics.java index 8094069cf5..a8cd6138ec 100644 --- a/ratis-test/src/test/java/org/apache/ratis/grpc/server/TestGrpcMessageMetrics.java +++ b/ratis-test/src/test/java/org/apache/ratis/grpc/server/TestGrpcMessageMetrics.java @@ -19,6 +19,7 @@ import org.apache.ratis.BaseTest; import org.apache.ratis.grpc.MiniRaftClusterWithGrpc; +import org.apache.ratis.grpc.metrics.MessageMetrics; import org.apache.ratis.server.impl.MiniRaftCluster; import org.apache.ratis.RaftTestUtil; import org.apache.ratis.client.RaftClient; @@ -69,7 +70,10 @@ static void assertMessageCount(RaftServer.Division server) { final GrpcServicesImpl services = (GrpcServicesImpl) RaftServerTestUtil.getServerRpc(server); final RatisMetricRegistry registry = services.getMessageMetrics().getRegistry(); String counter_prefix = serverId + "_" + "ratis.grpc.RaftServerProtocolService"; - Assertions.assertTrue( - registry.counter(counter_prefix + "_" + "requestVote" + "_OK_completed_total").getCount() > 0); + final String metricPrefix = counter_prefix + "_" + "requestVote" + "_OK"; + final long before = registry.counter(metricPrefix + "_completed_total").getCount(); + services.getMessageMetrics().rpcCompleted(metricPrefix); + final long after = registry.counter(metricPrefix + "_completed_total").getCount(); + Assertions.assertEquals(before + 1, after); } -} \ No newline at end of file +} From 375dd0ccddf707be7343da43fd294c7e9f7ae369 Mon Sep 17 00:00:00 2001 From: slfan1989 <55643692+slfan1989@users.noreply.github.com> Date: Sat, 14 Feb 2026 11:14:26 +0800 Subject: [PATCH 253/397] RATIS-2406. Fix resource cleanup bug in GrpcServicesImpl.closeImpl() method. (#1347) --- .../ratis/grpc/server/GrpcServicesImpl.java | 37 ++++++++++++++----- 1 file changed, 27 insertions(+), 10 deletions(-) diff --git a/ratis-grpc/src/main/java/org/apache/ratis/grpc/server/GrpcServicesImpl.java b/ratis-grpc/src/main/java/org/apache/ratis/grpc/server/GrpcServicesImpl.java index b1af0960dc..d554ca583a 100644 --- a/ratis-grpc/src/main/java/org/apache/ratis/grpc/server/GrpcServicesImpl.java +++ b/ratis-grpc/src/main/java/org/apache/ratis/grpc/server/GrpcServicesImpl.java @@ -341,23 +341,40 @@ public void startImpl() { } @Override - public void closeImpl() throws IOException { + public void closeImpl() { + for (Server server : servers.values()) { + server.shutdownNow(); + } + boolean interrupted = false; for (Map.Entry server : servers.entrySet()) { - final String name = getId() + ": shutdown server " + server.getKey(); - LOG.info("{} now", name); - final Server s = server.getValue().shutdownNow(); - super.closeImpl(); try { - s.awaitTermination(); + server.getValue().awaitTermination(); + LOG.info("{}: Shutdown {} successfully", getId(), server.getKey()); } catch (InterruptedException e) { Thread.currentThread().interrupt(); - throw IOUtils.toInterruptedIOException(name + " failed", e); + LOG.warn("{}: Interrupted shutdown {}", getId(), server.getKey()); + interrupted = true; + break; } - LOG.info("{} successfully", name); } - serverInterceptor.close(); - ConcurrentUtils.shutdownAndWait(executor); + try { + serverInterceptor.close(); + } catch (Exception e) { + LOG.warn("{}: Failed to unregister metrics", getId(), e); + } + + if (interrupted) { + executor.shutdown(); // shutdown but not wait + } else { + ConcurrentUtils.shutdownAndWait(executor); + } + + try { + super.closeImpl(); + } catch (IOException e) { + LOG.warn("{}: Failed to close proxies", getId(), e); + } } @Override From 1d41cc0bb9c9a81267700867994313a8c94ad813 Mon Sep 17 00:00:00 2001 From: slfan1989 <55643692+slfan1989@users.noreply.github.com> Date: Sat, 14 Feb 2026 11:39:37 +0800 Subject: [PATCH 254/397] RATIS-2407. Missing return statement after stream closed check in OrderedRequestStreamObserver. (#1348) --- .../org/apache/ratis/grpc/server/GrpcClientProtocolService.java | 1 + 1 file changed, 1 insertion(+) diff --git a/ratis-grpc/src/main/java/org/apache/ratis/grpc/server/GrpcClientProtocolService.java b/ratis-grpc/src/main/java/org/apache/ratis/grpc/server/GrpcClientProtocolService.java index 9c19684677..1da3587e91 100644 --- a/ratis-grpc/src/main/java/org/apache/ratis/grpc/server/GrpcClientProtocolService.java +++ b/ratis-grpc/src/main/java/org/apache/ratis/grpc/server/GrpcClientProtocolService.java @@ -338,6 +338,7 @@ void processClientRequest(RaftClientRequest r) { if (isClosed()) { final AlreadyClosedException exception = new AlreadyClosedException(getName() + ": the stream is closed"); responseError(exception, () -> "processClientRequest (stream already closed) for " + r); + return; } final RaftGroupId requestGroupId = r.getRaftGroupId(); From 31b71cff29147ea319a82678666435af2d86285c Mon Sep 17 00:00:00 2001 From: slfan1989 <55643692+slfan1989@users.noreply.github.com> Date: Mon, 16 Feb 2026 18:24:35 +0800 Subject: [PATCH 255/397] RATIS-2409. Fix typos in ratis-docs configurations.md. (#1350) --- ratis-docs/src/site/markdown/configurations.md | 10 +++++----- 1 file changed, 5 insertions(+), 5 deletions(-) diff --git a/ratis-docs/src/site/markdown/configurations.md b/ratis-docs/src/site/markdown/configurations.md index 0f20540a20..52eef048fa 100644 --- a/ratis-docs/src/site/markdown/configurations.md +++ b/ratis-docs/src/site/markdown/configurations.md @@ -121,7 +121,7 @@ if it fails to receive any RPC responses from this peer within this specified ti | **Property** | `raft.server.threadpool.proxy.cached` | |:----------------|:--------------------------------------------------------| -| **Description** | use CachedThreadPool, otherwise, uee newFixedThreadPool | +| **Description** | use CachedThreadPool, otherwise, use newFixedThreadPool | | **Type** | boolean | | **Default** | true | @@ -138,7 +138,7 @@ if it fails to receive any RPC responses from this peer within this specified ti | **Property** | `raft.server.threadpool.server.cached` | |:----------------|:--------------------------------------------------------| -| **Description** | use CachedThreadPool, otherwise, uee newFixedThreadPool | +| **Description** | use CachedThreadPool, otherwise, use newFixedThreadPool | | **Type** | boolean | | **Default** | true | @@ -155,7 +155,7 @@ if it fails to receive any RPC responses from this peer within this specified ti | **Property** | `raft.server.threadpool.client.cached` | |:----------------|:--------------------------------------------------------| -| **Description** | use CachedThreadPool, otherwise, uee newFixedThreadPool | +| **Description** | use CachedThreadPool, otherwise, use newFixedThreadPool | | **Type** | boolean | | **Default** | true | @@ -562,7 +562,7 @@ further wait for 5sec for max times ((5sec*980)/2 times ~= 40min) | **Property** | `raft.server.data-stream.async.request.thread.pool.cached` | |:----------------|:-----------------------------------------------------------| -| **Description** | use CachedThreadPool, otherwise, uee newFixedThreadPool | +| **Description** | use CachedThreadPool, otherwise, use newFixedThreadPool | | **Type** | boolean | | **Default** | false | @@ -575,7 +575,7 @@ further wait for 5sec for max times ((5sec*980)/2 times ~= 40min) | **Property** | `raft.server.data-stream.async.write.thread.pool.cached` | |:----------------|:---------------------------------------------------------| -| **Description** | use CachedThreadPool, otherwise, uee newFixedThreadPool | +| **Description** | use CachedThreadPool, otherwise, use newFixedThreadPool | | **Type** | boolean | | **Default** | false | From f9c0b3d8290ac8b6122b03bef6a265bdd8e5f036 Mon Sep 17 00:00:00 2001 From: slfan1989 <55643692+slfan1989@users.noreply.github.com> Date: Tue, 17 Feb 2026 02:52:21 +0800 Subject: [PATCH 256/397] RATIS-2410. GrpcStubPool should use GrpcUtil for channel shutdown. (#1351) --- .../main/java/org/apache/ratis/grpc/server/GrpcStubPool.java | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/ratis-grpc/src/main/java/org/apache/ratis/grpc/server/GrpcStubPool.java b/ratis-grpc/src/main/java/org/apache/ratis/grpc/server/GrpcStubPool.java index fd27ac996a..9667661d07 100644 --- a/ratis-grpc/src/main/java/org/apache/ratis/grpc/server/GrpcStubPool.java +++ b/ratis-grpc/src/main/java/org/apache/ratis/grpc/server/GrpcStubPool.java @@ -17,6 +17,7 @@ */ package org.apache.ratis.grpc.server; +import org.apache.ratis.grpc.GrpcUtil; import org.apache.ratis.thirdparty.io.grpc.ManagedChannel; import org.apache.ratis.thirdparty.io.grpc.netty.NegotiationType; import org.apache.ratis.thirdparty.io.grpc.netty.NettyChannelBuilder; @@ -77,7 +78,7 @@ void release() { } void shutdown() { - ch.shutdown(); + GrpcUtil.shutdownManagedChannel(ch); } } From 2f52cd4280d4559f9cc4f8849db252fe7e844660 Mon Sep 17 00:00:00 2001 From: "Doroszlai, Attila" <6454655+adoroszlai@users.noreply.github.com> Date: Tue, 17 Feb 2026 04:21:43 +0100 Subject: [PATCH 257/397] RATIS-2411. Bump ratis-thirdparty to 1.0.11 (#1352) --- pom.xml | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/pom.xml b/pom.xml index e3b7fcb9f0..96f07ab012 100644 --- a/pom.xml +++ b/pom.xml @@ -183,11 +183,11 @@ ${maven.compiler.release} - 1.0.10 + 1.0.11 3.25.8 - 1.75.0 + 1.77.1 true From dae55df2bc37f559cad342b318fd1e34003f4842 Mon Sep 17 00:00:00 2001 From: slfan1989 <55643692+slfan1989@users.noreply.github.com> Date: Thu, 19 Feb 2026 01:36:48 +0800 Subject: [PATCH 258/397] RATIS-2408. Add configurable exponential backoff reconnection for Netty DataStream client. (#1349) --- .../ratis/retry/ExponentialBackoffRetry.java | 14 ++- .../org/apache/ratis/retry/RetryPolicy.java | 26 ++++++ .../apache/ratis/netty/NettyConfigKeys.java | 12 +++ .../netty/client/NettyClientStreamRpc.java | 72 ++++++++++++-- ...tNettyClientStreamRpcReconnectBackoff.java | 79 ++++++++++++++++ ...ttyDataStreamReconnectWithGrpcCluster.java | 93 +++++++++++++++++++ 6 files changed, 286 insertions(+), 10 deletions(-) create mode 100644 ratis-test/src/test/java/org/apache/ratis/netty/client/TestNettyClientStreamRpcReconnectBackoff.java create mode 100644 ratis-test/src/test/java/org/apache/ratis/netty/client/TestNettyDataStreamReconnectWithGrpcCluster.java diff --git a/ratis-common/src/main/java/org/apache/ratis/retry/ExponentialBackoffRetry.java b/ratis-common/src/main/java/org/apache/ratis/retry/ExponentialBackoffRetry.java index d506c85c80..3c9ffbf457 100644 --- a/ratis-common/src/main/java/org/apache/ratis/retry/ExponentialBackoffRetry.java +++ b/ratis-common/src/main/java/org/apache/ratis/retry/ExponentialBackoffRetry.java @@ -17,6 +17,7 @@ */ package org.apache.ratis.retry; +import org.apache.ratis.util.Preconditions; import org.apache.ratis.util.TimeDuration; import java.util.Objects; @@ -31,7 +32,6 @@ * in the range [s*0.5, s*1.5). */ public final class ExponentialBackoffRetry implements RetryPolicy { - public static final class Builder { private Builder() {} @@ -56,9 +56,7 @@ public Builder setMaxSleepTime(TimeDuration maxSleepTime) { } public ExponentialBackoffRetry build() { - Objects.requireNonNull(baseSleepTime, "baseSleepTime == null"); - return new ExponentialBackoffRetry(baseSleepTime, maxSleepTime, - maxAttempts); + return new ExponentialBackoffRetry(baseSleepTime, maxSleepTime, maxAttempts); } } @@ -67,6 +65,14 @@ public ExponentialBackoffRetry build() { private final int maxAttempts; private ExponentialBackoffRetry(TimeDuration baseSleepTime, TimeDuration maxSleepTime, int maxAttempts) { + Objects.requireNonNull(baseSleepTime, "baseSleepTime == null"); + Preconditions.assertTrue(baseSleepTime.isPositive(), () -> "baseSleepTime = " + baseSleepTime + " <= 0"); + if (maxSleepTime != null) { + Preconditions.assertTrue(maxSleepTime.compareTo(baseSleepTime) >= 0, + () -> "maxSleepTime = " + maxSleepTime + " < baseSleepTime = " + baseSleepTime); + } + Preconditions.assertTrue(maxAttempts >= 0, () -> "maxAttempts = " + maxAttempts + " < 0"); + this.baseSleepTime = baseSleepTime; this.maxSleepTime = maxSleepTime; this.maxAttempts = maxAttempts; diff --git a/ratis-common/src/main/java/org/apache/ratis/retry/RetryPolicy.java b/ratis-common/src/main/java/org/apache/ratis/retry/RetryPolicy.java index 1de07f19e1..6916858e57 100644 --- a/ratis-common/src/main/java/org/apache/ratis/retry/RetryPolicy.java +++ b/ratis-common/src/main/java/org/apache/ratis/retry/RetryPolicy.java @@ -19,6 +19,9 @@ import org.apache.ratis.util.TimeDuration; +import java.util.Objects; +import java.util.concurrent.TimeUnit; + /** * Policy abstract for retrying. */ @@ -72,4 +75,27 @@ default Throwable getCause() { * @return the action it should take. */ Action handleAttemptFailure(Event event); + + static RetryPolicy parse(String commaSeparated) { + Objects.requireNonNull(commaSeparated, "commaSeparated == null"); + final String[] args = commaSeparated.split(","); + if (args.length < 1) { + throw new IllegalArgumentException("Failed to parse RetryPolicy: args.length = " + + args.length + " < 1 for " + commaSeparated); + } + final String classname = args[0].trim(); + if (classname.equals(ExponentialBackoffRetry.class.getSimpleName())) { + if (args.length != 4) { + throw new IllegalArgumentException("Failed to parse ExponentialBackoffRetry: args.length = " + + args.length + " != 4 for " + commaSeparated); + } + return ExponentialBackoffRetry.newBuilder() + .setBaseSleepTime(TimeDuration.valueOf(args[1], TimeUnit.MILLISECONDS)) + .setMaxSleepTime(TimeDuration.valueOf(args[2], TimeUnit.MILLISECONDS)) + .setMaxAttempts(Integer.parseInt(args[3].trim())) + .build(); + } + throw new IllegalArgumentException("Failed to parse RetryPolicy: unknown class " + + args[0] + " for " + commaSeparated); + } } diff --git a/ratis-netty/src/main/java/org/apache/ratis/netty/NettyConfigKeys.java b/ratis-netty/src/main/java/org/apache/ratis/netty/NettyConfigKeys.java index be3ad8ee67..e84cb4eb24 100644 --- a/ratis-netty/src/main/java/org/apache/ratis/netty/NettyConfigKeys.java +++ b/ratis-netty/src/main/java/org/apache/ratis/netty/NettyConfigKeys.java @@ -176,6 +176,18 @@ static TimeDuration replyQueueGracePeriod(RaftProperties properties) { static void setReplyQueueGracePeriod(RaftProperties properties, TimeDuration timeoutDuration) { setTimeDuration(properties::setTimeDuration, REPLY_QUEUE_GRACE_PERIOD_KEY, timeoutDuration); } + + /** A retry policy specified in comma separated format. */ + String RECONNECT_POLICY_KEY = PREFIX + ".reconnect.policy"; + /** ExponentialBackoffRetry with base sleep 100ms, max sleep 5s and max attempt 100. */ + String RECONNECT_POLICY_DEFAULT = "ExponentialBackoffRetry,100ms,5s,100"; + static String reconnectPolicy(RaftProperties properties) { + return properties.get(RECONNECT_POLICY_KEY, RECONNECT_POLICY_DEFAULT); + } + static void setReconnectPolicy(RaftProperties properties, String retryPolicy) { + properties.set(RECONNECT_POLICY_KEY, retryPolicy); + } + } interface Server { diff --git a/ratis-netty/src/main/java/org/apache/ratis/netty/client/NettyClientStreamRpc.java b/ratis-netty/src/main/java/org/apache/ratis/netty/client/NettyClientStreamRpc.java index 5e111daff8..44e91d283e 100644 --- a/ratis-netty/src/main/java/org/apache/ratis/netty/client/NettyClientStreamRpc.java +++ b/ratis-netty/src/main/java/org/apache/ratis/netty/client/NettyClientStreamRpc.java @@ -35,7 +35,9 @@ import org.apache.ratis.protocol.RaftPeer; import org.apache.ratis.protocol.exceptions.AlreadyClosedException; import org.apache.ratis.protocol.exceptions.TimeoutIOException; +import org.apache.ratis.retry.RetryPolicy; import org.apache.ratis.security.TlsConf; +import org.apache.ratis.thirdparty.com.google.common.annotations.VisibleForTesting; import org.apache.ratis.thirdparty.io.netty.bootstrap.Bootstrap; import org.apache.ratis.thirdparty.io.netty.buffer.ByteBuf; import org.apache.ratis.thirdparty.io.netty.channel.Channel; @@ -71,6 +73,8 @@ import java.util.Optional; import java.util.concurrent.CompletableFuture; import java.util.concurrent.TimeUnit; +import java.util.concurrent.atomic.AtomicBoolean; +import java.util.concurrent.atomic.AtomicInteger; import java.util.concurrent.atomic.AtomicReference; import java.util.function.Function; import java.util.function.Supplier; @@ -136,20 +140,24 @@ void shutdownGracefully() { } static class Connection { - static final TimeDuration RECONNECT = TimeDuration.valueOf(100, TimeUnit.MILLISECONDS); + static final TimeDuration FIVE_HUNDRED_MS = TimeDuration.valueOf(500, TimeUnit.MILLISECONDS); private final InetSocketAddress address; private final WorkerGroupGetter workerGroup; private final Supplier> channelInitializerSupplier; + private final RetryPolicy reconnectPolicy; /** The {@link ChannelFuture} is null when this connection is closed. */ private final AtomicReference> ref; + private final AtomicBoolean reconnectScheduled = new AtomicBoolean(false); + private final AtomicInteger reconnectAttempts = new AtomicInteger(); Connection(InetSocketAddress address, WorkerGroupGetter workerGroup, - Supplier> channelInitializerSupplier) { + Supplier> channelInitializerSupplier, RetryPolicy reconnectPolicy) { this.address = address; this.workerGroup = workerGroup; this.channelInitializerSupplier = channelInitializerSupplier; + this.reconnectPolicy = reconnectPolicy; this.ref = new AtomicReference<>(MemoizedSupplier.valueOf(this::connect)); } @@ -191,21 +199,47 @@ public void operationComplete(ChannelFuture future) { if (!future.isSuccess()) { scheduleReconnect(Connection.this + " failed", future.cause()); } else { + reconnectAttempts.set(0); LOG.trace("{} succeed.", Connection.this); } } }); } + /** + * Schedules a reconnection attempt with exponential backoff and jitter. + * + * @param message description of the failure + * @param cause the exception that triggered reconnection (may be null) + */ void scheduleReconnect(String message, Throwable cause) { if (isClosed()) { return; } - LOG.warn("{}: {}; schedule reconnecting to {} in {}", this, message, address, RECONNECT); + if (!reconnectScheduled.compareAndSet(false, true)) { + return; + } + // Use retry index starting at 0 so the first delay equals base sleep time. + final int attempt = reconnectAttempts.getAndIncrement(); + final RetryPolicy.Action action = reconnectPolicy.handleAttemptFailure(() -> attempt); + if (!action.shouldRetry()) { + reconnectScheduled.set(false); + LOG.warn("{}: {}; no more retries to {} after attempt {}", this, message, address, attempt); + return; + } + final TimeDuration delay = action.getSleepTime(); if (cause != null) { - LOG.warn("", cause); + LOG.warn("{}: {}; reconnect to {} in {} for attempt {}", + this, message, address, delay, attempt, cause); + } else if (delay.compareTo(FIVE_HUNDRED_MS) < 0) { + LOG.info("{}: {}; reconnect to {} in {} for attempt {}", this, message, address, delay, attempt); + } else { + LOG.warn("{}: {}; reconnect to {} in {} for attempt {}", this, message, address, delay, attempt); } - getWorkerGroup().schedule(this::reconnect, RECONNECT.getDuration(), RECONNECT.getUnit()); + getWorkerGroup().schedule(() -> { + reconnectScheduled.set(false); + reconnect(); + }, delay.getDuration(), delay.getUnit()); } private synchronized ChannelFuture reconnect() { @@ -313,8 +347,10 @@ public NettyClientStreamRpc(RaftPeer server, TlsConf tlsConf, RaftProperties pro final InetSocketAddress address = NetUtils.createSocketAddr(server.getDataStreamAddress()); final SslContext sslContext = NettyUtils.buildSslContextForClient(tlsConf); + final RetryPolicy reconnectPolicy = + RetryPolicy.parse(NettyConfigKeys.DataStream.Client.reconnectPolicy(properties)); this.connection = new Connection(address, WorkerGroupGetter.newInstance(properties), - () -> newChannelInitializer(address, sslContext, getClientHandler())); + () -> newChannelInitializer(address, sslContext, getClientHandler()), reconnectPolicy); } private ChannelInboundHandler getClientHandler(){ @@ -492,4 +528,28 @@ public void close() { public String toString() { return name; } + + // Visible for tests. + @VisibleForTesting + RetryPolicy getReconnectPolicy() { + return connection.reconnectPolicy; + } + + // Visible for tests. + boolean waitForChannelActive(TimeDuration timeout) { + final long deadline = System.nanoTime() + timeout.toLong(TimeUnit.NANOSECONDS); + while (System.nanoTime() < deadline) { + final Channel channel = connection.getChannelUninterruptibly(); + if (channel != null && channel.isActive()) { + return true; + } + try { + Thread.sleep(100L); + } catch (InterruptedException e) { + Thread.currentThread().interrupt(); + return false; + } + } + return false; + } } diff --git a/ratis-test/src/test/java/org/apache/ratis/netty/client/TestNettyClientStreamRpcReconnectBackoff.java b/ratis-test/src/test/java/org/apache/ratis/netty/client/TestNettyClientStreamRpcReconnectBackoff.java new file mode 100644 index 0000000000..304c488fc1 --- /dev/null +++ b/ratis-test/src/test/java/org/apache/ratis/netty/client/TestNettyClientStreamRpcReconnectBackoff.java @@ -0,0 +1,79 @@ +/** + * 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.ratis.netty.client; + +import org.apache.ratis.conf.RaftProperties; +import org.apache.ratis.netty.NettyConfigKeys; +import org.apache.ratis.protocol.RaftPeer; +import org.apache.ratis.retry.ExponentialBackoffRetry; +import org.apache.ratis.retry.RetryPolicy; +import org.apache.ratis.util.TimeDuration; +import org.junit.jupiter.api.Test; + +import java.net.InetSocketAddress; +import java.util.concurrent.TimeUnit; + +import static org.junit.jupiter.api.Assertions.assertFalse; +import static org.junit.jupiter.api.Assertions.assertTrue; + +public class TestNettyClientStreamRpcReconnectBackoff { + @Test + public void testReconnectPolicyBackoffRanges() throws Exception { + // Use a small base/max to keep the test fast and deterministic in range checks. + final RaftProperties properties = new RaftProperties(); + final TimeDuration base = TimeDuration.valueOf(100, TimeUnit.MILLISECONDS); + final TimeDuration max = TimeDuration.valueOf(400, TimeUnit.MILLISECONDS); + final int maxAttempts = 5; + NettyConfigKeys.DataStream.Client.setReconnectPolicy(properties, + "ExponentialBackoffRetry," + base + "," + max + "," + maxAttempts); + + final RaftPeer peer = RaftPeer.newBuilder() + .setId("s1") + .setDataStreamAddress(new InetSocketAddress("127.0.0.1", 1)) + .build(); + + final NettyClientStreamRpc rpc = new NettyClientStreamRpc(peer, null, properties); + try { + // Verify the reconnect policy is exponential and uses the configured maxAttempts. + final RetryPolicy policy = rpc.getReconnectPolicy(); + assertTrue(policy instanceof ExponentialBackoffRetry); + assertFalse(policy.handleAttemptFailure(() -> maxAttempts).shouldRetry()); + + // attempt=0 -> base delay; attempt=1 -> 2x base; attempt=3 -> capped by max. + assertSleepInRange(policy, 0, base, max); + assertSleepInRange(policy, 1, base, max); + // Attempt 3 should be capped by max sleep time. + assertSleepInRange(policy, 3, base, max); + } finally { + rpc.close(); + } + } + + private static void assertSleepInRange(RetryPolicy policy, int attempt, TimeDuration base, TimeDuration max) { + final RetryPolicy.Action action = policy.handleAttemptFailure(() -> attempt); + assertTrue(action.shouldRetry()); + + final long baseMillis = base.toLong(TimeUnit.MILLISECONDS); + final long maxMillis = max.toLong(TimeUnit.MILLISECONDS); + final long expected = Math.min(maxMillis, baseMillis * (1L << attempt)); + final long actual = action.getSleepTime().toLong(TimeUnit.MILLISECONDS); + + assertTrue(actual >= expected / 2, "delay too small: " + actual); + assertTrue(actual <= expected + expected / 2, "delay too large: " + actual); + } +} diff --git a/ratis-test/src/test/java/org/apache/ratis/netty/client/TestNettyDataStreamReconnectWithGrpcCluster.java b/ratis-test/src/test/java/org/apache/ratis/netty/client/TestNettyDataStreamReconnectWithGrpcCluster.java new file mode 100644 index 0000000000..2be0bc2607 --- /dev/null +++ b/ratis-test/src/test/java/org/apache/ratis/netty/client/TestNettyDataStreamReconnectWithGrpcCluster.java @@ -0,0 +1,93 @@ +/* + * 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.ratis.netty.client; + +import org.apache.ratis.BaseTest; +import org.apache.ratis.RaftTestUtil; +import org.apache.ratis.client.DataStreamClient; +import org.apache.ratis.client.RaftClient; +import org.apache.ratis.conf.RaftProperties; +import org.apache.ratis.datastream.DataStreamTestUtils.MultiDataStreamStateMachine; +import org.apache.ratis.datastream.MiniRaftClusterWithRpcTypeGrpcAndDataStreamTypeNetty; +import org.apache.ratis.netty.NettyConfigKeys; +import org.apache.ratis.protocol.RaftPeer; +import org.apache.ratis.retry.ExponentialBackoffRetry; +import org.apache.ratis.retry.RetryPolicy; +import org.apache.ratis.util.IOUtils; +import org.apache.ratis.util.TimeDuration; +import org.junit.jupiter.api.Test; +import org.junit.jupiter.api.Timeout; + +import java.util.concurrent.TimeUnit; + +import static org.junit.jupiter.api.Assertions.assertTrue; + +@Timeout(value = 120) +public class TestNettyDataStreamReconnectWithGrpcCluster extends BaseTest + implements MiniRaftClusterWithRpcTypeGrpcAndDataStreamTypeNetty.FactoryGet { + { + setStateMachine(MultiDataStreamStateMachine.class); + } + + @Test + public void testReconnectConfigApplied() throws Exception { + final RaftProperties properties = getProperties(); + final TimeDuration reconnectDelay = TimeDuration.valueOf(200, TimeUnit.MILLISECONDS); + final TimeDuration reconnectMaxDelay = TimeDuration.valueOf(400, TimeUnit.MILLISECONDS); + NettyConfigKeys.DataStream.Client.setReconnectPolicy(properties, + "ExponentialBackoffRetry," + reconnectDelay + "," + reconnectMaxDelay + ",10"); + + runWithNewCluster(1, cluster -> { + RaftTestUtil.waitForLeader(cluster); + final RaftPeer primary = cluster.getLeader().getPeer(); + + final RaftClient client = cluster.createClient(primary); + try { + final DataStreamClient dataStreamClient = (DataStreamClient) client.getDataStreamApi(); + final NettyClientStreamRpc rpc = (NettyClientStreamRpc) dataStreamClient.getClientRpc(); + + // Verify reconnect configuration is applied. + final RetryPolicy policy = rpc.getReconnectPolicy(); + assertTrue(policy instanceof ExponentialBackoffRetry); + assertSleepInRange(policy, 0, reconnectDelay, reconnectMaxDelay); + assertSleepInRange(policy, 1, reconnectDelay, reconnectMaxDelay); + + // Verify the data stream channel can be established. + assertTrue(rpc.waitForChannelActive(TimeDuration.valueOf(5, TimeUnit.SECONDS)), + "Data stream channel should be active"); + } finally { + IOUtils.cleanup(LOG, client); + } + }); + } + + private static void assertSleepInRange(RetryPolicy policy, int attempt, TimeDuration base, TimeDuration max) { + final RetryPolicy.Action action = policy.handleAttemptFailure(() -> attempt); + assertTrue(action.shouldRetry()); + + final long baseMillis = base.toLong(TimeUnit.MILLISECONDS); + final long maxMillis = max.toLong(TimeUnit.MILLISECONDS); + final long expected = Math.min(maxMillis, baseMillis * (1L << attempt)); + final long actual = action.getSleepTime().toLong(TimeUnit.MILLISECONDS); + + assertTrue(actual >= expected / 2, "delay too small: " + actual); + assertTrue(actual <= expected + expected / 2, "delay too large: " + actual); + } + +} From 132c3e580e72cd886c215403388a814d7eb5ab77 Mon Sep 17 00:00:00 2001 From: slfan1989 <55643692+slfan1989@users.noreply.github.com> Date: Thu, 19 Feb 2026 05:23:13 +0800 Subject: [PATCH 259/397] RATIS-2412. NettyClient LoggingHandler not working. (#1353) --- .../src/main/java/org/apache/ratis/netty/NettyClient.java | 4 ---- .../src/main/java/org/apache/ratis/netty/NettyRpcProxy.java | 5 ++++- 2 files changed, 4 insertions(+), 5 deletions(-) diff --git a/ratis-netty/src/main/java/org/apache/ratis/netty/NettyClient.java b/ratis-netty/src/main/java/org/apache/ratis/netty/NettyClient.java index a42ddaca83..56ca6b030c 100644 --- a/ratis-netty/src/main/java/org/apache/ratis/netty/NettyClient.java +++ b/ratis-netty/src/main/java/org/apache/ratis/netty/NettyClient.java @@ -24,8 +24,6 @@ import org.apache.ratis.thirdparty.io.netty.channel.ChannelInitializer; import org.apache.ratis.thirdparty.io.netty.channel.EventLoopGroup; import org.apache.ratis.thirdparty.io.netty.channel.socket.SocketChannel; -import org.apache.ratis.thirdparty.io.netty.handler.logging.LogLevel; -import org.apache.ratis.thirdparty.io.netty.handler.logging.LoggingHandler; import org.apache.ratis.util.JavaUtils; import org.apache.ratis.util.LifeCycle; import org.apache.ratis.util.NetUtils; @@ -47,12 +45,10 @@ public class NettyClient implements Closeable { public void connect(EventLoopGroup group, ChannelInitializer initializer) throws InterruptedException { final InetSocketAddress address = NetUtils.createSocketAddr(serverAddress); - lifeCycle.startAndTransition( () -> channel = new Bootstrap() .group(group) .channel(NettyUtils.getSocketChannelClass(group)) - .handler(new LoggingHandler(LogLevel.INFO)) .handler(initializer) .connect(address) .sync() diff --git a/ratis-netty/src/main/java/org/apache/ratis/netty/NettyRpcProxy.java b/ratis-netty/src/main/java/org/apache/ratis/netty/NettyRpcProxy.java index f77096e186..675e0e1fe5 100644 --- a/ratis-netty/src/main/java/org/apache/ratis/netty/NettyRpcProxy.java +++ b/ratis-netty/src/main/java/org/apache/ratis/netty/NettyRpcProxy.java @@ -31,6 +31,8 @@ import org.apache.ratis.proto.netty.NettyProtos.RaftNettyServerReplyProto; import org.apache.ratis.proto.netty.NettyProtos.RaftNettyServerRequestProto; import org.apache.ratis.protocol.exceptions.AlreadyClosedException; +import org.apache.ratis.thirdparty.io.netty.handler.logging.LogLevel; +import org.apache.ratis.thirdparty.io.netty.handler.logging.LoggingHandler; import org.apache.ratis.util.IOUtils; import org.apache.ratis.util.PeerProxyMap; import org.apache.ratis.util.ProtoUtils; @@ -141,9 +143,10 @@ public void channelInactive(ChannelHandlerContext ctx) throws Exception { final ChannelInitializer initializer = new ChannelInitializer() { @Override - protected void initChannel(SocketChannel ch) throws Exception { + protected void initChannel(SocketChannel ch) { final ChannelPipeline p = ch.pipeline(); + p.addLast(new LoggingHandler(LogLevel.WARN)); p.addLast(new ProtobufVarint32FrameDecoder()); p.addLast(new ProtobufDecoder(RaftNettyServerReplyProto.getDefaultInstance())); p.addLast(new ProtobufVarint32LengthFieldPrepender()); From 963d31002c802725a4d7e78da0aff652f791a56f Mon Sep 17 00:00:00 2001 From: slfan1989 <55643692+slfan1989@users.noreply.github.com> Date: Fri, 20 Feb 2026 09:29:53 +0800 Subject: [PATCH 260/397] RATIS-2413. Support different RetryPolicy implementations. (#1354) --- .../org/apache/ratis/retry/RetryPolicy.java | 37 +++++++++- .../ratis/retry/TestRetryPolicyParse.java | 67 +++++++++++++++++++ .../src/site/markdown/configurations.md | 26 +++++-- .../ratis/grpc/server/GrpcLogAppender.java | 6 +- .../netty/client/NettyClientStreamRpc.java | 5 +- 5 files changed, 128 insertions(+), 13 deletions(-) create mode 100644 ratis-common/src/test/java/org/apache/ratis/retry/TestRetryPolicyParse.java diff --git a/ratis-common/src/main/java/org/apache/ratis/retry/RetryPolicy.java b/ratis-common/src/main/java/org/apache/ratis/retry/RetryPolicy.java index 6916858e57..0885e0a44a 100644 --- a/ratis-common/src/main/java/org/apache/ratis/retry/RetryPolicy.java +++ b/ratis-common/src/main/java/org/apache/ratis/retry/RetryPolicy.java @@ -19,6 +19,7 @@ import org.apache.ratis.util.TimeDuration; +import java.util.Arrays; import java.util.Objects; import java.util.concurrent.TimeUnit; @@ -76,12 +77,19 @@ default Throwable getCause() { */ Action handleAttemptFailure(Event event); + static RetryPolicy parse(String commaSeparated, String name) { + try { + return parse(commaSeparated); + } catch (Exception e) { + throw new IllegalArgumentException("Failed to parse " + name + ": \"" + commaSeparated + "\"", e); + } + } + static RetryPolicy parse(String commaSeparated) { Objects.requireNonNull(commaSeparated, "commaSeparated == null"); final String[] args = commaSeparated.split(","); if (args.length < 1) { - throw new IllegalArgumentException("Failed to parse RetryPolicy: args.length = " - + args.length + " < 1 for " + commaSeparated); + throw new IllegalArgumentException("Failed to parse RetryPolicy: empty comma separated string"); } final String classname = args[0].trim(); if (classname.equals(ExponentialBackoffRetry.class.getSimpleName())) { @@ -95,7 +103,30 @@ static RetryPolicy parse(String commaSeparated) { .setMaxAttempts(Integer.parseInt(args[3].trim())) .build(); } + if (classname.equals(MultipleLinearRandomRetry.class.getSimpleName())) { + if (args.length == 1) { + throw new IllegalArgumentException( + "Failed to parse MultipleLinearRandomRetry: the parameter list is empty for " + commaSeparated); + } + final String params = String.join(",", Arrays.copyOfRange(args, 1, args.length)); + return MultipleLinearRandomRetry.parseCommaSeparated(params); + } + // Backward compatibility: legacy config omits class name and starts with a duration (e.g. "1ms"). + if (isLegacyMultipleLinearRandomRetryParams(classname)) { + return MultipleLinearRandomRetry.parseCommaSeparated(commaSeparated); + } + // If a class name is present but unknown, fail fast to surface config errors. throw new IllegalArgumentException("Failed to parse RetryPolicy: unknown class " - + args[0] + " for " + commaSeparated); + + classname + " for " + commaSeparated); + } + + static boolean isLegacyMultipleLinearRandomRetryParams(String firstElement) { + // The legacy format starts with a duration token, not a class name. + try { + final TimeDuration t = TimeDuration.valueOf(firstElement, TimeUnit.MILLISECONDS); + return t.isPositive(); + } catch (RuntimeException e) { + return false; + } } } diff --git a/ratis-common/src/test/java/org/apache/ratis/retry/TestRetryPolicyParse.java b/ratis-common/src/test/java/org/apache/ratis/retry/TestRetryPolicyParse.java new file mode 100644 index 0000000000..17edb38682 --- /dev/null +++ b/ratis-common/src/test/java/org/apache/ratis/retry/TestRetryPolicyParse.java @@ -0,0 +1,67 @@ +/* + * 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.ratis.retry; + +import org.junit.jupiter.api.Test; + +import static org.junit.jupiter.api.Assertions.assertEquals; +import static org.junit.jupiter.api.Assertions.assertInstanceOf; +import static org.junit.jupiter.api.Assertions.assertThrows; + +class TestRetryPolicyParse { + @Test + void testParseExponentialBackoffRetry() { + final RetryPolicy policy = RetryPolicy.parse("ExponentialBackoffRetry,100ms,5s,100"); + assertInstanceOf(ExponentialBackoffRetry.class, policy); + } + + @Test + void testParseMultipleLinearRandomRetryWithClassname() { + final MultipleLinearRandomRetry expected = + MultipleLinearRandomRetry.parseCommaSeparated("1ms,10,1s,20,5s,1000"); + final RetryPolicy actual = + RetryPolicy.parse("MultipleLinearRandomRetry,1ms,10,1s,20,5s,1000"); + assertEquals(expected, actual); + } + + @Test + void testParseMultipleLinearRandomRetryWithoutClassname() { + final MultipleLinearRandomRetry expected = + MultipleLinearRandomRetry.parseCommaSeparated("1ms,10,1s,20,5s,1000"); + final RetryPolicy actual = RetryPolicy.parse("1ms,10,1s,20,5s,1000"); + assertEquals(expected, actual); + } + + @Test + void testParseUnknownClassnameThrows() { + assertThrows(IllegalArgumentException.class, + () -> RetryPolicy.parse("UnknownRetry,1ms,10")); + } + + @Test + void testParseMultipleLinearRandomRetryMissingParamsThrows() { + assertThrows(IllegalArgumentException.class, + () -> RetryPolicy.parse("MultipleLinearRandomRetry")); + } + + @Test + void testParseNonLegacyUnknownFirstTokenThrows() { + assertThrows(IllegalArgumentException.class, + () -> RetryPolicy.parse("not_a_duration,1ms,10")); + } +} diff --git a/ratis-docs/src/site/markdown/configurations.md b/ratis-docs/src/site/markdown/configurations.md index 52eef048fa..67e988348c 100644 --- a/ratis-docs/src/site/markdown/configurations.md +++ b/ratis-docs/src/site/markdown/configurations.md @@ -509,11 +509,27 @@ The follower's statemachine is responsible for fetching and installing snapshot | **Type** | string | | **Default** | 1ms,10, 1s,20, 5s,1000 | -"1ms,10, 1s,20, 5s,1000" means -The min wait time as 1ms (0 is not allowed) for first 10, -(5 iteration with 2 times grpc client retry), -next wait 1sec for next 20 retry (10 iteration with 2 times grpc client) -further wait for 5sec for max times ((5sec*980)/2 times ~= 40min) +Format: +`,` +If `` is omitted, it defaults to `MultipleLinearRandomRetry` for backward compatibility. + +Examples: +- `MultipleLinearRandomRetry,1ms,10,1s,20,5s,1000` +- `1ms,10,1s,20,5s,1000` (same as above) +- `ExponentialBackoffRetry,100ms,5s,100` + +For `MultipleLinearRandomRetry`, the parameter "1ms,10, 1s,20, 5s,1000" means +that the wait time is 1ms on average for the first 10 retries. +Then, it becomes 1s on average for next 20 retries +and 5s on average for the last 1000 retries. + +For `ExponentialBackoffRetry`, the parameter "100ms,5s,100" means +that the base wait time is 100ms, the maximum wait time is 5s +and the number of attempts is 100. +The wait time is $\min(2^{n-1} \times 100\text{ms}, 5\text{s})$ on average for the n-th retry. +In other words, the wait times are on average 100ms, 200ms, 400ms, 800ms, 1.6s, 3.2s, 5s, 5s and so on. + +Note that the actual wait time is randomized by a multiplier in the range [0.5, 1.5) for all retry policies. -------------------------------------------------------------------------------- diff --git a/ratis-grpc/src/main/java/org/apache/ratis/grpc/server/GrpcLogAppender.java b/ratis-grpc/src/main/java/org/apache/ratis/grpc/server/GrpcLogAppender.java index 9ce45d1abb..b4d78c207a 100644 --- a/ratis-grpc/src/main/java/org/apache/ratis/grpc/server/GrpcLogAppender.java +++ b/ratis-grpc/src/main/java/org/apache/ratis/grpc/server/GrpcLogAppender.java @@ -24,7 +24,6 @@ import org.apache.ratis.metrics.Timekeeper; import org.apache.ratis.proto.RaftProtos.InstallSnapshotResult; import org.apache.ratis.protocol.RaftPeerId; -import org.apache.ratis.retry.MultipleLinearRandomRetry; import org.apache.ratis.retry.RetryPolicy; import org.apache.ratis.server.RaftServer; import org.apache.ratis.server.RaftServerConfigKeys; @@ -198,8 +197,9 @@ public GrpcLogAppender(RaftServer.Division server, LeaderState leaderState, Foll lock = new AutoCloseableReadWriteLock(this); caller = LOG.isTraceEnabled()? JavaUtils.getCallerStackTraceElement(): null; - errorRetryWaitPolicy = MultipleLinearRandomRetry.parseCommaSeparated( - RaftServerConfigKeys.Log.Appender.retryPolicy(properties)); + errorRetryWaitPolicy = RetryPolicy.parse( + RaftServerConfigKeys.Log.Appender.retryPolicy(properties), + RaftServerConfigKeys.Log.Appender.RETRY_POLICY_KEY); } @Override diff --git a/ratis-netty/src/main/java/org/apache/ratis/netty/client/NettyClientStreamRpc.java b/ratis-netty/src/main/java/org/apache/ratis/netty/client/NettyClientStreamRpc.java index 44e91d283e..54ad8acf66 100644 --- a/ratis-netty/src/main/java/org/apache/ratis/netty/client/NettyClientStreamRpc.java +++ b/ratis-netty/src/main/java/org/apache/ratis/netty/client/NettyClientStreamRpc.java @@ -347,8 +347,9 @@ public NettyClientStreamRpc(RaftPeer server, TlsConf tlsConf, RaftProperties pro final InetSocketAddress address = NetUtils.createSocketAddr(server.getDataStreamAddress()); final SslContext sslContext = NettyUtils.buildSslContextForClient(tlsConf); - final RetryPolicy reconnectPolicy = - RetryPolicy.parse(NettyConfigKeys.DataStream.Client.reconnectPolicy(properties)); + final RetryPolicy reconnectPolicy = RetryPolicy.parse( + NettyConfigKeys.DataStream.Client.reconnectPolicy(properties), + NettyConfigKeys.DataStream.Client.RECONNECT_POLICY_KEY); this.connection = new Connection(address, WorkerGroupGetter.newInstance(properties), () -> newChannelInitializer(address, sslContext, getClientHandler()), reconnectPolicy); } From 6ac85aef7a33897bd925acf36b3a57be942a42ea Mon Sep 17 00:00:00 2001 From: slfan1989 <55643692+slfan1989@users.noreply.github.com> Date: Sun, 22 Feb 2026 03:16:05 +0800 Subject: [PATCH 261/397] RATIS-2414. Add leak detection for ZeroCopyMessageMarshaller. (#1355) --- .../grpc/util/ZeroCopyMessageMarshaller.java | 23 ++++++++++++++ .../grpc/util/GrpcZeroCopyTestServer.java | 31 +++++++++++++++++-- .../ratis/grpc/util/TestGrpcZeroCopy.java | 30 +++++++++++++----- 3 files changed, 75 insertions(+), 9 deletions(-) diff --git a/ratis-grpc/src/main/java/org/apache/ratis/grpc/util/ZeroCopyMessageMarshaller.java b/ratis-grpc/src/main/java/org/apache/ratis/grpc/util/ZeroCopyMessageMarshaller.java index bb8183a247..3fb3f067be 100644 --- a/ratis-grpc/src/main/java/org/apache/ratis/grpc/util/ZeroCopyMessageMarshaller.java +++ b/ratis-grpc/src/main/java/org/apache/ratis/grpc/util/ZeroCopyMessageMarshaller.java @@ -222,4 +222,27 @@ private T parseFrom(CodedInputStream stream) throws InvalidProtocolBufferExcepti public InputStream popStream(T message) { return unclosedStreams.remove(message); } + + void assertNoUnclosedStreams() { + // Intended for tests/teardown to fail fast if callers forgot to release streams. + final int size = unclosedStreams.size(); + Preconditions.assertTrue(size == 0, () -> name + ": " + size + " unclosed stream(s)"); + } + + public void close() { + // Cleanup helper for tests/teardown; do not call while streams may still be in use. + synchronized (unclosedStreams) { + if (unclosedStreams.isEmpty()) { + return; + } + for (InputStream stream : unclosedStreams.values()) { + try { + stream.close(); + } catch (IOException e) { + LOG.warn("{}: Failed to close leaked stream.", name, e); + } + } + unclosedStreams.clear(); + } + } } diff --git a/ratis-test/src/test/java/org/apache/ratis/grpc/util/GrpcZeroCopyTestServer.java b/ratis-test/src/test/java/org/apache/ratis/grpc/util/GrpcZeroCopyTestServer.java index a07872c066..ca2709270b 100644 --- a/ratis-test/src/test/java/org/apache/ratis/grpc/util/GrpcZeroCopyTestServer.java +++ b/ratis-test/src/test/java/org/apache/ratis/grpc/util/GrpcZeroCopyTestServer.java @@ -77,12 +77,19 @@ public synchronized String toString() { private final Count nonZeroCopyCount = new Count(); private final Server server; + // Allow tests to disable release to validate leak detection. + private final boolean releaseRequests; private final ZeroCopyMessageMarshaller marshaller = new ZeroCopyMessageMarshaller<>( BinaryRequest.getDefaultInstance(), zeroCopyCount::inc, nonZeroCopyCount::inc); GrpcZeroCopyTestServer(int port) { + this(port, true); + } + + GrpcZeroCopyTestServer(int port, boolean releaseRequests) { + this.releaseRequests = releaseRequests; final GreeterImpl greeter = new GreeterImpl(); final MethodDescriptor binary = GreeterGrpc.getBinaryMethod(); final String binaryFullMethodName = binary.getFullMethodName(); @@ -128,11 +135,29 @@ int start() throws IOException { @Override public void close() throws IOException { + // Shutdown server first, then assert no leaked streams and cleanup if needed. + IOException ioe = null; try { server.shutdown().awaitTermination(5, TimeUnit.SECONDS); } catch (InterruptedException e) { Thread.currentThread().interrupt(); - throw IOUtils.toInterruptedIOException("Failed to close", e); + ioe = IOUtils.toInterruptedIOException("Failed to close", e); + } + + try { + marshaller.assertNoUnclosedStreams(); + } catch (RuntimeException e) { + if (ioe != null) { + ioe.addSuppressed(e); + throw ioe; + } + throw e; + } finally { + marshaller.close(); + } + + if (ioe != null) { + throw ioe; } } @@ -179,7 +204,9 @@ public void onNext(BinaryRequest request) { ByteBuffer.wrap(bytes).putInt(data.size()); responseObserver.onNext(BinaryReply.newBuilder().setData(UnsafeByteOperations.unsafeWrap(bytes)).build()); } finally { - marshaller.release(request); + if (releaseRequests) { + marshaller.release(request); + } } } diff --git a/ratis-test/src/test/java/org/apache/ratis/grpc/util/TestGrpcZeroCopy.java b/ratis-test/src/test/java/org/apache/ratis/grpc/util/TestGrpcZeroCopy.java index 9ffe7ecd08..dde44e5799 100644 --- a/ratis-test/src/test/java/org/apache/ratis/grpc/util/TestGrpcZeroCopy.java +++ b/ratis-test/src/test/java/org/apache/ratis/grpc/util/TestGrpcZeroCopy.java @@ -26,9 +26,13 @@ import org.apache.ratis.thirdparty.io.netty.buffer.PooledByteBufAllocator; import org.apache.ratis.util.NetUtils; import org.apache.ratis.util.TraditionalBinaryPrefix; -import org.junit.jupiter.api.Assertions; import org.junit.jupiter.api.Test; +import static org.junit.jupiter.api.Assertions.assertEquals; +import static org.junit.jupiter.api.Assertions.assertThrows; +import static org.junit.jupiter.api.Assertions.assertTrue; +import static org.junit.jupiter.api.Assumptions.assumeTrue; + import java.util.ArrayList; import java.util.List; import java.util.Random; @@ -60,8 +64,8 @@ static void verify(long seed, ByteString b) { RANDOM.nextBytes(ARRAY); final ByteString expected = UnsafeByteOperations.unsafeWrap(ARRAY, 0, remaining); final ByteString computed = b.substring(offset, offset + remaining); - Assertions.assertEquals(expected.size(), computed.size()); - Assertions.assertEquals(expected, computed); + assertEquals(expected.size(), computed.size()); + assertEquals(expected, computed); offset += remaining; } } @@ -99,7 +103,7 @@ public static boolean isReady() { /** Test a zero-copy marshaller is available from the versions of gRPC and Protobuf. */ @Test public void testReadiness() { - Assertions.assertTrue(isReady()); + assertTrue(isReady()); } @@ -108,6 +112,18 @@ public void testZeroCopy() throws Exception { runTestZeroCopy(); } + @Test + public void testLeakCheck() throws Exception { + // Verify leak detection by disabling release on the server side. + assumeTrue(isReady()); + final GrpcZeroCopyTestServer server = new GrpcZeroCopyTestServer(NetUtils.getFreePort(), false); + final int port = server.start(); + try (GrpcZeroCopyTestClient client = new GrpcZeroCopyTestClient(NetUtils.LOCALHOST, port)) { + sendBinaries(1, client, server); + } + assertThrows(IllegalStateException.class, server::close); + } + void runTestZeroCopy() throws Exception { try (GrpcZeroCopyTestServer server = new GrpcZeroCopyTestServer(NetUtils.getFreePort())) { final int port = server.start(); @@ -134,7 +150,7 @@ void sendMessages(int n, GrpcZeroCopyTestClient client, GrpcZeroCopyTestServer s for (int i = 0; i < futures.size(); i++) { final String expected = GrpcZeroCopyTestServer.toReply(i, messages.get(i)); final String reply = futures.get(i).get(); - Assertions.assertEquals(expected, reply, "expected = " + expected + " != reply = " + reply); + assertEquals(expected, reply, "expected = " + expected + " != reply = " + reply); server.assertCounts(numElements, numBytes); } } @@ -159,8 +175,8 @@ void sendBinaries(int n, GrpcZeroCopyTestClient client, GrpcZeroCopyTestServer s } final ByteString reply = future.get(); - Assertions.assertEquals(4, reply.size()); - Assertions.assertEquals(size, reply.asReadOnlyByteBuffer().getInt()); + assertEquals(4, reply.size()); + assertEquals(size, reply.asReadOnlyByteBuffer().getInt()); numElements++; numBytes += size; From dc4facd8aa95fb4ec20ec537c2fe82fdfa33e790 Mon Sep 17 00:00:00 2001 From: slfan1989 <55643692+slfan1989@users.noreply.github.com> Date: Sun, 22 Feb 2026 04:54:27 +0800 Subject: [PATCH 262/397] RATIS-2415. Fix queue corruption in NettyRpcProxy when request sending fails. (#1356) --- .../org/apache/ratis/netty/NettyRpcProxy.java | 106 ++++++++++++----- .../ratis/server/impl/RaftServerImpl.java | 7 +- .../ratis/server/impl/ServerProtoUtils.java | 8 +- .../apache/ratis/netty/TestNettyRpcProxy.java | 112 ++++++++++++++++++ .../ratis/netty/TestRaftAsyncWithNetty.java | 10 +- .../TestLeaderElectionServerInterface.java | 5 +- 6 files changed, 212 insertions(+), 36 deletions(-) create mode 100644 ratis-test/src/test/java/org/apache/ratis/netty/TestNettyRpcProxy.java diff --git a/ratis-netty/src/main/java/org/apache/ratis/netty/NettyRpcProxy.java b/ratis-netty/src/main/java/org/apache/ratis/netty/NettyRpcProxy.java index 675e0e1fe5..cfcabc2741 100644 --- a/ratis-netty/src/main/java/org/apache/ratis/netty/NettyRpcProxy.java +++ b/ratis-netty/src/main/java/org/apache/ratis/netty/NettyRpcProxy.java @@ -34,7 +34,9 @@ import org.apache.ratis.thirdparty.io.netty.handler.logging.LogLevel; import org.apache.ratis.thirdparty.io.netty.handler.logging.LoggingHandler; import org.apache.ratis.util.IOUtils; +import org.apache.ratis.util.JavaUtils; import org.apache.ratis.util.PeerProxyMap; +import org.apache.ratis.util.Preconditions; import org.apache.ratis.util.ProtoUtils; import org.apache.ratis.util.TimeDuration; import org.slf4j.Logger; @@ -42,9 +44,10 @@ import java.io.Closeable; import java.io.IOException; -import java.util.LinkedList; -import java.util.Queue; +import java.util.Map; +import java.util.Objects; import java.util.concurrent.CompletableFuture; +import java.util.concurrent.ConcurrentHashMap; import java.util.concurrent.ExecutionException; import java.util.concurrent.TimeUnit; import java.util.concurrent.TimeoutException; @@ -80,6 +83,41 @@ public void close() { } } + static RaftRpcRequestProto getRequest(RaftNettyServerRequestProto proto) { + final RaftNettyServerRequestProto.RaftNettyServerRequestCase requestCase = proto.getRaftNettyServerRequestCase(); + switch (requestCase) { + case REQUESTVOTEREQUEST: + return proto.getRequestVoteRequest().getServerRequest(); + case APPENDENTRIESREQUEST: + return proto.getAppendEntriesRequest().getServerRequest(); + case INSTALLSNAPSHOTREQUEST: + return proto.getInstallSnapshotRequest().getServerRequest(); + case RAFTCLIENTREQUEST: + return proto.getRaftClientRequest().getRpcRequest(); + case SETCONFIGURATIONREQUEST: + return proto.getSetConfigurationRequest().getRpcRequest(); + case GROUPMANAGEMENTREQUEST: + return proto.getGroupManagementRequest().getRpcRequest(); + case GROUPLISTREQUEST: + return proto.getGroupListRequest().getRpcRequest(); + case GROUPINFOREQUEST: + return proto.getGroupInfoRequest().getRpcRequest(); + case TRANSFERLEADERSHIPREQUEST: + return proto.getTransferLeadershipRequest().getRpcRequest(); + case STARTLEADERELECTIONREQUEST: + return proto.getStartLeaderElectionRequest().getServerRequest(); + case SNAPSHOTMANAGEMENTREQUEST: + return proto.getSnapshotManagementRequest().getRpcRequest(); + case LEADERELECTIONMANAGEMENTREQUEST: + return proto.getLeaderElectionManagementRequest().getRpcRequest(); + + case RAFTNETTYSERVERREQUEST_NOT_SET: + throw new IllegalArgumentException("Request case not set in proto: " + requestCase); + default: + throw new UnsupportedOperationException("Request case not supported: " + requestCase); + } + } + public static long getCallId(RaftNettyServerReplyProto proto) { switch (proto.getRaftNettyServerReplyCase()) { case REQUESTVOTEREPLY: @@ -92,6 +130,10 @@ public static long getCallId(RaftNettyServerReplyProto proto) { return proto.getInstallSnapshotReply().getServerReply().getCallId(); case RAFTCLIENTREPLY: return proto.getRaftClientReply().getRpcReply().getCallId(); + case GROUPLISTREPLY: + return proto.getGroupListReply().getRpcReply().getCallId(); + case GROUPINFOREPLY: + return proto.getGroupInfoReply().getRpcReply().getCallId(); case EXCEPTIONREPLY: return proto.getExceptionReply().getRpcReply().getCallId(); case RAFTNETTYSERVERREPLY_NOT_SET: @@ -106,8 +148,7 @@ public static long getCallId(RaftNettyServerReplyProto proto) { class Connection implements Closeable { private final NettyClient client = new NettyClient(peer.getAddress()); - private final Queue> replies - = new LinkedList<>(); + private final Map> replies = new ConcurrentHashMap<>(); Connection(EventLoopGroup group) throws InterruptedException { final ChannelInboundHandler inboundHandler @@ -115,11 +156,7 @@ class Connection implements Closeable { @Override protected void channelRead0(ChannelHandlerContext ctx, RaftNettyServerReplyProto proto) { - final CompletableFuture future = pollReply(); - if (future == null) { - throw new IllegalStateException("Request #" + getCallId(proto) - + " not found"); - } + final CompletableFuture future = getReplyFuture(getCallId(proto), null); if (proto.getRaftNettyServerReplyCase() == EXCEPTIONREPLY) { final Object ioe = ProtoUtils.toObject(proto.getExceptionReply().getException()); future.completeExceptionally((IOException)ioe); @@ -159,14 +196,38 @@ protected void initChannel(SocketChannel ch) { client.connect(group, initializer); } - synchronized ChannelFuture offer(RaftNettyServerRequestProto request, - CompletableFuture reply) throws AlreadyClosedException { - replies.offer(reply); - return client.writeAndFlush(request); + private CompletableFuture getReplyFuture(long callId, + CompletableFuture expected) { + final CompletableFuture removed = replies.remove(callId); + Objects.requireNonNull(removed, () -> "Request #" + callId + " not found"); + if (expected != null) { + Preconditions.assertSame(expected, removed, "removed"); + } + return removed; } - synchronized CompletableFuture pollReply() { - return replies.poll(); + synchronized CompletableFuture offer(RaftNettyServerRequestProto request) { + final ChannelFuture future; + try { + future = client.writeAndFlush(request); + } catch (AlreadyClosedException e) { + return JavaUtils.completeExceptionally(e); + } + + final CompletableFuture reply = new CompletableFuture<>(); + final long callId = getRequest(request).getCallId(); + final CompletableFuture previous = replies.put(callId, reply); + Preconditions.assertNull(previous, "previous"); + + future.addListener(cf -> { + if (!cf.isSuccess()) { + // Remove from queue on async write failure to prevent reply mismatch. + // Only complete exceptionally if removal succeeds (not already polled). + getReplyFuture(callId, reply).completeExceptionally(cf.cause()); + client.close(); + } + }); + return reply; } @Override @@ -179,7 +240,7 @@ private synchronized void failOutstandingRequests(Throwable cause) { if (!replies.isEmpty()) { LOG.warn("Still have {} requests outstanding from {} connection: {}", replies.size(), peer, cause.toString()); - replies.forEach(f -> f.completeExceptionally(cause)); + replies.values().forEach(f -> f.completeExceptionally(cause)); replies.clear(); } } @@ -201,23 +262,14 @@ public void close() { } public CompletableFuture sendAsync(RaftNettyServerRequestProto proto) { - final CompletableFuture reply = new CompletableFuture<>(); - try { - connection.offer(proto, reply); - } catch (AlreadyClosedException e) { - reply.completeExceptionally(e); - } - return reply; + return connection.offer(proto); } public RaftNettyServerReplyProto send( RaftRpcRequestProto request, RaftNettyServerRequestProto proto) throws IOException { - final CompletableFuture reply = new CompletableFuture<>(); - final ChannelFuture channelFuture = connection.offer(proto, reply); - + final CompletableFuture reply = sendAsync(proto); try { - channelFuture.sync(); TimeDuration newDuration = requestTimeoutDuration.add(request.getTimeoutMs(), TimeUnit.MILLISECONDS); return reply.get(newDuration.getDuration(), newDuration.getUnit()); } catch (InterruptedException e) { diff --git a/ratis-server/src/main/java/org/apache/ratis/server/impl/RaftServerImpl.java b/ratis-server/src/main/java/org/apache/ratis/server/impl/RaftServerImpl.java index e5ebfafb37..d9dd09d966 100644 --- a/ratis-server/src/main/java/org/apache/ratis/server/impl/RaftServerImpl.java +++ b/ratis-server/src/main/java/org/apache/ratis/server/impl/RaftServerImpl.java @@ -1430,12 +1430,13 @@ public RequestVoteReplyProto requestVote(RequestVoteRequestProto r) throws IOExc RaftPeerId.valueOf(request.getRequestorId()), ProtoUtils.toRaftGroupId(request.getRaftGroupId()), r.getCandidateTerm(), - TermIndex.valueOf(r.getCandidateLastEntry())); + TermIndex.valueOf(r.getCandidateLastEntry()), + request.getCallId()); } private RequestVoteReplyProto requestVote(Phase phase, RaftPeerId candidateId, RaftGroupId candidateGroupId, - long candidateTerm, TermIndex candidateLastEntry) throws IOException { + long candidateTerm, TermIndex candidateLastEntry, long callId) throws IOException { CodeInjectionForTesting.execute(REQUEST_VOTE, getId(), candidateId, candidateTerm, candidateLastEntry); LOG.info("{}: receive requestVote({}, {}, {}, {}, {})", @@ -1470,7 +1471,7 @@ private RequestVoteReplyProto requestVote(Phase phase, shouldShutdown = true; } reply = toRequestVoteReplyProto(candidateId, getMemberId(), - voteGranted, state.getCurrentTerm(), shouldShutdown, state.getLastEntry()); + voteGranted, state.getCurrentTerm(), shouldShutdown, state.getLastEntry(), callId); if (LOG.isInfoEnabled()) { LOG.info("{} replies to {} vote request: {}. Peer's state: {}", getMemberId(), phase, toRequestVoteReplyString(reply), state); diff --git a/ratis-server/src/main/java/org/apache/ratis/server/impl/ServerProtoUtils.java b/ratis-server/src/main/java/org/apache/ratis/server/impl/ServerProtoUtils.java index f491aaee26..19d4ce6a75 100644 --- a/ratis-server/src/main/java/org/apache/ratis/server/impl/ServerProtoUtils.java +++ b/ratis-server/src/main/java/org/apache/ratis/server/impl/ServerProtoUtils.java @@ -24,6 +24,7 @@ import org.apache.ratis.protocol.RaftGroupMemberId; import org.apache.ratis.protocol.RaftPeer; import org.apache.ratis.protocol.RaftPeerId; +import org.apache.ratis.rpc.CallId; import org.apache.ratis.server.protocol.TermIndex; import org.apache.ratis.server.raftlog.RaftLog; import org.apache.ratis.util.Preconditions; @@ -44,9 +45,9 @@ private static RaftRpcReplyProto.Builder toRaftRpcReplyProtoBuilder( static RequestVoteReplyProto toRequestVoteReplyProto( RaftPeerId requestorId, RaftGroupMemberId replyId, boolean success, long term, boolean shouldShutdown, - TermIndex lastEntry) { + TermIndex lastEntry, long callId) { return RequestVoteReplyProto.newBuilder() - .setServerReply(toRaftRpcReplyProtoBuilder(requestorId, replyId, success)) + .setServerReply(toRaftRpcReplyProtoBuilder(requestorId, replyId, success).setCallId(callId)) .setTerm(term) .setShouldShutdown(shouldShutdown) .setLastEntry((lastEntry != null? lastEntry : TermIndex.INITIAL_VALUE).toProto()) @@ -56,7 +57,8 @@ static RequestVoteReplyProto toRequestVoteReplyProto( static RequestVoteRequestProto toRequestVoteRequestProto( RaftGroupMemberId requestorId, RaftPeerId replyId, long term, TermIndex lastEntry, boolean preVote) { final RequestVoteRequestProto.Builder b = RequestVoteRequestProto.newBuilder() - .setServerRequest(ClientProtoUtils.toRaftRpcRequestProtoBuilder(requestorId, replyId)) + .setServerRequest(ClientProtoUtils.toRaftRpcRequestProtoBuilder(requestorId, replyId) + .setCallId(CallId.getAndIncrement())) .setCandidateTerm(term) .setPreVote(preVote); Optional.ofNullable(lastEntry).map(TermIndex::toProto).ifPresent(b::setCandidateLastEntry); diff --git a/ratis-test/src/test/java/org/apache/ratis/netty/TestNettyRpcProxy.java b/ratis-test/src/test/java/org/apache/ratis/netty/TestNettyRpcProxy.java new file mode 100644 index 0000000000..780c249b6b --- /dev/null +++ b/ratis-test/src/test/java/org/apache/ratis/netty/TestNettyRpcProxy.java @@ -0,0 +1,112 @@ +/* + * 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.ratis.netty; + +import org.apache.ratis.BaseTest; +import org.apache.ratis.conf.RaftProperties; +import org.apache.ratis.protocol.RaftPeer; +import org.apache.ratis.protocol.exceptions.AlreadyClosedException; +import org.apache.ratis.proto.netty.NettyProtos.RaftNettyServerReplyProto; +import org.apache.ratis.proto.netty.NettyProtos.RaftNettyServerRequestProto; +import org.apache.ratis.thirdparty.io.netty.bootstrap.ServerBootstrap; +import org.apache.ratis.thirdparty.io.netty.channel.Channel; +import org.apache.ratis.thirdparty.io.netty.channel.ChannelFuture; +import org.apache.ratis.thirdparty.io.netty.channel.ChannelHandlerContext; +import org.apache.ratis.thirdparty.io.netty.channel.ChannelInitializer; +import org.apache.ratis.thirdparty.io.netty.channel.EventLoopGroup; +import org.apache.ratis.thirdparty.io.netty.channel.SimpleChannelInboundHandler; +import org.apache.ratis.thirdparty.io.netty.channel.socket.SocketChannel; +import org.apache.ratis.util.JavaUtils; +import org.junit.jupiter.api.Test; + +import java.lang.reflect.Field; +import java.net.InetSocketAddress; +import java.util.Map; +import java.util.concurrent.CompletableFuture; +import java.util.concurrent.CompletionException; + +import static org.junit.jupiter.api.Assertions.assertInstanceOf; +import static org.junit.jupiter.api.Assertions.assertThrows; +import static org.junit.jupiter.api.Assertions.assertTrue; + +public class TestNettyRpcProxy extends BaseTest { + @Test + public void testOfferRollbackOnAlreadyClosed() throws Exception { + // Minimal netty server to allow client connect; we don't need to process requests. + final EventLoopGroup bossGroup = NettyUtils.newEventLoopGroup("test-netty-boss", 1, false); + final EventLoopGroup workerGroup = NettyUtils.newEventLoopGroup("test-netty-worker", 1, false); + final EventLoopGroup clientGroup = NettyUtils.newEventLoopGroup("test-netty-client", 1, false); + Channel serverChannel = null; + NettyRpcProxy proxy = null; + try { + final ChannelFuture bindFuture = new ServerBootstrap() + .group(bossGroup, workerGroup) + .channel(NettyUtils.getServerChannelClass(workerGroup)) + .childHandler(new ChannelInitializer() { + @Override + protected void initChannel(SocketChannel ch) { + ch.pipeline().addLast(new SimpleChannelInboundHandler() { + @Override + protected void channelRead0(ChannelHandlerContext ctx, Object msg) { + } + }); + } + }) + .bind(0) + .sync(); + serverChannel = bindFuture.channel(); + + final InetSocketAddress address = (InetSocketAddress) serverChannel.localAddress(); + final String peerAddress = address.getHostString() + ":" + address.getPort(); + final RaftPeer peer = RaftPeer.newBuilder().setId("s0").setAddress(peerAddress).build(); + proxy = new NettyRpcProxy(peer, new RaftProperties(), clientGroup); + + // Close to force AlreadyClosedException on write and trigger rollback logic. + proxy.close(); + final CompletableFuture reply = + proxy.sendAsync(RaftNettyServerRequestProto.getDefaultInstance()); + + // Ensure the future completes exceptionally with AlreadyClosedException. + final Throwable thrown = assertThrows(CompletionException.class, reply::join); + final Throwable unwrapped = JavaUtils.unwrapCompletionException(thrown); + assertInstanceOf(AlreadyClosedException.class, unwrapped); + + // The replies queue must be empty after rollback; use reflection to reach it. + final Object connection = getField(proxy, "connection"); + final Map replies = getField(connection, "replies"); + assertTrue(replies.isEmpty()); + } finally { + if (proxy != null) { + proxy.close(); + } + if (serverChannel != null) { + serverChannel.close().sync(); + } + bossGroup.shutdownGracefully(); + workerGroup.shutdownGracefully(); + clientGroup.shutdownGracefully(); + } + } + + @SuppressWarnings("unchecked") + private static T getField(Object target, String name) throws Exception { + final Field field = target.getClass().getDeclaredField(name); + field.setAccessible(true); + return (T) field.get(target); + } +} diff --git a/ratis-test/src/test/java/org/apache/ratis/netty/TestRaftAsyncWithNetty.java b/ratis-test/src/test/java/org/apache/ratis/netty/TestRaftAsyncWithNetty.java index c09b07876f..d2f23e1439 100644 --- a/ratis-test/src/test/java/org/apache/ratis/netty/TestRaftAsyncWithNetty.java +++ b/ratis-test/src/test/java/org/apache/ratis/netty/TestRaftAsyncWithNetty.java @@ -18,10 +18,18 @@ package org.apache.ratis.netty; import org.apache.ratis.RaftAsyncTests; +import org.junit.jupiter.api.Test; import org.junit.jupiter.api.Timeout; @Timeout(100) public class TestRaftAsyncWithNetty extends RaftAsyncTests implements MiniRaftClusterWithNetty.FactoryGet { -} \ No newline at end of file + + @Override + @Test + @Timeout(500) + public void testWithLoadAsync() throws Exception { + super.testWithLoadAsync(); + } +} diff --git a/ratis-test/src/test/java/org/apache/ratis/server/impl/TestLeaderElectionServerInterface.java b/ratis-test/src/test/java/org/apache/ratis/server/impl/TestLeaderElectionServerInterface.java index 3a91f9a343..92b7ad4203 100644 --- a/ratis-test/src/test/java/org/apache/ratis/server/impl/TestLeaderElectionServerInterface.java +++ b/ratis-test/src/test/java/org/apache/ratis/server/impl/TestLeaderElectionServerInterface.java @@ -107,7 +107,8 @@ public RequestVoteReplyProto requestVote(RequestVoteRequestProto r) { final long term = (lastEntry != null? lastEntry : TermIndex.INITIAL_VALUE).getTerm(); // voter replies to candidate - return ServerProtoUtils.toRequestVoteReplyProto(getId(), voter, true, term, false, lastEntry); + return ServerProtoUtils.toRequestVoteReplyProto( + getId(), voter, true, term, false, lastEntry, r.getServerRequest().getCallId()); } @Override @@ -190,4 +191,4 @@ void runTestVoterWithEmptyLog(boolean expectToPass, TermIndex... lastEntries) { election.startInForeground(); } -} \ No newline at end of file +} From 9381df6a0994c4568da01e3c4ee8c94cfd582026 Mon Sep 17 00:00:00 2001 From: slfan1989 <55643692+slfan1989@users.noreply.github.com> Date: Tue, 24 Feb 2026 10:09:41 +0800 Subject: [PATCH 263/397] RATIS-2416. Fix memory leak in NettyClientReplies for stale replies. (#1357) --- .../netty/client/NettyClientReplies.java | 6 ++- .../netty/client/NettyClientStreamRpc.java | 2 +- .../netty/client/TestNettyClientReplies.java | 45 +++++++++++++++++++ 3 files changed, 51 insertions(+), 2 deletions(-) create mode 100644 ratis-test/src/test/java/org/apache/ratis/netty/client/TestNettyClientReplies.java diff --git a/ratis-netty/src/main/java/org/apache/ratis/netty/client/NettyClientReplies.java b/ratis-netty/src/main/java/org/apache/ratis/netty/client/NettyClientReplies.java index 4c49b1d160..695177262c 100644 --- a/ratis-netty/src/main/java/org/apache/ratis/netty/client/NettyClientReplies.java +++ b/ratis-netty/src/main/java/org/apache/ratis/netty/client/NettyClientReplies.java @@ -40,11 +40,15 @@ public class NettyClientReplies { private final ConcurrentMap replies = new ConcurrentHashMap<>(); - ReplyMap getReplyMap(ClientInvocationId clientInvocationId) { + ReplyMap getOrCreateReplyMap(ClientInvocationId clientInvocationId) { final MemoizedSupplier q = MemoizedSupplier.valueOf(() -> new ReplyMap(clientInvocationId)); return replies.computeIfAbsent(clientInvocationId, key -> q.get()); } + ReplyMap getReplyMap(ClientInvocationId clientInvocationId) { + return replies.get(clientInvocationId); + } + class ReplyMap { private final ClientInvocationId clientInvocationId; private final Map map = new ConcurrentHashMap<>(); diff --git a/ratis-netty/src/main/java/org/apache/ratis/netty/client/NettyClientStreamRpc.java b/ratis-netty/src/main/java/org/apache/ratis/netty/client/NettyClientStreamRpc.java index 54ad8acf66..4970d244a1 100644 --- a/ratis-netty/src/main/java/org/apache/ratis/netty/client/NettyClientStreamRpc.java +++ b/ratis-netty/src/main/java/org/apache/ratis/netty/client/NettyClientStreamRpc.java @@ -473,7 +473,7 @@ public CompletableFuture streamAsync(DataStreamRequest request) ClientInvocationId clientInvocationId = ClientInvocationId.valueOf(request.getClientId(), request.getStreamId()); final boolean isClose = request.getWriteOptionList().contains(StandardWriteOption.CLOSE); - final NettyClientReplies.ReplyMap replyMap = replies.getReplyMap(clientInvocationId); + final NettyClientReplies.ReplyMap replyMap = replies.getOrCreateReplyMap(clientInvocationId); final ChannelFuture channelFuture; final Channel channel; final NettyClientReplies.RequestEntry requestEntry = new NettyClientReplies.RequestEntry(request); diff --git a/ratis-test/src/test/java/org/apache/ratis/netty/client/TestNettyClientReplies.java b/ratis-test/src/test/java/org/apache/ratis/netty/client/TestNettyClientReplies.java new file mode 100644 index 0000000000..5e22761e24 --- /dev/null +++ b/ratis-test/src/test/java/org/apache/ratis/netty/client/TestNettyClientReplies.java @@ -0,0 +1,45 @@ +/* + * 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.ratis.netty.client; + +import org.apache.ratis.protocol.ClientId; +import org.apache.ratis.protocol.ClientInvocationId; +import org.junit.jupiter.api.Test; + +import static org.junit.jupiter.api.Assertions.assertNotNull; +import static org.junit.jupiter.api.Assertions.assertNull; +import static org.junit.jupiter.api.Assertions.assertSame; + +public class TestNettyClientReplies { + @Test + public void testGetReplyMapDoesNotCreate() { + final NettyClientReplies replies = new NettyClientReplies(); + final ClientInvocationId clientInvocationId = + ClientInvocationId.valueOf(ClientId.randomId(), 1L); + + assertNull(replies.getReplyMap(clientInvocationId)); + + final NettyClientReplies.ReplyMap created = replies.getOrCreateReplyMap(clientInvocationId); + assertNotNull(created); + assertSame(created, replies.getReplyMap(clientInvocationId)); + + final ClientInvocationId other = + ClientInvocationId.valueOf(ClientId.randomId(), 2L); + assertNull(replies.getReplyMap(other)); + } +} From f6f3c76b3f0c95cde3105f3b05466f87fab413ad Mon Sep 17 00:00:00 2001 From: slfan1989 <55643692+slfan1989@users.noreply.github.com> Date: Wed, 25 Feb 2026 02:22:24 +0800 Subject: [PATCH 264/397] RATIS-2417. Fix ChannelMap memory leak in DataStreamManagement when read() fails early. (#1358) --- .../apache/ratis/netty/server/ChannelMap.java | 6 + .../netty/server/DataStreamManagement.java | 18 ++- .../server/TestDataStreamManagement.java | 114 ++++++++++++++++++ 3 files changed, 133 insertions(+), 5 deletions(-) create mode 100644 ratis-test/src/test/java/org/apache/ratis/netty/server/TestDataStreamManagement.java diff --git a/ratis-netty/src/main/java/org/apache/ratis/netty/server/ChannelMap.java b/ratis-netty/src/main/java/org/apache/ratis/netty/server/ChannelMap.java index 7b0d761840..4edf878dfa 100644 --- a/ratis-netty/src/main/java/org/apache/ratis/netty/server/ChannelMap.java +++ b/ratis-netty/src/main/java/org/apache/ratis/netty/server/ChannelMap.java @@ -40,6 +40,12 @@ void remove(ChannelId channelId, ClientInvocationId clientInvocationId) { .ifPresent((ids) -> ids.remove(clientInvocationId)); } + int size(ChannelId channelId) { + return Optional.ofNullable(map.get(channelId)) + .map(Map::size) + .orElse(0); + } + Set remove(ChannelId channelId) { return Optional.ofNullable(map.remove(channelId)) .map(Map::keySet) diff --git a/ratis-netty/src/main/java/org/apache/ratis/netty/server/DataStreamManagement.java b/ratis-netty/src/main/java/org/apache/ratis/netty/server/DataStreamManagement.java index 127ed09835..0e10b0f4dc 100644 --- a/ratis-netty/src/main/java/org/apache/ratis/netty/server/DataStreamManagement.java +++ b/ratis-netty/src/main/java/org/apache/ratis/netty/server/DataStreamManagement.java @@ -51,6 +51,7 @@ import org.apache.ratis.statemachine.StateMachine; import org.apache.ratis.statemachine.StateMachine.DataStream; import org.apache.ratis.statemachine.StateMachine.DataChannel; +import org.apache.ratis.thirdparty.com.google.common.annotations.VisibleForTesting; import org.apache.ratis.thirdparty.io.netty.buffer.ByteBuf; import org.apache.ratis.thirdparty.io.netty.channel.ChannelHandlerContext; import org.apache.ratis.thirdparty.io.netty.channel.ChannelId; @@ -411,14 +412,22 @@ void cleanUpOnChannelInactive(ChannelId channelId, TimeDuration channelInactiveG }); } + @VisibleForTesting + int getChannelInvocationCount(ChannelId channelId) { + return channels.size(channelId); + } + void read(DataStreamRequestByteBuf request, ChannelHandlerContext ctx, CheckedBiFunction, Set, IOException> getStreams) { LOG.debug("{}: read {}", this, request); + final ClientInvocationId key = ClientInvocationId.valueOf(request.getClientId(), request.getStreamId()); + final ChannelId channelId = ctx.channel().id(); try { - readImpl(request, ctx, getStreams); + readImpl(request, ctx, getStreams, key, channelId); } catch (Throwable t) { replyDataStreamException(t, request, ctx); - removeDataStream(ClientInvocationId.valueOf(request.getClientId(), request.getStreamId())); + removeDataStream(key); + channels.remove(channelId, key); } } @@ -431,12 +440,11 @@ private StreamInfo removeDataStream(ClientInvocationId invocationId) { } private void readImpl(DataStreamRequestByteBuf request, ChannelHandlerContext ctx, - CheckedBiFunction, Set, IOException> getStreams) { + CheckedBiFunction, Set, IOException> getStreams, + ClientInvocationId key, ChannelId channelId) { final boolean close = request.getWriteOptionList().contains(StandardWriteOption.CLOSE); - ClientInvocationId key = ClientInvocationId.valueOf(request.getClientId(), request.getStreamId()); // add to ChannelMap - final ChannelId channelId = ctx.channel().id(); channels.add(channelId, key); final StreamInfo info; diff --git a/ratis-test/src/test/java/org/apache/ratis/netty/server/TestDataStreamManagement.java b/ratis-test/src/test/java/org/apache/ratis/netty/server/TestDataStreamManagement.java new file mode 100644 index 0000000000..5c06ddd319 --- /dev/null +++ b/ratis-test/src/test/java/org/apache/ratis/netty/server/TestDataStreamManagement.java @@ -0,0 +1,114 @@ +/* + * 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.ratis.netty.server; + +import org.apache.ratis.client.impl.DataStreamClientImpl.DataStreamOutputImpl; +import org.apache.ratis.conf.RaftProperties; +import org.apache.ratis.datastream.impl.DataStreamRequestByteBuf; +import org.apache.ratis.io.StandardWriteOption; +import org.apache.ratis.netty.metrics.NettyServerStreamRpcMetrics; +import org.apache.ratis.proto.RaftProtos.DataStreamPacketHeaderProto.Type; +import org.apache.ratis.protocol.ClientId; +import org.apache.ratis.protocol.RaftClientRequest; +import org.apache.ratis.protocol.RaftPeer; +import org.apache.ratis.protocol.RaftPeerId; +import org.apache.ratis.server.RaftServer; +import org.apache.ratis.thirdparty.io.netty.buffer.Unpooled; +import org.apache.ratis.thirdparty.io.netty.channel.ChannelHandlerContext; +import org.apache.ratis.thirdparty.io.netty.channel.ChannelId; +import org.apache.ratis.thirdparty.io.netty.channel.ChannelInboundHandlerAdapter; +import org.apache.ratis.thirdparty.io.netty.channel.embedded.EmbeddedChannel; +import org.apache.ratis.util.function.CheckedBiFunction; +import org.junit.jupiter.api.Test; + +import java.io.IOException; +import java.lang.reflect.Proxy; +import java.util.Collections; +import java.util.Set; + +import static org.junit.jupiter.api.Assertions.assertEquals; +import static org.junit.jupiter.api.Assertions.assertNotNull; + +class TestDataStreamManagement { + @Test + void readCleansChannelMapOnEarlyException() throws Exception { + // Scenario: STREAM_DATA arrives without prior STREAM_HEADER, so readImpl fails early. + // Expectation: read(...) catch path must still remove channelId->invocationId mapping + // to avoid leaks when the channel remains active. + final RaftPeerId serverId = RaftPeerId.valueOf("s1"); + final RaftProperties properties = new RaftProperties(); + final RaftServer server = newRaftServer(serverId, properties); + + final NettyServerStreamRpcMetrics metrics = new NettyServerStreamRpcMetrics("s1"); + final DataStreamManagement management = new DataStreamManagement(server, metrics); + + // Use a real Netty pipeline to obtain a concrete ChannelHandlerContext. + final EmbeddedChannel embeddedChannel = new EmbeddedChannel(new ChannelInboundHandlerAdapter()); + final ChannelHandlerContext ctx = embeddedChannel.pipeline().firstContext(); + assertNotNull(ctx, "ChannelHandlerContext should be initialized"); + final ChannelId channelId = embeddedChannel.id(); + + final DataStreamRequestByteBuf request = new DataStreamRequestByteBuf( + ClientId.randomId(), + Type.STREAM_DATA, + 1L, + 0L, + Collections.singletonList(StandardWriteOption.CLOSE), + Unpooled.buffer(0)); + + final CheckedBiFunction, Set, IOException> getStreams = + (r, p) -> Collections.emptySet(); + + try { + // This read should fail early (missing stream info) and must clear ChannelMap entries. + management.read(request, ctx, getStreams); + assertEquals(0, management.getChannelInvocationCount(channelId), + "channel map should be cleared on early read failure"); + } finally { + embeddedChannel.finishAndReleaseAll(); + management.shutdown(); + } + } + + private static RaftServer newRaftServer(RaftPeerId serverId, RaftProperties properties) { + return (RaftServer) Proxy.newProxyInstance(TestDataStreamManagement.class.getClassLoader(), + new Class[]{RaftServer.class}, + (proxy, method, args) -> { + if (method.getDeclaringClass() == Object.class) { + switch (method.getName()) { + case "toString": + return "RaftServerProxy(" + serverId + ")"; + case "hashCode": + return System.identityHashCode(proxy); + case "equals": + return proxy == args[0]; + default: + return null; + } + } + switch (method.getName()) { + case "getId": + return serverId; + case "getProperties": + return properties; + default: + throw new UnsupportedOperationException("Unexpected RaftServer call: " + method); + } + }); + } +} From 5306c471277790546dc55846d1cb99e4835841ec Mon Sep 17 00:00:00 2001 From: "Doroszlai, Attila" <6454655+adoroszlai@users.noreply.github.com> Date: Thu, 26 Feb 2026 06:30:23 +0100 Subject: [PATCH 265/397] =?UTF-8?q?RATIS-2419.=20checkstyle=20should=20not?= =?UTF-8?q?=20check=20generated-sources=C2=A0=20(#1360)?= MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit --- dev-support/checkstyle.xml | 4 ++++ ratis-proto/pom.xml | 5 ----- 2 files changed, 4 insertions(+), 5 deletions(-) diff --git a/dev-support/checkstyle.xml b/dev-support/checkstyle.xml index 6f8ac9f96f..db4954fb49 100644 --- a/dev-support/checkstyle.xml +++ b/dev-support/checkstyle.xml @@ -55,6 +55,10 @@ + + + + diff --git a/ratis-proto/pom.xml b/ratis-proto/pom.xml index e189685692..9eb9229603 100644 --- a/ratis-proto/pom.xml +++ b/ratis-proto/pom.xml @@ -52,11 +52,6 @@ false - - org.apache.maven.plugins - maven-checkstyle-plugin - - org.apache.maven.plugins From 122ae754aa987bf63b1a7a72da366cdfb7ee8c0c Mon Sep 17 00:00:00 2001 From: slfan1989 <55643692+slfan1989@users.noreply.github.com> Date: Fri, 27 Feb 2026 02:20:21 +0800 Subject: [PATCH 266/397] RATIS-2418. NettyClient LoggingHandler prints too many messages. (#1359) --- .../org/apache/ratis/netty/NettyRpcProxy.java | 43 ++++++++++++++----- .../java/org/apache/ratis/RaftBasicTests.java | 29 +++++++------ .../apache/ratis/netty/TestNettyRpcProxy.java | 13 +++++- .../ratis/netty/TestRaftAsyncWithNetty.java | 8 ++++ 4 files changed, 67 insertions(+), 26 deletions(-) diff --git a/ratis-netty/src/main/java/org/apache/ratis/netty/NettyRpcProxy.java b/ratis-netty/src/main/java/org/apache/ratis/netty/NettyRpcProxy.java index cfcabc2741..e72d6c6772 100644 --- a/ratis-netty/src/main/java/org/apache/ratis/netty/NettyRpcProxy.java +++ b/ratis-netty/src/main/java/org/apache/ratis/netty/NettyRpcProxy.java @@ -45,7 +45,6 @@ import java.io.Closeable; import java.io.IOException; import java.util.Map; -import java.util.Objects; import java.util.concurrent.CompletableFuture; import java.util.concurrent.ConcurrentHashMap; import java.util.concurrent.ExecutionException; @@ -156,7 +155,15 @@ class Connection implements Closeable { @Override protected void channelRead0(ChannelHandlerContext ctx, RaftNettyServerReplyProto proto) { - final CompletableFuture future = getReplyFuture(getCallId(proto), null); + final long callId = getCallId(proto); + final CompletableFuture future = getReplyFuture(callId, null, "reply"); + if (future == null) { + if (LOG.isDebugEnabled()) { + LOG.debug("Ignoring reply for callId={} from {} (no outstanding request, outstanding={})", + callId, peer, replies.size()); + } + return; + } if (proto.getRaftNettyServerReplyCase() == EXCEPTIONREPLY) { final Object ioe = ProtoUtils.toObject(proto.getExceptionReply().getException()); future.completeExceptionally((IOException)ioe); @@ -183,7 +190,8 @@ public void channelInactive(ChannelHandlerContext ctx) throws Exception { protected void initChannel(SocketChannel ch) { final ChannelPipeline p = ch.pipeline(); - p.addLast(new LoggingHandler(LogLevel.WARN)); + // LoggingHandler emits all events at the chosen level; use DEBUG to reduce noise by default. + p.addLast(new LoggingHandler(LogLevel.DEBUG)); p.addLast(new ProtobufVarint32FrameDecoder()); p.addLast(new ProtobufDecoder(RaftNettyServerReplyProto.getDefaultInstance())); p.addLast(new ProtobufVarint32LengthFieldPrepender()); @@ -197,9 +205,12 @@ protected void initChannel(SocketChannel ch) { } private CompletableFuture getReplyFuture(long callId, - CompletableFuture expected) { + CompletableFuture expected, String reason) { final CompletableFuture removed = replies.remove(callId); - Objects.requireNonNull(removed, () -> "Request #" + callId + " not found"); + if (removed == null && LOG.isDebugEnabled()) { + LOG.debug("Request {} not found for callId={} from {} (reason={}, outstanding={})", + expected == null ? "future" : "reply", callId, peer, reason, replies.size()); + } if (expected != null) { Preconditions.assertSame(expected, removed, "removed"); } @@ -207,23 +218,30 @@ private CompletableFuture getReplyFuture(long callId, } synchronized CompletableFuture offer(RaftNettyServerRequestProto request) { + final CompletableFuture reply = new CompletableFuture<>(); + final long callId = getRequest(request).getCallId(); + final CompletableFuture previous = replies.put(callId, reply); + Preconditions.assertNull(previous, "previous"); + final ChannelFuture future; try { future = client.writeAndFlush(request); } catch (AlreadyClosedException e) { + replies.remove(callId, reply); return JavaUtils.completeExceptionally(e); } - final CompletableFuture reply = new CompletableFuture<>(); - final long callId = getRequest(request).getCallId(); - final CompletableFuture previous = replies.put(callId, reply); - Preconditions.assertNull(previous, "previous"); - future.addListener(cf -> { if (!cf.isSuccess()) { // Remove from queue on async write failure to prevent reply mismatch. // Only complete exceptionally if removal succeeds (not already polled). - getReplyFuture(callId, reply).completeExceptionally(cf.cause()); + final CompletableFuture removed = + getReplyFuture(callId, reply, "write-failure"); + if (removed != null) { + removed.completeExceptionally(cf.cause()); + } else if (LOG.isDebugEnabled()) { + LOG.debug("Write failed for callId={} to {} after request removed", callId, peer, cf.cause()); + } client.close(); } }); @@ -240,6 +258,9 @@ private synchronized void failOutstandingRequests(Throwable cause) { if (!replies.isEmpty()) { LOG.warn("Still have {} requests outstanding from {} connection: {}", replies.size(), peer, cause.toString()); + if (LOG.isDebugEnabled()) { + LOG.debug("Outstanding request ids from {}: {}", peer, replies.keySet()); + } replies.values().forEach(f -> f.completeExceptionally(cause)); replies.clear(); } diff --git a/ratis-server/src/test/java/org/apache/ratis/RaftBasicTests.java b/ratis-server/src/test/java/org/apache/ratis/RaftBasicTests.java index f1319cde7a..c71b57e826 100644 --- a/ratis-server/src/test/java/org/apache/ratis/RaftBasicTests.java +++ b/ratis-server/src/test/java/org/apache/ratis/RaftBasicTests.java @@ -318,10 +318,10 @@ public void run() { } } catch(Exception t) { if (exceptionInClientThread.compareAndSet(null, t)) { - log.error(this + " failed", t); + log.error("{} failed", this, t); } else { exceptionInClientThread.get().addSuppressed(t); - log.error(this + " failed again!", t); + log.error("{} failed again!", this, t); } } finally { isRunning.set(false); @@ -347,8 +347,8 @@ public void testWithLoad() throws Exception { static void testWithLoad(final int numClients, final int numMessages, boolean useAsync, MiniRaftCluster cluster, Logger log) throws Exception { - log.info("Running testWithLoad: numClients=" + numClients - + ", numMessages=" + numMessages + ", async=" + useAsync); + log.info("Running testWithLoad: numClients={}, numMessages={}, async={}", + numClients, numMessages, useAsync); waitForLeader(cluster); @@ -364,24 +364,25 @@ static void testWithLoad(final int numClients, final int numMessages, @Override public void run() { - log.info(cluster.printServers()); - log.info(BlockRequestHandlingInjection.getInstance().toString()); - log.info(cluster.toString()); - clients.forEach(c -> log.info(" " + c)); - JavaUtils.dumpAllThreads(s -> log.info(s)); - final int last = lastStep.get(); if (last != previousLastStep.get()) { previousLastStep.set(last); } else { + // Only dump cluster/client state when no progress is detected to reduce log noise. + log.info(cluster.printServers()); + log.info(BlockRequestHandlingInjection.getInstance().toString()); + log.info(cluster.toString()); + clients.forEach(c -> log.info(" {}", c)); + JavaUtils.dumpAllThreads(s -> log.info(s)); + final RaftServer.Division leader = cluster.getLeader(); - log.info("NO PROGRESS at " + last + ", try to restart leader=" + leader); + log.info("NO PROGRESS at {}, try to restart leader={}", last, leader); if (leader != null) { try { cluster.restartServer(leader.getId(), false); - log.info("Restarted leader=" + leader); + log.info("Restarted leader={}", leader); } catch (IOException e) { - log.error("Failed to restart leader=" + leader); + log.error("Failed to restart leader={}", leader); } } } @@ -415,7 +416,7 @@ public void run() { log.error("Failed to change leader ", e); } } - log.info("Leader change count=" + count); + log.info("Leader change count={}", count); timer.cancel(); for(Client4TestWithLoad c : clients) { diff --git a/ratis-test/src/test/java/org/apache/ratis/netty/TestNettyRpcProxy.java b/ratis-test/src/test/java/org/apache/ratis/netty/TestNettyRpcProxy.java index 780c249b6b..75d471061d 100644 --- a/ratis-test/src/test/java/org/apache/ratis/netty/TestNettyRpcProxy.java +++ b/ratis-test/src/test/java/org/apache/ratis/netty/TestNettyRpcProxy.java @@ -21,6 +21,8 @@ import org.apache.ratis.conf.RaftProperties; import org.apache.ratis.protocol.RaftPeer; import org.apache.ratis.protocol.exceptions.AlreadyClosedException; +import org.apache.ratis.proto.RaftProtos.GroupListRequestProto; +import org.apache.ratis.proto.RaftProtos.RaftRpcRequestProto; import org.apache.ratis.proto.netty.NettyProtos.RaftNettyServerReplyProto; import org.apache.ratis.proto.netty.NettyProtos.RaftNettyServerRequestProto; import org.apache.ratis.thirdparty.io.netty.bootstrap.ServerBootstrap; @@ -78,8 +80,17 @@ protected void channelRead0(ChannelHandlerContext ctx, Object msg) { // Close to force AlreadyClosedException on write and trigger rollback logic. proxy.close(); + final RaftRpcRequestProto rpcRequest = RaftRpcRequestProto.newBuilder() + .setCallId(1) + .build(); + final GroupListRequestProto groupListRequest = GroupListRequestProto.newBuilder() + .setRpcRequest(rpcRequest) + .build(); + final RaftNettyServerRequestProto request = RaftNettyServerRequestProto.newBuilder() + .setGroupListRequest(groupListRequest) + .build(); final CompletableFuture reply = - proxy.sendAsync(RaftNettyServerRequestProto.getDefaultInstance()); + proxy.sendAsync(request); // Ensure the future completes exceptionally with AlreadyClosedException. final Throwable thrown = assertThrows(CompletionException.class, reply::join); diff --git a/ratis-test/src/test/java/org/apache/ratis/netty/TestRaftAsyncWithNetty.java b/ratis-test/src/test/java/org/apache/ratis/netty/TestRaftAsyncWithNetty.java index d2f23e1439..c3e19a26e5 100644 --- a/ratis-test/src/test/java/org/apache/ratis/netty/TestRaftAsyncWithNetty.java +++ b/ratis-test/src/test/java/org/apache/ratis/netty/TestRaftAsyncWithNetty.java @@ -18,13 +18,21 @@ package org.apache.ratis.netty; import org.apache.ratis.RaftAsyncTests; +import org.apache.ratis.client.RaftClient; +import org.apache.ratis.server.RaftServer; +import org.apache.ratis.util.Slf4jUtils; import org.junit.jupiter.api.Test; import org.junit.jupiter.api.Timeout; +import org.slf4j.event.Level; @Timeout(100) public class TestRaftAsyncWithNetty extends RaftAsyncTests implements MiniRaftClusterWithNetty.FactoryGet { + { + Slf4jUtils.setLogLevel(RaftServer.Division.LOG, Level.INFO); + Slf4jUtils.setLogLevel(RaftClient.LOG, Level.INFO); + } @Override @Test From 26f80a3546f579cb3af532ca0a86723ab5783100 Mon Sep 17 00:00:00 2001 From: Snehasish Roy Date: Tue, 3 Mar 2026 23:46:01 +0530 Subject: [PATCH 267/397] RATIS-2420: Fix CT_CONSTRUCTOR_THROW in AtomicFileOutputStream (#1361) --- .../org/apache/ratis/client/api/SnapshotManagementApi.java | 2 +- ratis-common/dev-support/findbugsExcludeFile.xml | 4 ---- .../java/org/apache/ratis/util/AtomicFileOutputStream.java | 2 +- 3 files changed, 2 insertions(+), 6 deletions(-) diff --git a/ratis-client/src/main/java/org/apache/ratis/client/api/SnapshotManagementApi.java b/ratis-client/src/main/java/org/apache/ratis/client/api/SnapshotManagementApi.java index f83d976040..359763fe90 100644 --- a/ratis-client/src/main/java/org/apache/ratis/client/api/SnapshotManagementApi.java +++ b/ratis-client/src/main/java/org/apache/ratis/client/api/SnapshotManagementApi.java @@ -40,7 +40,7 @@ default RaftClientReply create(boolean force, long timeoutMs) throws IOException /** * Trigger to create a snapshot. * - * @param creationGap When (creationGap > 0) and (astAppliedIndex - lastSnapshotIndex < creationGap), + * @param creationGap When (creationGap > 0) and (lastAppliedIndex - lastSnapshotIndex < creationGap), * return lastSnapshotIndex; otherwise, take a new snapshot and then return its index. * When creationGap == 0, use the server configured value as the creationGap. * @return a reply. When {@link RaftClientReply#isSuccess()} is true, diff --git a/ratis-common/dev-support/findbugsExcludeFile.xml b/ratis-common/dev-support/findbugsExcludeFile.xml index 787621f17f..882f08b7fa 100644 --- a/ratis-common/dev-support/findbugsExcludeFile.xml +++ b/ratis-common/dev-support/findbugsExcludeFile.xml @@ -23,10 +23,6 @@ - - - - diff --git a/ratis-common/src/main/java/org/apache/ratis/util/AtomicFileOutputStream.java b/ratis-common/src/main/java/org/apache/ratis/util/AtomicFileOutputStream.java index 3961d7336b..ec0eda94f5 100644 --- a/ratis-common/src/main/java/org/apache/ratis/util/AtomicFileOutputStream.java +++ b/ratis-common/src/main/java/org/apache/ratis/util/AtomicFileOutputStream.java @@ -45,7 +45,7 @@ * NOTE that on Windows platforms, the output file, if it exists, is deleted * before the temporary file is moved. */ -public class AtomicFileOutputStream extends FilterOutputStream { +public final class AtomicFileOutputStream extends FilterOutputStream { static final Logger LOG = LoggerFactory.getLogger(AtomicFileOutputStream.class); public static final String TMP_EXTENSION = ".tmp"; From 5055b543cc88545201a524e1768db7bdbbb782a8 Mon Sep 17 00:00:00 2001 From: "Doroszlai, Attila" <6454655+adoroszlai@users.noreply.github.com> Date: Fri, 6 Mar 2026 18:11:25 +0100 Subject: [PATCH 268/397] RATIS-2422. TestNettyRpcProxy.testOfferRollbackOnAlreadyClosed fails with `Host is null` (#1364) --- .../src/test/java/org/apache/ratis/netty/TestNettyRpcProxy.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/ratis-test/src/test/java/org/apache/ratis/netty/TestNettyRpcProxy.java b/ratis-test/src/test/java/org/apache/ratis/netty/TestNettyRpcProxy.java index 75d471061d..2507220a8e 100644 --- a/ratis-test/src/test/java/org/apache/ratis/netty/TestNettyRpcProxy.java +++ b/ratis-test/src/test/java/org/apache/ratis/netty/TestNettyRpcProxy.java @@ -69,7 +69,7 @@ protected void channelRead0(ChannelHandlerContext ctx, Object msg) { }); } }) - .bind(0) + .bind("localhost", 0) .sync(); serverChannel = bindFuture.channel(); From 3319c53c852dd66f56400b657a60b264de788a31 Mon Sep 17 00:00:00 2001 From: Potato Date: Sat, 7 Mar 2026 14:35:17 +0800 Subject: [PATCH 269/397] RATIS-2423. Javadoc should be generated in English regardless of build locale (#1365) --- pom.xml | 2 ++ 1 file changed, 2 insertions(+) diff --git a/pom.xml b/pom.xml index 96f07ab012..c6ded6afa4 100644 --- a/pom.xml +++ b/pom.xml @@ -442,6 +442,8 @@ -Xmaxwarns 10000 -Xdoclint:-missing + -locale + en_US From de72ce5766c92ec955d2b8aab9249ccc47d2aa6d Mon Sep 17 00:00:00 2001 From: Potato Date: Mon, 9 Mar 2026 00:59:06 +0800 Subject: [PATCH 270/397] RATIS-2424. Fix javadoc locale configuration for JDK 8 compatibility (#1366) --- pom.xml | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/pom.xml b/pom.xml index c6ded6afa4..240f558b38 100644 --- a/pom.xml +++ b/pom.xml @@ -439,11 +439,11 @@ 8 + -J-Duser.language=en + -J-Duser.country=US -Xmaxwarns 10000 -Xdoclint:-missing - -locale - en_US From a2e130efa4eb42aee00cf4d6eb8797c1746257c3 Mon Sep 17 00:00:00 2001 From: "Doroszlai, Attila" <6454655+adoroszlai@users.noreply.github.com> Date: Sun, 8 Mar 2026 18:20:26 +0100 Subject: [PATCH 271/397] RATIS-2425. make_rc.sh does not set version in ratis-bom (#1367) --- dev-support/make_rc.sh | 6 +++++- 1 file changed, 5 insertions(+), 1 deletion(-) diff --git a/dev-support/make_rc.sh b/dev-support/make_rc.sh index b94305a054..551508ecfe 100755 --- a/dev-support/make_rc.sh +++ b/dev-support/make_rc.sh @@ -95,7 +95,11 @@ mvnFun() { cd "$projectdir" git reset --hard git clean -fdx - mvnFun versions:set -DnewVersion="$RATISVERSION" + mvnFun versions:set -DnewVersion="$RATISVERSION" -DprocessAllModules + if ! git diff --name-only | grep -q ratis-bom/pom.xml; then + echo 'ratis-bom not updated by `mvn versions:set`' >&2 + exit 1 + fi git commit --allow-empty -a -m "Change version for the version $RATISVERSION $RC" git config user.signingkey "${CODESIGNINGKEY}" From d44d5009a6dff34faaf32d2e188506a39742dfb4 Mon Sep 17 00:00:00 2001 From: Potato Date: Tue, 10 Mar 2026 21:43:27 +0800 Subject: [PATCH 272/397] RATIS-2429. ratis-bom module fails to deploy due to missing distributionManagement configuration (#1371) --- ratis-bom/pom.xml | 22 +++++++++++++++++++++- 1 file changed, 21 insertions(+), 1 deletion(-) diff --git a/ratis-bom/pom.xml b/ratis-bom/pom.xml index c023790cf5..a62f2798db 100644 --- a/ratis-bom/pom.xml +++ b/ratis-bom/pom.xml @@ -22,8 +22,28 @@ pom + + + ${distMgmtStagingId} + ${distMgmtStagingName} + ${distMgmtStagingUrl} + + + ${distMgmtSnapshotsId} + ${distMgmtSnapshotsName} + ${distMgmtSnapshotsUrl} + + + - 1.0.10 + apache.snapshots.https + Apache Development Snapshot Repository + https://repository.apache.org/content/repositories/snapshots + apache.staging.https + Apache Release Distribution Repository + https://repository.apache.org/service/local/staging/deploy/maven2 + + 1.0.11 true From cbddb5fa6845a7819c626b69178a7bb8b68ecc42 Mon Sep 17 00:00:00 2001 From: "Tak Lon (Stephen) Wu" Date: Thu, 12 Mar 2026 13:47:50 +0800 Subject: [PATCH 273/397] RATIS-2393 Add Span Context to RaftRpcRequestProto (#1341) --- pom.xml | 31 +++ .../ratis/client/impl/ClientProtoUtils.java | 4 + ratis-common/pom.xml | 21 ++ .../ratis/protocol/RaftClientRequest.java | 14 ++ .../apache/ratis/trace/RatisAttributes.java | 35 ++++ .../apache/ratis/trace/TraceConfigKeys.java | 45 +++++ .../org/apache/ratis/trace/TraceUtils.java | 187 ++++++++++++++++++ .../org/apache/ratis/util/VersionInfo.java | 8 + ratis-proto/src/main/proto/Raft.proto | 6 + .../ratis/server/impl/RaftServerImpl.java | 13 ++ .../impl/RaftServerImplTracingTests.java | 141 +++++++++++++ 11 files changed, 505 insertions(+) create mode 100644 ratis-common/src/main/java/org/apache/ratis/trace/RatisAttributes.java create mode 100644 ratis-common/src/main/java/org/apache/ratis/trace/TraceConfigKeys.java create mode 100644 ratis-common/src/main/java/org/apache/ratis/trace/TraceUtils.java create mode 100644 ratis-server/src/test/java/org/apache/ratis/server/impl/RaftServerImplTracingTests.java diff --git a/pom.xml b/pom.xml index 240f558b38..5c098c311e 100644 --- a/pom.xml +++ b/pom.xml @@ -189,6 +189,9 @@ 3.25.8 1.77.1 + + 1.59.0 + 1.40.0 true _ @@ -398,7 +401,35 @@ jakarta.annotation-api ${jakarta.annotation.version} + + + + io.opentelemetry + opentelemetry-api + ${opentelemetry.version} + + + io.opentelemetry + opentelemetry-sdk + ${opentelemetry.version} + + + io.opentelemetry + opentelemetry-sdk-testing + ${opentelemetry.version} + + + io.opentelemetry.semconv + opentelemetry-semconv + ${opentelemetry-semconv.version} + + + io.opentelemetry + opentelemetry-context + ${opentelemetry.version} + + diff --git a/ratis-client/src/main/java/org/apache/ratis/client/impl/ClientProtoUtils.java b/ratis-client/src/main/java/org/apache/ratis/client/impl/ClientProtoUtils.java index 36c0b3937f..d2146a521f 100644 --- a/ratis-client/src/main/java/org/apache/ratis/client/impl/ClientProtoUtils.java +++ b/ratis-client/src/main/java/org/apache/ratis/client/impl/ClientProtoUtils.java @@ -128,6 +128,7 @@ static RaftRpcRequestProto.Builder toRaftRpcRequestProtoBuilder(RaftClientReques Optional.ofNullable(request.getSlidingWindowEntry()).ifPresent(b::setSlidingWindowEntry); Optional.ofNullable(request.getRoutingTable()).map(RoutingTable::toProto).ifPresent(b::setRoutingTable); + Optional.ofNullable(request.getSpanContext()).ifPresent(b::setSpanContext); return b.setCallId(request.getCallId()) .setToLeader(request.isToLeader()) @@ -188,6 +189,9 @@ static RaftClientRequest toRaftClientRequest(RaftClientRequestProto p) { if (request.hasSlidingWindowEntry()) { b.setSlidingWindowEntry(request.getSlidingWindowEntry()); } + if (request.hasSpanContext()) { + b.setSpanContext(request.getSpanContext()); + } return b.setClientId(ClientId.valueOf(request.getRequestorId())) .setGroupId(ProtoUtils.toRaftGroupId(request.getRaftGroupId())) .setCallId(request.getCallId()) diff --git a/ratis-common/pom.xml b/ratis-common/pom.xml index f6bc0ee413..ba19c73e33 100644 --- a/ratis-common/pom.xml +++ b/ratis-common/pom.xml @@ -38,6 +38,27 @@ slf4j-api + + io.opentelemetry + opentelemetry-api + + + io.opentelemetry + opentelemetry-context + + + io.opentelemetry + opentelemetry-sdk + + + io.opentelemetry + opentelemetry-sdk-testing + + + io.opentelemetry.semconv + opentelemetry-semconv + + org.junit.jupiter junit-jupiter-api diff --git a/ratis-common/src/main/java/org/apache/ratis/protocol/RaftClientRequest.java b/ratis-common/src/main/java/org/apache/ratis/protocol/RaftClientRequest.java index ed41f1ea2c..b04402fe15 100644 --- a/ratis-common/src/main/java/org/apache/ratis/protocol/RaftClientRequest.java +++ b/ratis-common/src/main/java/org/apache/ratis/protocol/RaftClientRequest.java @@ -24,6 +24,7 @@ import org.apache.ratis.proto.RaftProtos.ReadRequestTypeProto; import org.apache.ratis.proto.RaftProtos.ReplicationLevel; import org.apache.ratis.proto.RaftProtos.SlidingWindowEntry; +import org.apache.ratis.proto.RaftProtos.SpanContextProto; import org.apache.ratis.proto.RaftProtos.StaleReadRequestTypeProto; import org.apache.ratis.proto.RaftProtos.WatchRequestTypeProto; import org.apache.ratis.proto.RaftProtos.WriteRequestTypeProto; @@ -305,6 +306,7 @@ public static class Builder { private SlidingWindowEntry slidingWindowEntry; private RoutingTable routingTable; private long timeoutMs; + private SpanContextProto spanContext; public RaftClientRequest build() { return new RaftClientRequest(this); @@ -366,6 +368,11 @@ public Builder setTimeoutMs(long timeoutMs) { this.timeoutMs = timeoutMs; return this; } + + public Builder setSpanContext(SpanContextProto spanContext) { + this.spanContext = spanContext; + return this; + } } public static Builder newBuilder() { @@ -397,6 +404,8 @@ public static RaftClientRequest toWriteRequest(RaftClientRequest r, Message mess private final boolean toLeader; + private final SpanContextProto spanContext; + /** Construct a request for sending to the given server. */ protected RaftClientRequest(ClientId clientId, RaftPeerId serverId, RaftGroupId groupId, long callId, Type type) { this(newBuilder() @@ -429,6 +438,7 @@ private RaftClientRequest(Builder b) { this.slidingWindowEntry = b.slidingWindowEntry; this.routingTable = b.routingTable; this.timeoutMs = b.timeoutMs; + this.spanContext = b.spanContext; } @Override @@ -472,6 +482,10 @@ public long getTimeoutMs() { return timeoutMs; } + public SpanContextProto getSpanContext() { + return spanContext; + } + @Override public String toString() { return super.toString() + ", seq=" + ProtoUtils.toString(slidingWindowEntry) + ", " diff --git a/ratis-common/src/main/java/org/apache/ratis/trace/RatisAttributes.java b/ratis-common/src/main/java/org/apache/ratis/trace/RatisAttributes.java new file mode 100644 index 0000000000..d74c63757f --- /dev/null +++ b/ratis-common/src/main/java/org/apache/ratis/trace/RatisAttributes.java @@ -0,0 +1,35 @@ +/* + * 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.ratis.trace; + +import io.opentelemetry.api.common.AttributeKey; + +/** + * The constants in this class correspond with the guidance outlined by the OpenTelemetry Semantic + * Conventions. + */ +public final class RatisAttributes { + public static final AttributeKey CLIENT_ID = AttributeKey.stringKey("raft.client.id"); + public static final AttributeKey MEMBER_ID = AttributeKey.stringKey("raft.member.id"); + public static final AttributeKey CALL_ID = AttributeKey.stringKey("raft.call.id"); + + + private RatisAttributes() { + } +} diff --git a/ratis-common/src/main/java/org/apache/ratis/trace/TraceConfigKeys.java b/ratis-common/src/main/java/org/apache/ratis/trace/TraceConfigKeys.java new file mode 100644 index 0000000000..b0a1cbd9b1 --- /dev/null +++ b/ratis-common/src/main/java/org/apache/ratis/trace/TraceConfigKeys.java @@ -0,0 +1,45 @@ +/** + * 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.ratis.trace; + +import org.apache.ratis.conf.RaftProperties; + +import java.util.function.Consumer; + +import static org.apache.ratis.conf.ConfUtils.getBoolean; +import static org.apache.ratis.conf.ConfUtils.setBoolean; + +public interface TraceConfigKeys { + String PREFIX = "raft.otel.tracing"; + + String ENABLED_KEY = PREFIX + ".enabled"; + boolean ENABLED_DEFAULT = false; + + static boolean enabled(RaftProperties properties, Consumer logger) { + return getBoolean(properties::getBoolean, ENABLED_KEY, ENABLED_DEFAULT, logger); + } + + static boolean enabled(RaftProperties properties) { + return enabled(properties, null); + } + + static void setEnabled(RaftProperties properties, boolean enabled) { + setBoolean(properties::setBoolean, ENABLED_KEY, enabled); + } +} + diff --git a/ratis-common/src/main/java/org/apache/ratis/trace/TraceUtils.java b/ratis-common/src/main/java/org/apache/ratis/trace/TraceUtils.java new file mode 100644 index 0000000000..bb10e24553 --- /dev/null +++ b/ratis-common/src/main/java/org/apache/ratis/trace/TraceUtils.java @@ -0,0 +1,187 @@ +/* + * 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.ratis.trace; + +import io.opentelemetry.api.GlobalOpenTelemetry; +import io.opentelemetry.api.trace.Span; +import io.opentelemetry.api.trace.SpanKind; +import io.opentelemetry.api.trace.StatusCode; +import io.opentelemetry.api.trace.Tracer; +import io.opentelemetry.context.Context; +import io.opentelemetry.context.Scope; +import io.opentelemetry.context.propagation.TextMapPropagator; +import io.opentelemetry.context.propagation.TextMapGetter; +import org.apache.ratis.proto.RaftProtos.SpanContextProto; +import org.apache.ratis.protocol.RaftClientRequest; +import org.apache.ratis.util.JavaUtils; +import org.apache.ratis.util.function.CheckedSupplier; +import org.apache.ratis.util.VersionInfo; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.util.Map; +import java.util.Optional; +import java.util.TreeMap; +import java.util.concurrent.CompletableFuture; +import java.util.function.BiConsumer; +import java.util.function.Supplier; + +public final class TraceUtils { + + private static final Tracer TRACER = GlobalOpenTelemetry.getTracer("org.apache.ratis", + VersionInfo.getSoftwareInfoVersion()); + + private static final Logger LOG = LoggerFactory.getLogger(TraceUtils.class); + + private TraceUtils() { + } + + public static Tracer getGlobalTracer() { + return TRACER; + } + + /** + * Trace an asynchronous operation represented by a {@link CompletableFuture}. + * The returned future will complete with the same result or error as the original future, + * but the provided {@code span} will be ended when the future completes. + */ + static CompletableFuture traceAsyncMethod( + CheckedSupplier, THROWABLE> action, Supplier spanSupplier) throws THROWABLE { + final Span span = spanSupplier.get(); + try (Scope ignored = span.makeCurrent()) { + final CompletableFuture future; + try { + future = action.get(); + } catch (RuntimeException | Error e) { + setError(span, e); + span.end(); + throw e; + } catch (Throwable t) { + setError(span, t); + span.end(); + throw JavaUtils.cast(t); + } + endSpan(future, span); + return future; + } + } + + public static CompletableFuture traceAsyncMethod( + CheckedSupplier, THROWABLE> action, + RaftClientRequest request, String memberId, String spanName) throws THROWABLE { + return traceAsyncMethod(action, () -> createServerSpanFromClientRequest(request, memberId, spanName)); + } + + public static CompletableFuture traceAsyncMethodIfEnabled( + boolean enabled, + CheckedSupplier, THROWABLE> action, + RaftClientRequest request, String memberId, String spanName) throws THROWABLE { + return enabled ? traceAsyncMethod(action, request, memberId, spanName) : action.get(); + } + + private static Span createServerSpanFromClientRequest(RaftClientRequest request, String memberId, String spanName) { + final Context remoteContext = extractContextFromProto(request.getSpanContext()); + final Span span = getGlobalTracer() + .spanBuilder(spanName) + .setParent(remoteContext) + .setSpanKind(SpanKind.SERVER) + .startSpan(); + span.setAttribute(RatisAttributes.CLIENT_ID, String.valueOf(request.getClientId())); + span.setAttribute(RatisAttributes.CALL_ID, String.valueOf(request.getCallId())); + span.setAttribute(RatisAttributes.MEMBER_ID, memberId); + return span; + } + + private static void endSpan(CompletableFuture future, Span span) { + addListener(future, (resp, error) -> { + if (error != null) { + setError(span, error); + } else { + span.setStatus(StatusCode.OK); + } + span.end(); + }); + } + + public static void setError(Span span, Throwable error) { + span.recordException(error); + span.setStatus(StatusCode.ERROR); + } + + /** + * This is method is used when you just want to add a listener to the given future. We will call + * {@link CompletableFuture#whenComplete(BiConsumer)} to register the {@code action} to the + * {@code future}. Ignoring the return value of a Future is considered as a bad practice as it may + * suppress exceptions thrown from the code that completes the future, and this method will catch + * all the exception thrown from the {@code action} to catch possible code bugs. + *

+ * And the error phone check will always report FutureReturnValueIgnored because every method in + * the {@link CompletableFuture} class will return a new {@link CompletableFuture}, so you always + * have one future that has not been checked. So we introduce this method and add a suppression + * warnings annotation here. + */ + @SuppressWarnings("FutureReturnValueIgnored") + private static void addListener(CompletableFuture future, + BiConsumer action) { + future.whenComplete((resp, error) -> { + try { + // See this post on stack overflow(shorten since the url is too long), + // https://s.apache.org/completionexception + // For a chain of CompletableFuture, only the first child CompletableFuture can get the + // original exception, others will get a CompletionException, which wraps the original + // exception. So here we unwrap it before passing it to the callback action. + action.accept(resp, JavaUtils.unwrapCompletionException(error)); + } catch (Throwable t) { + LOG.error("Unexpected error caught when processing CompletableFuture", t); + } + }); + } + + private static final TextMapPropagator PROPAGATOR = + GlobalOpenTelemetry.getPropagators().getTextMapPropagator(); + + public static SpanContextProto injectContextToProto(Context context) { + Map carrier = new TreeMap<>(); + PROPAGATOR.inject(context, carrier, (map, key, value) -> map.put(key, value)); + return SpanContextProto.newBuilder().putAllContext(carrier).build(); + } + + public static Context extractContextFromProto(SpanContextProto proto) { + if (proto == null || proto.getContextMap().isEmpty()) { + return Context.current(); + } + final TextMapGetter getter = SpanContextGetter.INSTANCE; + return PROPAGATOR.extract(Context.current(), proto, getter); + } +} + +class SpanContextGetter implements TextMapGetter { + static final SpanContextGetter INSTANCE = new SpanContextGetter(); + + @Override + public Iterable keys(SpanContextProto carrier) { + return carrier.getContextMap().keySet(); + } + + @Override + public String get(SpanContextProto carrier, String key) { + return Optional.ofNullable(carrier).map(SpanContextProto::getContextMap) + .map(map -> map.get(key)).orElse(null); + } + +} \ No newline at end of file diff --git a/ratis-common/src/main/java/org/apache/ratis/util/VersionInfo.java b/ratis-common/src/main/java/org/apache/ratis/util/VersionInfo.java index 07136e9a40..4f24879f87 100644 --- a/ratis-common/src/main/java/org/apache/ratis/util/VersionInfo.java +++ b/ratis-common/src/main/java/org/apache/ratis/util/VersionInfo.java @@ -158,6 +158,14 @@ static void printSystemProperties(BiConsumer out) { sortedMap.forEach(out); } + /** + * Get the current ratis version. + * @return the current ratis version string. + */ + public static String getSoftwareInfoVersion() { + return VersionInfo.load(VersionInfo.class).softwareInfos.getOrDefault(SoftwareInfo.VERSION); + } + public static void main(String[] args) { printSystemProperties((key, value) -> System.out.printf("%-40s = %s%n", key, value)); diff --git a/ratis-proto/src/main/proto/Raft.proto b/ratis-proto/src/main/proto/Raft.proto index 6dbfdb15a5..eba5de3b74 100644 --- a/ratis-proto/src/main/proto/Raft.proto +++ b/ratis-proto/src/main/proto/Raft.proto @@ -117,6 +117,7 @@ message RaftRpcRequestProto { uint64 callId = 4; bool toLeader = 5; + SpanContextProto spanContext = 11; repeated uint64 repliedCallIds = 12; // The call ids of the replied requests uint64 timeoutMs = 13; RoutingTableProto routingTable = 14; @@ -569,3 +570,8 @@ message LogInfoProto { TermIndexProto committed = 3; TermIndexProto lastEntry = 4; } + +// The attribute map for opentelemetry trace +message SpanContextProto { + map context = 1; +} diff --git a/ratis-server/src/main/java/org/apache/ratis/server/impl/RaftServerImpl.java b/ratis-server/src/main/java/org/apache/ratis/server/impl/RaftServerImpl.java index d9dd09d966..0b891f6589 100644 --- a/ratis-server/src/main/java/org/apache/ratis/server/impl/RaftServerImpl.java +++ b/ratis-server/src/main/java/org/apache/ratis/server/impl/RaftServerImpl.java @@ -100,6 +100,8 @@ import org.apache.ratis.statemachine.impl.TransactionContextImpl; import org.apache.ratis.thirdparty.com.google.common.annotations.VisibleForTesting; import org.apache.ratis.thirdparty.com.google.protobuf.InvalidProtocolBufferException; +import org.apache.ratis.trace.TraceUtils; +import org.apache.ratis.trace.TraceConfigKeys; import org.apache.ratis.util.CodeInjectionForTesting; import org.apache.ratis.util.CollectionUtils; import org.apache.ratis.util.ConcurrentUtils; @@ -256,6 +258,8 @@ public long[] getFollowerMatchIndices() { private final ExecutorService serverExecutor; private final ExecutorService clientExecutor; + private final boolean tracingEnabled; + private final AtomicBoolean firstElectionSinceStartup = new AtomicBoolean(true); private final ThreadGroup threadGroup; @@ -282,6 +286,7 @@ public long[] getFollowerMatchIndices() { this.readOption = RaftServerConfigKeys.Read.option(properties); this.writeIndexCache = new WriteIndexCache(properties); this.transactionManager = new TransactionManager(id); + this.tracingEnabled = TraceConfigKeys.enabled(properties); this.leaderElectionMetrics = LeaderElectionMetrics.getLeaderElectionMetrics( getMemberId(), state::getLastLeaderElapsedTimeMs); @@ -943,6 +948,14 @@ CompletableFuture executeSubmitClientRequestAsync(RaftClientReq @Override public CompletableFuture submitClientRequestAsync( RaftClientRequest request) throws IOException { + return TraceUtils.traceAsyncMethodIfEnabled( + tracingEnabled, + () -> submitClientRequestAsyncInternal(request), + request, getMemberId().toString(), "raft.server.submitClientRequestAsync"); + } + + private CompletableFuture submitClientRequestAsyncInternal( + RaftClientRequest request) throws IOException { assertLifeCycleState(LifeCycle.States.RUNNING); LOG.debug("{}: receive client request({})", getMemberId(), request); final Timekeeper timer = raftServerMetrics.getClientRequestTimer(request.getType()); diff --git a/ratis-server/src/test/java/org/apache/ratis/server/impl/RaftServerImplTracingTests.java b/ratis-server/src/test/java/org/apache/ratis/server/impl/RaftServerImplTracingTests.java new file mode 100644 index 0000000000..300cf51cde --- /dev/null +++ b/ratis-server/src/test/java/org/apache/ratis/server/impl/RaftServerImplTracingTests.java @@ -0,0 +1,141 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.ratis.server.impl; + +import io.opentelemetry.api.trace.Span; +import io.opentelemetry.api.trace.SpanKind; +import io.opentelemetry.context.Context; +import io.opentelemetry.sdk.testing.junit5.OpenTelemetryExtension; +import io.opentelemetry.sdk.trace.data.SpanData; +import org.apache.ratis.conf.RaftProperties; +import org.apache.ratis.protocol.ClientId; +import org.apache.ratis.protocol.RaftClientRequest; +import org.apache.ratis.protocol.RaftGroup; +import org.apache.ratis.protocol.RaftGroupId; +import org.apache.ratis.protocol.RaftPeerId; +import org.apache.ratis.protocol.exceptions.ServerNotReadyException; +import org.apache.ratis.server.storage.RaftStorage; +import org.apache.ratis.statemachine.StateMachine; +import org.apache.ratis.statemachine.impl.SimpleStateMachine4Testing; +import org.apache.ratis.trace.TraceConfigKeys; +import org.apache.ratis.trace.TraceUtils; +import org.junit.jupiter.api.Test; +import org.junit.jupiter.api.extension.RegisterExtension; + +import java.util.ArrayList; +import java.util.List; + +import static org.junit.jupiter.api.Assertions.assertEquals; +import static org.junit.jupiter.api.Assertions.assertTrue; +import static org.mockito.Mockito.when; +import static org.mockito.Mockito.mock; + +public class RaftServerImplTracingTests { + + @RegisterExtension + private static final OpenTelemetryExtension openTelemetryExtension = + OpenTelemetryExtension.create(); + + @Test + public void testSubmitClientRequestAsync() throws Exception { + final List spans = submitClientRequestAndCollectNewSpans(true); + assertEquals(2, spans.size()); + assertTrue( + spans.stream().anyMatch(s -> s.getKind() == SpanKind.CLIENT && s.getName().equals("client-span")), + "Expected at least one span with SpanKind.CLIENT" + ); + assertTrue( + spans.stream().anyMatch(s -> s.getKind() == SpanKind.SERVER + && s.getName().equals("raft.server.submitClientRequestAsync")), + "Expected at least one span with SpanKind.SERVER" + ); + + } + + @Test + public void testSubmitClientRequestAsyncTracingDisabled() throws Exception { + final List spans = submitClientRequestAndCollectNewSpans(false); + // Even when server-side tracing is disabled, we still emit the client span used to + // generate the propagated context. + assertEquals(1, spans.size()); + assertTrue( + spans.stream().noneMatch(s -> s.getKind() == SpanKind.SERVER + && s.getName().equals("raft.server.submitClientRequestAsync")), + "Expected no SERVER span when tracing is disabled" + ); + assertTrue( + spans.stream().anyMatch(s -> s.getKind() == SpanKind.CLIENT && s.getName().equals("client-span")), + "Expected at least one span with SpanKind.CLIENT" + ); + } + + private static List submitClientRequestAndCollectNewSpans(boolean enableTracing) + throws Exception { + final int before = openTelemetryExtension.getSpans().size(); + + final RaftServerImpl server = newRaftServerImpl(enableTracing); + try { + final RaftClientRequest request = newRaftClientRequest(RaftClientRequest.writeRequestType()); + + try { + server.submitClientRequestAsync(request); + } catch (ServerNotReadyException ignored) { + // server is not running; only verifying span emission + } + } finally { + server.close(); + } + + final List after = openTelemetryExtension.getSpans(); + return new ArrayList<>(after.subList(before, after.size())); + } + + private static RaftServerImpl newRaftServerImpl(boolean enableTracing) throws Exception { + final RaftGroup group = RaftGroup.emptyGroup(); + final StateMachine sm = new SimpleStateMachine4Testing(); + final RaftServerProxy proxy = mock(RaftServerProxy.class); + when(proxy.getId()).thenReturn(RaftPeerId.valueOf("peer1")); + final RaftProperties properties = new RaftProperties(); + TraceConfigKeys.setEnabled(properties, enableTracing); + when(proxy.getProperties()).thenReturn(properties); + when(proxy.getThreadGroup()).thenReturn(new ThreadGroup("test")); + return new RaftServerImpl(group, sm, proxy, RaftStorage.StartupOption.FORMAT); + } + + private static RaftClientRequest newRaftClientRequest(RaftClientRequest.Type type) { + final Span clientSpan = + openTelemetryExtension.getOpenTelemetry().getTracer("test") + .spanBuilder("client-span") + .setSpanKind(SpanKind.CLIENT) + .startSpan(); + try { + final Context clientContext = Context.current().with(clientSpan); + return RaftClientRequest.newBuilder() + .setClientId(ClientId.randomId()) + .setServerId(RaftPeerId.valueOf("s0")) + .setGroupId(RaftGroupId.randomId()) + .setCallId(1L) + .setType(type) + .setSpanContext(TraceUtils.injectContextToProto(clientContext)) + .build(); + } finally { + clientSpan.end(); + } + } +} + From 0491d2699f17ce64cbff0c4868b92405cc9a202a Mon Sep 17 00:00:00 2001 From: Symious <14933944+symious@users.noreply.github.com> Date: Thu, 12 Mar 2026 15:15:24 +0800 Subject: [PATCH 274/397] RATIS-2427. Close LogAppender for illegalStateException (#1369) --- .../java/org/apache/ratis/server/raftlog/RaftLog.java | 3 +++ .../org/apache/ratis/server/leader/LogAppenderBase.java | 9 ++++++--- .../org/apache/ratis/server/raftlog/RaftLogBase.java | 2 +- 3 files changed, 10 insertions(+), 4 deletions(-) diff --git a/ratis-server-api/src/main/java/org/apache/ratis/server/raftlog/RaftLog.java b/ratis-server-api/src/main/java/org/apache/ratis/server/raftlog/RaftLog.java index e504462b80..e194f865ed 100644 --- a/ratis-server-api/src/main/java/org/apache/ratis/server/raftlog/RaftLog.java +++ b/ratis-server-api/src/main/java/org/apache/ratis/server/raftlog/RaftLog.java @@ -43,6 +43,9 @@ public interface RaftLog extends RaftLogSequentialOps, Closeable { /** Invalid log index is used to indicate that the log index is missing. */ long INVALID_LOG_INDEX = LEAST_VALID_LOG_INDEX - 1; + /** Is this log already opened but not yet closed? */ + boolean isOpened(); + /** Does this log contains the given {@link TermIndex}? */ default boolean contains(TermIndex ti) { Objects.requireNonNull(ti, "ti == null"); diff --git a/ratis-server/src/main/java/org/apache/ratis/server/leader/LogAppenderBase.java b/ratis-server/src/main/java/org/apache/ratis/server/leader/LogAppenderBase.java index 5a27cda510..be0404da36 100644 --- a/ratis-server/src/main/java/org/apache/ratis/server/leader/LogAppenderBase.java +++ b/ratis-server/src/main/java/org/apache/ratis/server/leader/LogAppenderBase.java @@ -124,7 +124,10 @@ public void start() { @Override public boolean isRunning() { - return daemon.isWorking() && server.getInfo().isLeader(); + return daemon.isWorking() + && server.getInfo().isAlive() + && server.getInfo().isLeader() + && getRaftLog().isOpened(); } @Override @@ -133,8 +136,8 @@ public CompletableFuture stopAsync() { } void restart() { - if (!server.getInfo().isAlive()) { - LOG.warn("Failed to restart {}: server {} is not alive", this, server.getMemberId()); + if (!isRunning()) { + LOG.warn("{} is not running: skipping restart", this); return; } getLeaderState().restart(this); diff --git a/ratis-server/src/main/java/org/apache/ratis/server/raftlog/RaftLogBase.java b/ratis-server/src/main/java/org/apache/ratis/server/raftlog/RaftLogBase.java index 8c2b66f969..48b410147c 100644 --- a/ratis-server/src/main/java/org/apache/ratis/server/raftlog/RaftLogBase.java +++ b/ratis-server/src/main/java/org/apache/ratis/server/raftlog/RaftLogBase.java @@ -113,7 +113,7 @@ public void checkLogState() { state.assertOpen(); } - /** Is this log already opened? */ + @Override public boolean isOpened() { return state.isOpened(); } From 25257919913f6bf71bce9305d506eb3cfb29b820 Mon Sep 17 00:00:00 2001 From: Potato Date: Fri, 13 Mar 2026 15:54:16 +0800 Subject: [PATCH 275/397] RATIS-2435. ratis-bom POM is not signed during release staging (#1374) --- ratis-bom/pom.xml | 28 ++++++++++++++++++++++++++++ 1 file changed, 28 insertions(+) diff --git a/ratis-bom/pom.xml b/ratis-bom/pom.xml index a62f2798db..46c8f556ed 100644 --- a/ratis-bom/pom.xml +++ b/ratis-bom/pom.xml @@ -145,4 +145,32 @@ + + + + apache-release + + + + org.apache.maven.plugins + maven-gpg-plugin + 3.2.7 + + + sign-release-artifacts + + sign + + + + --digest-algo=SHA512 + + + + + + + + + From 185f83c54a50963468aa18dcbf36fca9b82278b1 Mon Sep 17 00:00:00 2001 From: Wei-Chiu Chuang Date: Fri, 13 Mar 2026 19:39:29 -0700 Subject: [PATCH 276/397] RATIS-2434. Regression test for data race between SegmentedRaftLog.get() and LogSegment.append(). (#1378) --- .../server/raftlog/segmented/LogSegment.java | 7 ++ .../segmented/TestSegmentedRaftLog.java | 73 +++++++++++++++++++ 2 files changed, 80 insertions(+) diff --git a/ratis-server/src/main/java/org/apache/ratis/server/raftlog/segmented/LogSegment.java b/ratis-server/src/main/java/org/apache/ratis/server/raftlog/segmented/LogSegment.java index c40b91f708..e9cb2e50f9 100644 --- a/ratis-server/src/main/java/org/apache/ratis/server/raftlog/segmented/LogSegment.java +++ b/ratis-server/src/main/java/org/apache/ratis/server/raftlog/segmented/LogSegment.java @@ -47,6 +47,7 @@ import java.util.concurrent.atomic.AtomicLong; import java.util.concurrent.atomic.AtomicReference; import java.util.function.Consumer; +import org.apache.ratis.util.CodeInjectionForTesting; /** @@ -357,6 +358,7 @@ void appendToOpenSegment(LogEntryProto entry, Op op) { append(true, entry, op); } + public static final String APPEND_RECORD = LogSegment.class.getSimpleName() + ".append"; private void append(boolean keepEntryInCache, LogEntryProto entry, Op op) { Objects.requireNonNull(entry, "entry == null"); final LogRecord currentLast = records.getLast(); @@ -371,9 +373,14 @@ private void append(boolean keepEntryInCache, LogEntryProto entry, Op op) { final LogRecord record = new LogRecord(totalFileSize, entry); if (keepEntryInCache) { + // It is important to put the entry into the cache before appending the + // record to the record list. Otherwise, a reader thread may get the + // record from the list but not the entry from the cache. putEntryCache(record.getTermIndex(), entry, op); + CodeInjectionForTesting.execute(APPEND_RECORD, this, record.getTermIndex()); } records.append(record); + totalFileSize += getEntrySize(entry, op); endIndex = entry.getIndex(); } diff --git a/ratis-test/src/test/java/org/apache/ratis/server/raftlog/segmented/TestSegmentedRaftLog.java b/ratis-test/src/test/java/org/apache/ratis/server/raftlog/segmented/TestSegmentedRaftLog.java index a772b00029..43aafc8967 100644 --- a/ratis-test/src/test/java/org/apache/ratis/server/raftlog/segmented/TestSegmentedRaftLog.java +++ b/ratis-test/src/test/java/org/apache/ratis/server/raftlog/segmented/TestSegmentedRaftLog.java @@ -67,6 +67,7 @@ import java.util.function.LongSupplier; import java.util.function.Supplier; import java.util.stream.Stream; +import java.util.concurrent.atomic.AtomicBoolean; import org.junit.jupiter.api.AfterEach; import org.junit.jupiter.api.Assertions; @@ -75,6 +76,8 @@ import org.junit.jupiter.params.ParameterizedTest; import org.junit.jupiter.params.provider.Arguments; import org.junit.jupiter.params.provider.MethodSource; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; import org.slf4j.event.Level; import static java.lang.Boolean.FALSE; @@ -873,4 +876,74 @@ public void testAsyncFlushPerf2(Boolean useAsyncFlush, Boolean smSyncFlush) thro " ns with asyncFlush " + useAsyncFlush); } } + + public static final Logger LOG = LoggerFactory.getLogger(TestSegmentedRaftLog.class); + @Test + public void testConcurrentGetDuringAppend() throws Exception { + RaftServerConfigKeys.Log.setReadLockEnabled(properties, false); + final CountDownLatch injectionPaused = new CountDownLatch(1); + final CountDownLatch readerCanProceed = new CountDownLatch(1); + final AtomicReference error = new AtomicReference<>(); + final AtomicReference readEntry = new AtomicReference<>(); + + final CodeInjectionForTesting.Code code = (localId, remoteId, args) -> { + // in log worker thread, holding write lock + injectionPaused.countDown(); + try { + if (!readerCanProceed.await(5, TimeUnit.SECONDS)) { + error.set(new TimeoutException("The reader thread did not start in time.")); + } + } catch (InterruptedException e) { + error.set(e); + } + return true; + }; + + try (SegmentedRaftLog raftLog = newSegmentedRaftLog()) { + CodeInjectionForTesting.put(LogSegment.APPEND_RECORD, code); + raftLog.open(RaftLog.INVALID_LOG_INDEX, null); + final LogEntryProto newEntry = prepareLogEntry(1, 0, () -> "newEntry", false); + + // Run appendEntry asynchronously. + final Thread appender = new Thread(() -> { + raftLog.appendEntry(newEntry).join(); + }); + appender.start(); + + // Wait until the append operation is paused at the injection point. + Assertions.assertTrue(injectionPaused.await(5, TimeUnit.SECONDS), "Injection point was not hit."); + + // Start a new reader thread to call get(). + // This thread will block until the write lock is released. + final Thread reader = new Thread(() -> { + try { + readEntry.set(raftLog.get(newEntry.getIndex())); + Assertions.assertNull(readEntry.get()); + // Unblock the writer thread. + readerCanProceed.countDown(); + } catch (Throwable t) { + error.set(t); + } + }); + reader.start(); + + // Wait for both the append and the read to complete. + reader.join(); + appender.join(); + + // Check for errors. + if (error.get() != null) { + throw new Exception("Test failed", error.get()); + } + + // When the reader's get() call completed, the append was fully finished, + // so it should have returned the correct entry. + Assertions.assertEquals(newEntry.getIndex(), raftLog.getLastEntryTermIndex().getIndex()); + readEntry.set(raftLog.get(newEntry.getIndex())); + Assertions.assertNotNull(readEntry.get()); + Assertions.assertEquals(newEntry, readEntry.get()); + } finally { + CodeInjectionForTesting.remove(LogSegment.APPEND_RECORD); + } + } } From c030f16cca2c89d76f836973c8e6e21bc0fcca7c Mon Sep 17 00:00:00 2001 From: "Doroszlai, Attila" <6454655+adoroszlai@users.noreply.github.com> Date: Sat, 14 Mar 2026 07:28:29 +0100 Subject: [PATCH 277/397] RATIS-2436. Bump GitHub action versions (#1375) --- .github/workflows/check.yaml | 22 +++++++++++----------- .github/workflows/ci.yaml | 10 +++++----- .github/workflows/repeat-test.yaml | 10 +++++----- .github/workflows/vulnerability-check.yaml | 6 +++--- 4 files changed, 24 insertions(+), 24 deletions(-) diff --git a/.github/workflows/check.yaml b/.github/workflows/check.yaml index 996277d75d..cb3d31d1fe 100644 --- a/.github/workflows/check.yaml +++ b/.github/workflows/check.yaml @@ -90,11 +90,11 @@ jobs: steps: - name: Checkout project if: ${{ !inputs.needs-source-tarball }} - uses: actions/checkout@v4 + uses: actions/checkout@v6 - name: Download source tarball if: ${{ inputs.needs-source-tarball }} - uses: actions/download-artifact@v4 + uses: actions/download-artifact@v8 with: name: ratis-src @@ -105,7 +105,7 @@ jobs: - name: Create cache for Maven dependencies if: ${{ inputs.script == 'build' }} - uses: actions/cache@v4 + uses: actions/cache@v5 with: path: | ~/.m2/repository/*/*/* @@ -116,7 +116,7 @@ jobs: - name: Restore cache for Maven dependencies if: ${{ inputs.script != 'build' }} - uses: actions/cache/restore@v4 + uses: actions/cache/restore@v5 with: path: | ~/.m2/repository/*/*/* @@ -128,7 +128,7 @@ jobs: - name: Download Maven repo id: download-maven-repo if: ${{ inputs.needs-maven-repo }} - uses: actions/download-artifact@v4 + uses: actions/download-artifact@v8 with: name: maven-repo path: | @@ -136,7 +136,7 @@ jobs: - name: Download binary tarball if: ${{ inputs.needs-binary-tarball }} - uses: actions/download-artifact@v4 + uses: actions/download-artifact@v8 with: name: ratis-bin @@ -148,7 +148,7 @@ jobs: - name: Setup java ${{ inputs.java-version }} if: ${{ inputs.java-version }} - uses: actions/setup-java@v4 + uses: actions/setup-java@v5 with: distribution: 'temurin' java-version: ${{ inputs.java-version }} @@ -169,7 +169,7 @@ jobs: - name: Archive build results if: ${{ !cancelled() }} - uses: actions/upload-artifact@v4 + uses: actions/upload-artifact@v7 with: name: ${{ (inputs.split && format('{0}-{1}', inputs.script, inputs.split)) || inputs.script }} path: target/${{ inputs.script }} @@ -179,7 +179,7 @@ jobs: # to avoid the need for 3 more inputs. - name: Store binaries for tests if: ${{ inputs.script == 'build' && !cancelled() }} - uses: actions/upload-artifact@v4 + uses: actions/upload-artifact@v7 with: name: ratis-bin path: | @@ -188,7 +188,7 @@ jobs: - name: Store source tarball for compilation if: ${{ inputs.script == 'build' && !cancelled() }} - uses: actions/upload-artifact@v4 + uses: actions/upload-artifact@v7 with: name: ratis-src path: | @@ -197,7 +197,7 @@ jobs: - name: Store Maven repo for tests if: ${{ inputs.script == 'build' && !cancelled() }} - uses: actions/upload-artifact@v4 + uses: actions/upload-artifact@v7 with: name: maven-repo path: | diff --git a/.github/workflows/ci.yaml b/.github/workflows/ci.yaml index 6ef2b0c66d..50b88212a5 100644 --- a/.github/workflows/ci.yaml +++ b/.github/workflows/ci.yaml @@ -109,11 +109,11 @@ jobs: if: github.event_name != 'pull_request' steps: - name: Checkout project - uses: actions/checkout@v4 + uses: actions/checkout@v6 with: fetch-depth: 0 - name: Cache for maven dependencies - uses: actions/cache/restore@v4 + uses: actions/cache/restore@v5 with: path: | ~/.m2/repository @@ -122,12 +122,12 @@ jobs: restore-keys: | maven-repo- - name: Setup java 17 - uses: actions/setup-java@v4 + uses: actions/setup-java@v5 with: distribution: 'temurin' java-version: 17 - name: Download artifacts - uses: actions/download-artifact@v4 + uses: actions/download-artifact@v8 with: path: target/artifacts - name: Untar binaries @@ -143,7 +143,7 @@ jobs: SONAR_TOKEN: ${{ secrets.SONARCLOUD_TOKEN }} GITHUB_TOKEN: ${{ secrets.GITHUB_TOKEN }} - name: Archive build results - uses: actions/upload-artifact@v4 + uses: actions/upload-artifact@v7 if: always() with: name: ${{ github.job }} diff --git a/.github/workflows/repeat-test.yaml b/.github/workflows/repeat-test.yaml index 11b77b1d35..9a985fd23c 100644 --- a/.github/workflows/repeat-test.yaml +++ b/.github/workflows/repeat-test.yaml @@ -95,11 +95,11 @@ jobs: split: ${{ fromJson(needs.prepare.outputs.matrix) }} fail-fast: ${{ fromJson(github.event.inputs.fail-fast) }} steps: - - uses: actions/checkout@v4 + - uses: actions/checkout@v6 with: ref: ${{ needs.prepare.outputs.ref }} - name: Cache for maven dependencies - uses: actions/cache@v4 + uses: actions/cache@v5 with: path: | ~/.m2/repository @@ -108,7 +108,7 @@ jobs: restore-keys: | maven-repo- - name: Setup java - uses: actions/setup-java@v4 + uses: actions/setup-java@v5 with: distribution: 'temurin' java-version: 8 @@ -121,7 +121,7 @@ jobs: run: dev-support/checks/_summary.sh target/unit/summary.txt if: ${{ !cancelled() }} - name: Archive build results - uses: actions/upload-artifact@v4 + uses: actions/upload-artifact@v7 if: ${{ failure() }} with: name: result-${{ github.run_number }}-${{ github.run_id }}-split-${{ matrix.split }} @@ -132,7 +132,7 @@ jobs: runs-on: ubuntu-24.04 steps: - name: Download build results - uses: actions/download-artifact@v4 + uses: actions/download-artifact@v8 - name: Count failures run: | failures=$(find . -name 'summary.txt' | grep -v 'iteration' | xargs grep -v 'exit code: 0' | wc -l) diff --git a/.github/workflows/vulnerability-check.yaml b/.github/workflows/vulnerability-check.yaml index 49d4fa80ac..a0146f14de 100644 --- a/.github/workflows/vulnerability-check.yaml +++ b/.github/workflows/vulnerability-check.yaml @@ -35,9 +35,9 @@ jobs: runs-on: ubuntu-latest steps: - - uses: actions/checkout@v4 + - uses: actions/checkout@v6 - name: Set up JDK 11 - uses: actions/setup-java@v4 + uses: actions/setup-java@v5 with: distribution: corretto java-version: 11 @@ -57,7 +57,7 @@ jobs: echo "REPORT_DATE=$target_time" >> $GITHUB_ENV - name: Upload Artifact - uses: actions/upload-artifact@v4 + uses: actions/upload-artifact@v7 with: name: vulnerability-check-result-${{ env.REPORT_DATE }} path: target/dependency-check-report.html From 271b12d2618837ef1f568e91455deb62527484f5 Mon Sep 17 00:00:00 2001 From: "Doroszlai, Attila" <6454655+adoroszlai@users.noreply.github.com> Date: Wed, 18 Mar 2026 02:46:25 +0100 Subject: [PATCH 278/397] RATIS-2439. Remove URL from ratis-version.properties (#1380) --- .../src/main/java/org/apache/ratis/util/VersionInfo.java | 2 +- src/main/resources/ratis-version.properties | 1 - 2 files changed, 1 insertion(+), 2 deletions(-) diff --git a/ratis-common/src/main/java/org/apache/ratis/util/VersionInfo.java b/ratis-common/src/main/java/org/apache/ratis/util/VersionInfo.java index 4f24879f87..976b2ad6e1 100644 --- a/ratis-common/src/main/java/org/apache/ratis/util/VersionInfo.java +++ b/ratis-common/src/main/java/org/apache/ratis/util/VersionInfo.java @@ -47,7 +47,7 @@ public final class VersionInfo { private enum SoftwareInfo { // the ordering is the output ordering - NAME, VERSION, URL, REVISION; + NAME, VERSION, REVISION; static SoftwareInfo parse(String key) { for (SoftwareInfo info : SoftwareInfo.values()) { diff --git a/src/main/resources/ratis-version.properties b/src/main/resources/ratis-version.properties index 7413ff9942..f34dc73dc7 100644 --- a/src/main/resources/ratis-version.properties +++ b/src/main/resources/ratis-version.properties @@ -17,5 +17,4 @@ # name=${project.name} version=${project.version} -url=${version-info.scm.uri} revision=${version-info.scm.commit} From d94789d669572e32cb7d95f6425c5c4786c8b9cb Mon Sep 17 00:00:00 2001 From: Potato Date: Wed, 18 Mar 2026 09:46:57 +0800 Subject: [PATCH 279/397] RATIS-2437. make_rc.sh version check prevents releasing subsequent RCs (#1376) Signed-off-by: OneSizeFitsQuorum --- dev-support/make_rc.sh | 4 ---- 1 file changed, 4 deletions(-) diff --git a/dev-support/make_rc.sh b/dev-support/make_rc.sh index 551508ecfe..42cadb07d6 100755 --- a/dev-support/make_rc.sh +++ b/dev-support/make_rc.sh @@ -96,10 +96,6 @@ mvnFun() { git reset --hard git clean -fdx mvnFun versions:set -DnewVersion="$RATISVERSION" -DprocessAllModules - if ! git diff --name-only | grep -q ratis-bom/pom.xml; then - echo 'ratis-bom not updated by `mvn versions:set`' >&2 - exit 1 - fi git commit --allow-empty -a -m "Change version for the version $RATISVERSION $RC" git config user.signingkey "${CODESIGNINGKEY}" From f3c2128a3d8f7d96677c14197585fdc791560283 Mon Sep 17 00:00:00 2001 From: Snehasish Roy Date: Fri, 20 Mar 2026 23:37:10 +0530 Subject: [PATCH 280/397] RATIS-2438: Update membership-change documentation (#1379) Co-authored-by: Snehasish Roy --- ratis-docs/src/site/markdown/membership-change.md | 8 ++++---- 1 file changed, 4 insertions(+), 4 deletions(-) diff --git a/ratis-docs/src/site/markdown/membership-change.md b/ratis-docs/src/site/markdown/membership-change.md index 81ae5495cc..3a4bc46b6f 100644 --- a/ratis-docs/src/site/markdown/membership-change.md +++ b/ratis-docs/src/site/markdown/membership-change.md @@ -39,11 +39,11 @@ Be careful to keep both separate majorities online! To add a new node (e.g., `N3`) to an existing group (e.g., `N0`, `N1`, `N2`), follow these steps: -1. Start the new peer `N3` with **EMPTY** group. +1. Start the new peer `N3` with the existing groupId and provide an empty list of peers. ```java RaftServer N3 = RaftServer.newBuilder() - .setGroup(RaftGroup.emptygroup()) + .setGroup(RaftGroup.valueOf(groupId, Collections.emptyList())) .setProperties(properties) .setServerId(n3id) .setStateMachine(userStateMachine) @@ -52,8 +52,8 @@ To add a new node (e.g., `N3`) to an existing group (e.g., `N0`, `N1`, `N2`), fo ``` 2. Invoke a `setConfiguration` method in the [AdminApi]( -../../../../ratis-client/src/main/java/org/apache/ratis/client/api/AdminApi.java#L44) -with the new group as the parameter. +../../../../ratis-client/src/main/java/org/apache/ratis/client/api/AdminApi.java#L44) + on the existing cluster with the new list of peers as the parameter. It will wait for the new peer to catch up before returning the reply. ```java reply = client.admin().setConfiguration(List.of(N0, N1, N2, N3)) From 8d13def58e989ab43b6fdfed9b7166bde2e87543 Mon Sep 17 00:00:00 2001 From: "Doroszlai, Attila" <6454655+adoroszlai@users.noreply.github.com> Date: Sun, 22 Mar 2026 11:16:57 +0100 Subject: [PATCH 281/397] RATIS-2440. Let ratis-bom inherit from Apache parent (#1381) --- pom.xml | 12 +++--------- ratis-bom/pom.xml | 45 +++++++++++---------------------------------- 2 files changed, 14 insertions(+), 43 deletions(-) diff --git a/pom.xml b/pom.xml index 5c098c311e..14a35a7d12 100644 --- a/pom.xml +++ b/pom.xml @@ -140,9 +140,6 @@ - 2025-11-03T02:50:35Z - UTF-8 - UTF-8 ${project.build.directory}/maven-shared-archive-resources/META-INF/LICENSE @@ -168,12 +165,9 @@ 4.8.6 4.8.6.2 - apache.snapshots.https - Apache Development Snapshot Repository - https://repository.apache.org/content/repositories/snapshots - apache.staging.https - Apache Release Distribution Repository - https://repository.apache.org/service/local/staging/deploy/maven2 + ${distMgmtReleasesId} + ${distMgmtReleasesName} + ${distMgmtReleasesUrl} bash diff --git a/ratis-bom/pom.xml b/ratis-bom/pom.xml index 46c8f556ed..aac81f0170 100644 --- a/ratis-bom/pom.xml +++ b/ratis-bom/pom.xml @@ -14,6 +14,14 @@ --> 4.0.0 + + + org.apache + apache + 35 + + + org.apache.ratis ratis-bom 3.3.0-SNAPSHOT @@ -36,12 +44,9 @@ - apache.snapshots.https - Apache Development Snapshot Repository - https://repository.apache.org/content/repositories/snapshots - apache.staging.https - Apache Release Distribution Repository - https://repository.apache.org/service/local/staging/deploy/maven2 + ${distMgmtReleasesId} + ${distMgmtReleasesName} + ${distMgmtReleasesUrl} 1.0.11 @@ -145,32 +150,4 @@ - - - - apache-release - - - - org.apache.maven.plugins - maven-gpg-plugin - 3.2.7 - - - sign-release-artifacts - - sign - - - - --digest-algo=SHA512 - - - - - - - - - From f5b94ef2c8b874b91de8db194d30ac771fc553de Mon Sep 17 00:00:00 2001 From: Abhishek Pal Date: Sun, 22 Mar 2026 21:22:49 +0530 Subject: [PATCH 282/397] RATIS-2442. Wait for configuration to be applied to listeners in test (#1383) --- .../java/org/apache/ratis/server/impl/LeaderElectionTests.java | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/ratis-server/src/test/java/org/apache/ratis/server/impl/LeaderElectionTests.java b/ratis-server/src/test/java/org/apache/ratis/server/impl/LeaderElectionTests.java index 6959bd3422..3a47d127c5 100644 --- a/ratis-server/src/test/java/org/apache/ratis/server/impl/LeaderElectionTests.java +++ b/ratis-server/src/test/java/org/apache/ratis/server/impl/LeaderElectionTests.java @@ -64,7 +64,6 @@ import static org.apache.ratis.RaftTestUtil.getPeersWithPriority; import static org.apache.ratis.RaftTestUtil.waitForLeader; -import static org.apache.ratis.proto.RaftProtos.RaftPeerRole.LISTENER; import static org.apache.ratis.server.metrics.LeaderElectionMetrics.LAST_LEADER_ELECTION_ELAPSED_TIME; import static org.apache.ratis.server.metrics.LeaderElectionMetrics.LEADER_ELECTION_COUNT_METRIC; import static org.apache.ratis.server.metrics.LeaderElectionMetrics.LEADER_ELECTION_TIME_TAKEN; @@ -557,6 +556,8 @@ public void testChangeListenerToFollower() throws Exception { Collection peer = leader.getRaftConf().getAllPeers(RaftProtos.RaftPeerRole.LISTENER); assertEquals(0, peer.size()); + RaftServerTestUtil.waitAndCheckNewConf(cluster, cluster.getPeers(), 0, null); + listeners = cluster.getListeners() .stream().map(RaftServer.Division::getPeer).collect(Collectors.toList()); assertEquals(0, listeners.size()); From bdc149b3365ba6887c2edb0363896064237ca610 Mon Sep 17 00:00:00 2001 From: "Doroszlai, Attila" <6454655+adoroszlai@users.noreply.github.com> Date: Sun, 22 Mar 2026 17:49:19 +0100 Subject: [PATCH 283/397] RATIS-2443. Sync fork should not trigger CI (#1384) --- .github/workflows/post-commit.yaml | 7 +++++-- 1 file changed, 5 insertions(+), 2 deletions(-) diff --git a/.github/workflows/post-commit.yaml b/.github/workflows/post-commit.yaml index 583eb7bcf5..1d1ba06fd9 100644 --- a/.github/workflows/post-commit.yaml +++ b/.github/workflows/post-commit.yaml @@ -24,10 +24,13 @@ on: pull_request: concurrency: - group: ci-${{ github.event.pull_request.number || github.sha }} - cancel-in-progress: ${{ github.event_name == 'pull_request' }} + group: ci-${{ github.event.pull_request.number || case(github.repository == 'apache/ratis', github.sha, github.ref_name) }} + cancel-in-progress: ${{ github.event_name == 'pull_request' || github.repository != 'apache/ratis' }} jobs: CI: + if: github.event_name == 'pull_request' + || github.repository == 'apache/ratis' + || github.ref_name != 'master' uses: ./.github/workflows/ci.yaml secrets: inherit From e41476d9191a200998ed8b3494e6b7980d59ac5c Mon Sep 17 00:00:00 2001 From: "Tak Lon (Stephen) Wu" Date: Thu, 26 Mar 2026 05:03:10 +0800 Subject: [PATCH 284/397] RATIS-2395. Add Client Span for AyncImpl (#1385) --- .../apache/ratis/client/impl/AsyncImpl.java | 4 +- .../ratis/client/impl/RaftClientImpl.java | 2 + .../apache/ratis/trace/RatisAttributes.java | 4 + .../org/apache/ratis/trace/TraceClient.java | 63 +++++++++++ .../org/apache/ratis/trace/TraceServer.java | 59 ++++++++++ .../org/apache/ratis/trace/TraceUtils.java | 103 ++++++++++-------- .../apache/ratis/trace/TestTraceUtils.java | 66 +++++++++++ .../ratis/server/impl/RaftServerImpl.java | 9 +- .../grpc/TestRaftAsyncWithGrpcTracing.java | 75 +++++++++++++ .../grpc/TestRetryCacheWithGrpcTracing.java | 81 ++++++++++++++ 10 files changed, 412 insertions(+), 54 deletions(-) create mode 100644 ratis-common/src/main/java/org/apache/ratis/trace/TraceClient.java create mode 100644 ratis-common/src/main/java/org/apache/ratis/trace/TraceServer.java create mode 100644 ratis-common/src/test/java/org/apache/ratis/trace/TestTraceUtils.java create mode 100644 ratis-test/src/test/java/org/apache/ratis/grpc/TestRaftAsyncWithGrpcTracing.java create mode 100644 ratis-test/src/test/java/org/apache/ratis/grpc/TestRetryCacheWithGrpcTracing.java diff --git a/ratis-client/src/main/java/org/apache/ratis/client/impl/AsyncImpl.java b/ratis-client/src/main/java/org/apache/ratis/client/impl/AsyncImpl.java index 8547ce2665..973b0db0cf 100644 --- a/ratis-client/src/main/java/org/apache/ratis/client/impl/AsyncImpl.java +++ b/ratis-client/src/main/java/org/apache/ratis/client/impl/AsyncImpl.java @@ -27,6 +27,7 @@ import org.apache.ratis.protocol.RaftClientReply; import org.apache.ratis.protocol.RaftClientRequest; import org.apache.ratis.protocol.RaftPeerId; +import org.apache.ratis.trace.TraceClient; /** Async api implementations. */ class AsyncImpl implements AsyncRpcApi { @@ -38,7 +39,8 @@ class AsyncImpl implements AsyncRpcApi { CompletableFuture send( RaftClientRequest.Type type, Message message, RaftPeerId server) { - return client.getOrderedAsync().send(type, message, server); + return TraceClient.asyncSend( + () -> client.getOrderedAsync().send(type, message, server), type, server); } @Override diff --git a/ratis-client/src/main/java/org/apache/ratis/client/impl/RaftClientImpl.java b/ratis-client/src/main/java/org/apache/ratis/client/impl/RaftClientImpl.java index db789aef2f..27ae2e6bab 100644 --- a/ratis-client/src/main/java/org/apache/ratis/client/impl/RaftClientImpl.java +++ b/ratis-client/src/main/java/org/apache/ratis/client/impl/RaftClientImpl.java @@ -43,6 +43,7 @@ import org.apache.ratis.retry.RetryPolicy; import org.apache.ratis.thirdparty.com.google.common.cache.Cache; import org.apache.ratis.thirdparty.com.google.common.cache.CacheBuilder; +import org.apache.ratis.trace.TraceUtils; import org.apache.ratis.util.CollectionUtils; import org.apache.ratis.util.IOUtils; import org.apache.ratis.util.JavaUtils; @@ -218,6 +219,7 @@ private synchronized Set getAndReset() { .setParameters(parameters) .build()); this.adminApi = JavaUtils.memoize(() -> new AdminImpl(this)); + TraceUtils.setTracerWhenEnabled(properties); } @Override diff --git a/ratis-common/src/main/java/org/apache/ratis/trace/RatisAttributes.java b/ratis-common/src/main/java/org/apache/ratis/trace/RatisAttributes.java index d74c63757f..3c3be83e79 100644 --- a/ratis-common/src/main/java/org/apache/ratis/trace/RatisAttributes.java +++ b/ratis-common/src/main/java/org/apache/ratis/trace/RatisAttributes.java @@ -29,6 +29,10 @@ public final class RatisAttributes { public static final AttributeKey MEMBER_ID = AttributeKey.stringKey("raft.member.id"); public static final AttributeKey CALL_ID = AttributeKey.stringKey("raft.call.id"); + public static final AttributeKey PEER_ID = AttributeKey.stringKey("raft.peer.id"); + public static final AttributeKey OPERATION_NAME = AttributeKey.stringKey("raft.operation.name"); + public static final AttributeKey OPERATION_TYPE = AttributeKey.stringKey("raft.operation.type"); + private RatisAttributes() { } diff --git a/ratis-common/src/main/java/org/apache/ratis/trace/TraceClient.java b/ratis-common/src/main/java/org/apache/ratis/trace/TraceClient.java new file mode 100644 index 0000000000..0ab34e689e --- /dev/null +++ b/ratis-common/src/main/java/org/apache/ratis/trace/TraceClient.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.ratis.trace; + +import io.opentelemetry.api.trace.Span; +import io.opentelemetry.api.trace.SpanKind; +import org.apache.ratis.protocol.RaftClientRequest; +import org.apache.ratis.protocol.RaftPeerId; +import org.apache.ratis.util.Preconditions; +import org.apache.ratis.util.function.CheckedSupplier; + +import java.util.concurrent.CompletableFuture; + +/** Client-side OpenTelemetry helpers. */ +public final class TraceClient { + private static final String LEADER = "LEADER"; + + private TraceClient() { + } + + /** + * Traces an asynchronous client send ({@code Async::send}) when tracing is enabled. + */ + public static CompletableFuture asyncSend( + CheckedSupplier, THROWABLE> action, + RaftClientRequest.Type type, RaftPeerId server) throws THROWABLE { + if (!TraceUtils.isEnabled()) { + return action.get(); + } + return TraceUtils.traceAsyncMethod(action, + () -> createClientOperationSpan(type, server, "Async::send")); + } + + private static Span createClientOperationSpan(RaftClientRequest.Type type, RaftPeerId server, + String spanName) { + Preconditions.assertNotNull(spanName, () -> "Span name cannot be null"); + Preconditions.assertTrue(!spanName.isEmpty(), "Span name should not be empty"); + String peerId = server == null ? LEADER : String.valueOf(server); + final Span span = TraceUtils.getGlobalTracer() + .spanBuilder(spanName) + .setSpanKind(SpanKind.CLIENT) + .startSpan(); + span.setAttribute(RatisAttributes.PEER_ID, peerId); + span.setAttribute(RatisAttributes.OPERATION_NAME, spanName); + span.setAttribute(RatisAttributes.OPERATION_TYPE, String.valueOf(type)); + return span; + } +} diff --git a/ratis-common/src/main/java/org/apache/ratis/trace/TraceServer.java b/ratis-common/src/main/java/org/apache/ratis/trace/TraceServer.java new file mode 100644 index 0000000000..9670f0d763 --- /dev/null +++ b/ratis-common/src/main/java/org/apache/ratis/trace/TraceServer.java @@ -0,0 +1,59 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.ratis.trace; + +import io.opentelemetry.api.trace.Span; +import io.opentelemetry.api.trace.SpanKind; +import io.opentelemetry.context.Context; +import org.apache.ratis.protocol.RaftClientRequest; +import org.apache.ratis.util.function.CheckedSupplier; + +import java.util.concurrent.CompletableFuture; + +/** Server-side OpenTelemetry helpers. */ +public final class TraceServer { + private TraceServer() { + } + + /** + * Traces {@code submitClientRequestAsync} when tracing is enabled. + */ + public static CompletableFuture traceAsyncMethod( + CheckedSupplier, THROWABLE> action, + RaftClientRequest request, String memberId, String spanName) throws THROWABLE { + if (!TraceUtils.isEnabled()) { + return action.get(); + } + return TraceUtils.traceAsyncMethod(action, + () -> createServerSpanFromClientRequest(request, memberId, spanName)); + } + + private static Span createServerSpanFromClientRequest(RaftClientRequest request, String memberId, + String spanName) { + final Context remoteContext = TraceUtils.extractContextFromProto(request.getSpanContext()); + final Span span = TraceUtils.getGlobalTracer() + .spanBuilder(spanName) + .setParent(remoteContext) + .setSpanKind(SpanKind.SERVER) + .startSpan(); + span.setAttribute(RatisAttributes.CLIENT_ID, String.valueOf(request.getClientId())); + span.setAttribute(RatisAttributes.CALL_ID, String.valueOf(request.getCallId())); + span.setAttribute(RatisAttributes.MEMBER_ID, memberId); + return span; + } +} diff --git a/ratis-common/src/main/java/org/apache/ratis/trace/TraceUtils.java b/ratis-common/src/main/java/org/apache/ratis/trace/TraceUtils.java index bb10e24553..f350ca8884 100644 --- a/ratis-common/src/main/java/org/apache/ratis/trace/TraceUtils.java +++ b/ratis-common/src/main/java/org/apache/ratis/trace/TraceUtils.java @@ -19,15 +19,14 @@ import io.opentelemetry.api.GlobalOpenTelemetry; import io.opentelemetry.api.trace.Span; -import io.opentelemetry.api.trace.SpanKind; import io.opentelemetry.api.trace.StatusCode; import io.opentelemetry.api.trace.Tracer; import io.opentelemetry.context.Context; import io.opentelemetry.context.Scope; import io.opentelemetry.context.propagation.TextMapPropagator; import io.opentelemetry.context.propagation.TextMapGetter; +import org.apache.ratis.conf.RaftProperties; import org.apache.ratis.proto.RaftProtos.SpanContextProto; -import org.apache.ratis.protocol.RaftClientRequest; import org.apache.ratis.util.JavaUtils; import org.apache.ratis.util.function.CheckedSupplier; import org.apache.ratis.util.VersionInfo; @@ -38,27 +37,58 @@ import java.util.Optional; import java.util.TreeMap; import java.util.concurrent.CompletableFuture; +import java.util.concurrent.atomic.AtomicReference; import java.util.function.BiConsumer; import java.util.function.Supplier; +/** Common OpenTelemetry utilities shared by {@link TraceClient} and {@link TraceServer}. */ public final class TraceUtils { - private static final Tracer TRACER = GlobalOpenTelemetry.getTracer("org.apache.ratis", - VersionInfo.getSoftwareInfoVersion()); - + private static final AtomicReference TRACER = new AtomicReference<>(); private static final Logger LOG = LoggerFactory.getLogger(TraceUtils.class); private TraceUtils() { } public static Tracer getGlobalTracer() { - return TRACER; + return TRACER.get(); + } + + /** + * Initializes the global tracer from configuration when tracing is enabled, or clears it when + * disabled. Call from {@link org.apache.ratis.server.RaftServer} and + * {@link org.apache.ratis.client.RaftClient} construction so tracing follows + * {@link TraceConfigKeys}. + * + * @param properties raft configuration; tracing is on when {@link TraceConfigKeys#enabled} is true + */ + public static void setTracerWhenEnabled(RaftProperties properties) { + setTracerWhenEnabled(TraceConfigKeys.enabled(properties)); + } + + /** + * Enables or disables the tracer without reading {@link RaftProperties}. Intended for tests and + * simple toggles; production code should prefer {@link #setTracerWhenEnabled(RaftProperties)}. + * + * @param enabled when true, lazily obtains the OpenTelemetry tracer; when false, clears it + */ + public static void setTracerWhenEnabled(boolean enabled) { + if (enabled) { + TRACER.updateAndGet(previous -> previous != null ? previous + : GlobalOpenTelemetry.getTracer("org.apache.ratis", VersionInfo.getSoftwareInfoVersion())); + } else { + TRACER.set(null); + } + } + + static boolean isEnabled() { + return TRACER.get() != null; } /** - * Trace an asynchronous operation represented by a {@link CompletableFuture}. - * The returned future will complete with the same result or error as the original future, - * but the provided {@code span} will be ended when the future completes. + * Traces an asynchronous operation represented by a {@link CompletableFuture}. The returned future + * completes with the same outcome as the supplied future; the span is ended when that future + * completes. */ static CompletableFuture traceAsyncMethod( CheckedSupplier, THROWABLE> action, Supplier spanSupplier) throws THROWABLE { @@ -81,40 +111,23 @@ static CompletableFuture traceAsyncMethod( } } - public static CompletableFuture traceAsyncMethod( - CheckedSupplier, THROWABLE> action, - RaftClientRequest request, String memberId, String spanName) throws THROWABLE { - return traceAsyncMethod(action, () -> createServerSpanFromClientRequest(request, memberId, spanName)); - } - - public static CompletableFuture traceAsyncMethodIfEnabled( - boolean enabled, - CheckedSupplier, THROWABLE> action, - RaftClientRequest request, String memberId, String spanName) throws THROWABLE { - return enabled ? traceAsyncMethod(action, request, memberId, spanName) : action.get(); - } - - private static Span createServerSpanFromClientRequest(RaftClientRequest request, String memberId, String spanName) { - final Context remoteContext = extractContextFromProto(request.getSpanContext()); - final Span span = getGlobalTracer() - .spanBuilder(spanName) - .setParent(remoteContext) - .setSpanKind(SpanKind.SERVER) - .startSpan(); - span.setAttribute(RatisAttributes.CLIENT_ID, String.valueOf(request.getClientId())); - span.setAttribute(RatisAttributes.CALL_ID, String.valueOf(request.getCallId())); - span.setAttribute(RatisAttributes.MEMBER_ID, memberId); - return span; - } - private static void endSpan(CompletableFuture future, Span span) { + if (span == null) { + LOG.debug("Span is null, cannot trace the future {}", future); + return; + } addListener(future, (resp, error) -> { - if (error != null) { - setError(span, error); - } else { - span.setStatus(StatusCode.OK); + try { + if (error != null) { + setError(span, error); + } else { + span.setStatus(StatusCode.OK); + } + } catch (Throwable t) { + LOG.error("Error setting span status, ending span anyway", t); + } finally { + span.end(); } - span.end(); }); } @@ -140,12 +153,8 @@ private static void addListener(CompletableFuture future, BiConsumer action) { future.whenComplete((resp, error) -> { try { - // See this post on stack overflow(shorten since the url is too long), - // https://s.apache.org/completionexception - // For a chain of CompletableFuture, only the first child CompletableFuture can get the - // original exception, others will get a CompletionException, which wraps the original - // exception. So here we unwrap it before passing it to the callback action. - action.accept(resp, JavaUtils.unwrapCompletionException(error)); + // https://s.apache.org/completionexception — unwrap CompletionException for callers + action.accept(resp, error == null ? null : JavaUtils.unwrapCompletionException(error)); } catch (Throwable t) { LOG.error("Unexpected error caught when processing CompletableFuture", t); } @@ -184,4 +193,4 @@ public String get(SpanContextProto carrier, String key) { .map(map -> map.get(key)).orElse(null); } -} \ No newline at end of file +} diff --git a/ratis-common/src/test/java/org/apache/ratis/trace/TestTraceUtils.java b/ratis-common/src/test/java/org/apache/ratis/trace/TestTraceUtils.java new file mode 100644 index 0000000000..71f0f25643 --- /dev/null +++ b/ratis-common/src/test/java/org/apache/ratis/trace/TestTraceUtils.java @@ -0,0 +1,66 @@ +/* + * 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.ratis.trace; + +import static org.junit.jupiter.api.Assertions.assertFalse; +import static org.junit.jupiter.api.Assertions.assertTrue; + +import io.opentelemetry.api.trace.SpanKind; +import io.opentelemetry.sdk.testing.junit5.OpenTelemetryExtension; +import io.opentelemetry.sdk.trace.data.SpanData; +import org.apache.ratis.protocol.RaftPeerId; +import org.junit.jupiter.api.Test; +import org.junit.jupiter.api.extension.RegisterExtension; + +import java.util.List; +import java.util.concurrent.CompletableFuture; + +public class TestTraceUtils { + + @RegisterExtension + private static final OpenTelemetryExtension openTelemetryExtension = + OpenTelemetryExtension.create(); + + private void runTraceAsyncAndAssertClientSpan(boolean tracingEnabled, boolean expectClientSpan) + throws Exception { + TraceUtils.setTracerWhenEnabled(tracingEnabled); + TraceClient.asyncSend( + () -> CompletableFuture.completedFuture("ok"), + null, + RaftPeerId.valueOf("s0") + ).get(); + + List spans = openTelemetryExtension.getSpans(); + boolean hasClientSpan = spans.stream().anyMatch(s -> s.getKind() == SpanKind.CLIENT); + if (expectClientSpan) { + assertTrue(hasClientSpan, "Expected CLIENT span from traceAsyncRpcSend, got: " + spans); + } else { + assertFalse(hasClientSpan, "Expected no CLIENT span when tracing disabled, got: " + spans); + } + } + + @Test + public void testTraceAsyncRpcSendCreatesClientSpan() throws Exception { + runTraceAsyncAndAssertClientSpan(true, true); + } + + @Test + public void testTraceAsyncRpcSendCreatesClientSpanDisabled() throws Exception { + runTraceAsyncAndAssertClientSpan(false, false); + } +} diff --git a/ratis-server/src/main/java/org/apache/ratis/server/impl/RaftServerImpl.java b/ratis-server/src/main/java/org/apache/ratis/server/impl/RaftServerImpl.java index 0b891f6589..958da846d2 100644 --- a/ratis-server/src/main/java/org/apache/ratis/server/impl/RaftServerImpl.java +++ b/ratis-server/src/main/java/org/apache/ratis/server/impl/RaftServerImpl.java @@ -100,8 +100,8 @@ import org.apache.ratis.statemachine.impl.TransactionContextImpl; import org.apache.ratis.thirdparty.com.google.common.annotations.VisibleForTesting; import org.apache.ratis.thirdparty.com.google.protobuf.InvalidProtocolBufferException; +import org.apache.ratis.trace.TraceServer; import org.apache.ratis.trace.TraceUtils; -import org.apache.ratis.trace.TraceConfigKeys; import org.apache.ratis.util.CodeInjectionForTesting; import org.apache.ratis.util.CollectionUtils; import org.apache.ratis.util.ConcurrentUtils; @@ -258,8 +258,6 @@ public long[] getFollowerMatchIndices() { private final ExecutorService serverExecutor; private final ExecutorService clientExecutor; - private final boolean tracingEnabled; - private final AtomicBoolean firstElectionSinceStartup = new AtomicBoolean(true); private final ThreadGroup threadGroup; @@ -286,7 +284,7 @@ public long[] getFollowerMatchIndices() { this.readOption = RaftServerConfigKeys.Read.option(properties); this.writeIndexCache = new WriteIndexCache(properties); this.transactionManager = new TransactionManager(id); - this.tracingEnabled = TraceConfigKeys.enabled(properties); + TraceUtils.setTracerWhenEnabled(properties); this.leaderElectionMetrics = LeaderElectionMetrics.getLeaderElectionMetrics( getMemberId(), state::getLastLeaderElapsedTimeMs); @@ -948,8 +946,7 @@ CompletableFuture executeSubmitClientRequestAsync(RaftClientReq @Override public CompletableFuture submitClientRequestAsync( RaftClientRequest request) throws IOException { - return TraceUtils.traceAsyncMethodIfEnabled( - tracingEnabled, + return TraceServer.traceAsyncMethod( () -> submitClientRequestAsyncInternal(request), request, getMemberId().toString(), "raft.server.submitClientRequestAsync"); } diff --git a/ratis-test/src/test/java/org/apache/ratis/grpc/TestRaftAsyncWithGrpcTracing.java b/ratis-test/src/test/java/org/apache/ratis/grpc/TestRaftAsyncWithGrpcTracing.java new file mode 100644 index 0000000000..d7c619cc95 --- /dev/null +++ b/ratis-test/src/test/java/org/apache/ratis/grpc/TestRaftAsyncWithGrpcTracing.java @@ -0,0 +1,75 @@ +/* + * 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.ratis.grpc; + +import static org.junit.jupiter.api.Assertions.assertTrue; + +import io.opentelemetry.api.trace.SpanKind; +import io.opentelemetry.sdk.testing.junit5.OpenTelemetryExtension; +import io.opentelemetry.sdk.trace.data.SpanData; +import org.apache.ratis.RaftAsyncTests; +import org.apache.ratis.trace.TraceConfigKeys; +import org.junit.jupiter.api.Test; +import org.junit.jupiter.api.Timeout; +import org.junit.jupiter.api.extension.RegisterExtension; + +import java.util.List; + +@Timeout(100) +public class TestRaftAsyncWithGrpcTracing extends RaftAsyncTests + implements MiniRaftClusterWithGrpc.FactoryGet { + { + TraceConfigKeys.setEnabled(getProperties(), true); + } + + @RegisterExtension + private static final OpenTelemetryExtension openTelemetryExtension = + OpenTelemetryExtension.create(); + + /** + * Verifies traceAsyncRpcSend creates CLIENT spans when tracing is enabled. + * testBasicAppendEntriesAsync uses client.async().send() which goes through AsyncImpl.send(). + */ + @Test + public void testBasicAppendEntriesAsync() throws Exception { + super.testBasicAppendEntriesAsync(); + List spans = openTelemetryExtension.getSpans(); + assertTrue( + spans.stream().anyMatch(s -> s.getKind() == SpanKind.CLIENT), + "Expected at least one span with SpanKind.CLIENT (from traceAsyncRpcSend)" + ); + assertTrue( + spans.stream().anyMatch(s -> s.getKind() == SpanKind.SERVER), + "Expected at least one span with SpanKind.SERVER" + ); + } + + @Test + public void testWithLoadAsync() throws Exception { + super.testWithLoadAsync(); + List spans = openTelemetryExtension.getSpans(); + assertTrue( + spans.stream().anyMatch(s -> s.getKind() == SpanKind.CLIENT), + "Expected at least one span with SpanKind.CLIENT (from traceAsyncRpcSend)" + ); + assertTrue( + spans.stream().anyMatch(s -> s.getKind() == SpanKind.SERVER), + "Expected at least one span with SpanKind.SERVER" + ); + } +} \ No newline at end of file diff --git a/ratis-test/src/test/java/org/apache/ratis/grpc/TestRetryCacheWithGrpcTracing.java b/ratis-test/src/test/java/org/apache/ratis/grpc/TestRetryCacheWithGrpcTracing.java new file mode 100644 index 0000000000..722eae2cc5 --- /dev/null +++ b/ratis-test/src/test/java/org/apache/ratis/grpc/TestRetryCacheWithGrpcTracing.java @@ -0,0 +1,81 @@ +/* + * 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.ratis.grpc; + +import static org.junit.jupiter.api.Assertions.assertTrue; + +import io.opentelemetry.api.GlobalOpenTelemetry; +import io.opentelemetry.api.trace.SpanKind; +import io.opentelemetry.sdk.testing.junit5.OpenTelemetryExtension; +import io.opentelemetry.sdk.trace.data.SpanData; +import org.apache.ratis.trace.TraceConfigKeys; +import org.junit.jupiter.api.AfterEach; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.Test; +import org.junit.jupiter.api.extension.RegisterExtension; + +import java.util.List; + +public class TestRetryCacheWithGrpcTracing + extends TestRetryCacheWithGrpc { + @RegisterExtension + private static final OpenTelemetryExtension openTelemetryExtension = + OpenTelemetryExtension.create(); + { + TraceConfigKeys.setEnabled(getProperties(), true); + } + + private List spans; + + @BeforeEach + void setUpOpenTelemetry() { + GlobalOpenTelemetry.resetForTest(); + GlobalOpenTelemetry.set(openTelemetryExtension.getOpenTelemetry()); + } + + @AfterEach + void tearDownOpenTelemetry() { + GlobalOpenTelemetry.resetForTest(); + } + + /** + * Verifies traceAsyncRpcSend creates CLIENT spans when tracing is enabled. + * Uses testInvalidateRepliedCalls which exercises client.async().send() (traceAsyncRpcSend path). + * testBasicRetry uses rpc.sendRequest() (blocking) which bypasses the async tracing path. + */ + @Test + public void testBasicRetry() throws Exception { + runWithNewCluster(3, cluster -> new InvalidateRepliedCallsTest(cluster).run()); + + long deadline = System.currentTimeMillis() + 10000; + do { + spans = openTelemetryExtension.getSpans(); + if (!spans.isEmpty()) break; + Thread.sleep(100); + } while (System.currentTimeMillis() < deadline); + + assertTrue( + spans.stream().anyMatch(s -> s.getKind() == SpanKind.CLIENT), + "Expected at least one span with SpanKind.CLIENT (from traceAsyncRpcSend)" + ); + assertTrue( + spans.stream().anyMatch(s -> s.getKind() == SpanKind.SERVER), + "Expected at least one span with SpanKind.SERVER" + ); + } +} From e80140381dda66e5b0c505ab22e4864867140ae4 Mon Sep 17 00:00:00 2001 From: "Doroszlai, Attila" <6454655+adoroszlai@users.noreply.github.com> Date: Fri, 27 Mar 2026 08:22:00 +0100 Subject: [PATCH 285/397] RATIS-2446. Enable Dependabot dependency updates (#1386) --- .github/dependabot.yml | 34 ++++++++++++++++++++++++++++++++++ 1 file changed, 34 insertions(+) create mode 100644 .github/dependabot.yml diff --git a/.github/dependabot.yml b/.github/dependabot.yml new file mode 100644 index 0000000000..feb09054aa --- /dev/null +++ b/.github/dependabot.yml @@ -0,0 +1,34 @@ +# 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. + +version: 2 + +updates: + - package-ecosystem: "github-actions" + directory: "/" + schedule: + # 'daily' only runs on weekdays + interval: "cron" + cronjob: "15 9 * * *" + cooldown: + default-days: 4 + - package-ecosystem: "maven" + directory: "/" + schedule: + # 'daily' only runs on weekdays + interval: "cron" + cronjob: "15 10 * * *" + cooldown: + default-days: 4 From d133eff70263b332cc886ea9956ba69e8822d19a Mon Sep 17 00:00:00 2001 From: "dependabot[bot]" <49699333+dependabot[bot]@users.noreply.github.com> Date: Fri, 27 Mar 2026 09:14:09 +0100 Subject: [PATCH 286/397] RATIS-2451. Bump hadoop-maven-plugins to 3.4.3 (#1388) --- pom.xml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/pom.xml b/pom.xml index 14a35a7d12..6c9d082571 100644 --- a/pom.xml +++ b/pom.xml @@ -149,7 +149,7 @@ 5.1.8 3.5.3 - 3.4.2 + 3.4.3 5.3.0.6276 From ddd5379852f279270dfb2607df069d129ee924bc Mon Sep 17 00:00:00 2001 From: "dependabot[bot]" <49699333+dependabot[bot]@users.noreply.github.com> Date: Fri, 27 Mar 2026 09:29:15 +0100 Subject: [PATCH 287/397] RATIS-2450. Bump common-custom-user-data-maven-extension to 2.1.0 (#1389) --- .mvn/extensions.xml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/.mvn/extensions.xml b/.mvn/extensions.xml index 8ceede33b9..a32ab684b6 100644 --- a/.mvn/extensions.xml +++ b/.mvn/extensions.xml @@ -29,6 +29,6 @@ com.gradle common-custom-user-data-maven-extension - 2.0.1 + 2.1.0 From c8fb9e199c7993dca8239ee61a1977f4f06b4ba4 Mon Sep 17 00:00:00 2001 From: "dependabot[bot]" <49699333+dependabot[bot]@users.noreply.github.com> Date: Fri, 27 Mar 2026 10:22:59 +0100 Subject: [PATCH 288/397] RATIS-2452. Bump extra-enforcer-rules to 1.12.0 (#1390) --- pom.xml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/pom.xml b/pom.xml index 6c9d082571..d08f874fed 100644 --- a/pom.xml +++ b/pom.xml @@ -155,7 +155,7 @@ 3.6.1 3.6.2 - 1.11.0 + 1.12.0 3.0.0 2.7.0 From 3abfd8278641ca0e4af4c2d59f077be13e90c77a Mon Sep 17 00:00:00 2001 From: "dependabot[bot]" <49699333+dependabot[bot]@users.noreply.github.com> Date: Fri, 27 Mar 2026 13:43:33 +0100 Subject: [PATCH 289/397] RATIS-2459. Bump dropwizard3 to 3.2.6 (#1394) --- ratis-metrics-dropwizard3/pom.xml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/ratis-metrics-dropwizard3/pom.xml b/ratis-metrics-dropwizard3/pom.xml index 3d06f57cc5..2dd329858e 100644 --- a/ratis-metrics-dropwizard3/pom.xml +++ b/ratis-metrics-dropwizard3/pom.xml @@ -25,7 +25,7 @@ Apache Ratis Metrics Dropwizard 3 Implementation - 3.2.5 + 3.2.6 From 6a0b30be5d4c0203e1c89a975ebe6e53aaabb56c Mon Sep 17 00:00:00 2001 From: "dependabot[bot]" <49699333+dependabot[bot]@users.noreply.github.com> Date: Fri, 27 Mar 2026 14:11:49 +0100 Subject: [PATCH 290/397] RATIS-2460. Bump bouncycastle to 1.83 (#1393) --- ratis-test/pom.xml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/ratis-test/pom.xml b/ratis-test/pom.xml index 854c2b6068..050600319c 100644 --- a/ratis-test/pom.xml +++ b/ratis-test/pom.xml @@ -24,7 +24,7 @@ Apache Ratis Test - 1.79 + 1.83 From 477655bb25d712dd2a14ab59b632ede83eedebf1 Mon Sep 17 00:00:00 2001 From: Abhishek Pal Date: Sat, 28 Mar 2026 09:48:39 +0530 Subject: [PATCH 291/397] RATIS-2432. Close PRs after three months of inactivity (#1373) --- .github/workflows/close-stale-pr.yaml | 39 +++++++++++++++++++++++++++ 1 file changed, 39 insertions(+) create mode 100644 .github/workflows/close-stale-pr.yaml diff --git a/.github/workflows/close-stale-pr.yaml b/.github/workflows/close-stale-pr.yaml new file mode 100644 index 0000000000..6c24bf0e51 --- /dev/null +++ b/.github/workflows/close-stale-pr.yaml @@ -0,0 +1,39 @@ +# 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. +name: close-stale-prs + +on: + schedule: + # We can run this only once a week on Sundays so contributors get sufficient time to follow up. + - cron: '0 0 * * 0' + +jobs: + close-stale-prs: + permissions: + pull-requests: write + runs-on: ubuntu-slim + steps: + - name: Close Stale PRs + uses: actions/stale@v10 + with: + stale-pr-label: 'stale' + exempt-draft-pr: false + days-before-issue-stale: -1 + days-before-pr-stale: 60 + days-before-pr-close: 30 + remove-pr-stale-when-updated: true + operations-per-run: 500 + stale-pr-message: 'This PR has been marked as stale due to 60 days of inactivity. Please comment or remove the stale label to keep it open. Otherwise, it will be automatically closed in ~30 days.' + close-pr-message: 'Thank you for your contribution. This PR is being closed due to inactivity. Please contact a maintainer if you would like to reopen it.' From 152520f1d2b2dc537d85ce7a32ca9f07d0b575ef Mon Sep 17 00:00:00 2001 From: Abhishek Pal Date: Sat, 28 Mar 2026 09:54:20 +0530 Subject: [PATCH 292/397] RATIS-244. Skip snapshot file if corresponding MD5 file is missing (#1320) --- .../impl/SimpleStateMachineStorage.java | 76 +++-- .../impl/SingleFileSnapshotInfo.java | 5 + .../ratis/server/storage/TestRaftStorage.java | 303 +++++++++++++++++- 3 files changed, 340 insertions(+), 44 deletions(-) diff --git a/ratis-server/src/main/java/org/apache/ratis/statemachine/impl/SimpleStateMachineStorage.java b/ratis-server/src/main/java/org/apache/ratis/statemachine/impl/SimpleStateMachineStorage.java index 0ca6734a07..5322778995 100644 --- a/ratis-server/src/main/java/org/apache/ratis/statemachine/impl/SimpleStateMachineStorage.java +++ b/ratis-server/src/main/java/org/apache/ratis/statemachine/impl/SimpleStateMachineStorage.java @@ -39,7 +39,6 @@ import java.nio.file.Path; import java.util.ArrayList; import java.util.Comparator; -import java.util.Iterator; import java.util.List; import java.util.Objects; import java.util.Optional; @@ -82,6 +81,9 @@ public void format() throws IOException { // TODO } + /** + * Fetch all the snapshot files irrespective of whether they have an MD5 file or not + */ static List getSingleFileSnapshotInfos(Path dir) throws IOException { final List infos = new ArrayList<>(); try (DirectoryStream stream = Files.newDirectoryStream(dir)) { @@ -92,7 +94,8 @@ static List getSingleFileSnapshotInfos(Path dir) throws if (matcher.matches()) { final long term = Long.parseLong(matcher.group(1)); final long index = Long.parseLong(matcher.group(2)); - final FileInfo fileInfo = new FileInfo(path, null); //No FileDigest here. + final MD5Hash md5 = MD5FileUtil.readStoredMd5ForFile(path.toFile()); + final FileInfo fileInfo = new FileInfo(path, md5); infos.add(new SingleFileSnapshotInfo(fileInfo, term, index)); } } @@ -115,10 +118,25 @@ public void cleanupOldSnapshots(SnapshotRetentionPolicy snapshotRetentionPolicy) } final List allSnapshotFiles = getSingleFileSnapshotInfos(stateMachineDir.toPath()); + allSnapshotFiles.sort(Comparator.comparing(SingleFileSnapshotInfo::getIndex).reversed()); + int numSnapshotsWithMd5 = 0; + int deleteIdx = -1; - if (allSnapshotFiles.size() > numSnapshotsRetained) { - allSnapshotFiles.sort(Comparator.comparing(SingleFileSnapshotInfo::getIndex).reversed()); - allSnapshotFiles.subList(numSnapshotsRetained, allSnapshotFiles.size()) + for (int i = 0; i < allSnapshotFiles.size(); i++) { + final SingleFileSnapshotInfo snapshot = allSnapshotFiles.get(i); + if (snapshot.hasMd5()) { + if (++numSnapshotsWithMd5 == numSnapshotsRetained) { + // We have found the last snapshot with an MD5 file that needs to be retained + deleteIdx = i + 1; + break; + } + } else { + LOG.warn("Snapshot file {} has missing MD5 file.", snapshot); + } + } + + if (deleteIdx > 0) { + allSnapshotFiles.subList(deleteIdx, allSnapshotFiles.size()) .stream() .map(SingleFileSnapshotInfo::getFile) .map(FileInfo::getPath) @@ -126,20 +144,21 @@ public void cleanupOldSnapshots(SnapshotRetentionPolicy snapshotRetentionPolicy) LOG.info("Deleting old snapshot at {}", snapshotPath.toAbsolutePath()); FileUtils.deletePathQuietly(snapshotPath); }); - // clean up the md5 files if the corresponding snapshot file does not exist - try (DirectoryStream stream = Files.newDirectoryStream(stateMachineDir.toPath(), - SNAPSHOT_MD5_FILTER)) { - for (Path md5path : stream) { - Path md5FileNamePath = md5path.getFileName(); - if (md5FileNamePath == null) { - continue; - } - final String md5FileName = md5FileNamePath.toString(); - final File snapshotFile = new File(stateMachineDir, - md5FileName.substring(0, md5FileName.length() - MD5_SUFFIX.length())); - if (!snapshotFile.exists()) { - FileUtils.deletePathQuietly(md5path); - } + } + + // clean up the md5 files if the corresponding snapshot file does not exist + try (DirectoryStream stream = Files.newDirectoryStream(stateMachineDir.toPath(), + SNAPSHOT_MD5_FILTER)) { + for (Path md5path : stream) { + Path md5FileNamePath = md5path.getFileName(); + if (md5FileNamePath == null) { + continue; + } + final String md5FileName = md5FileNamePath.toString(); + final File snapshotFile = new File(stateMachineDir, + md5FileName.substring(0, md5FileName.length() - MD5_SUFFIX.length())); + if (!snapshotFile.exists()) { + FileUtils.deletePathQuietly(md5path); } } } @@ -182,24 +201,19 @@ protected File getCorruptSnapshotFile(long term, long endIndex) { } static SingleFileSnapshotInfo findLatestSnapshot(Path dir) throws IOException { - final Iterator i = getSingleFileSnapshotInfos(dir).iterator(); - if (!i.hasNext()) { + final List infos = getSingleFileSnapshotInfos(dir); + if (infos.isEmpty()) { return null; } + infos.sort(Comparator.comparing(SingleFileSnapshotInfo::getIndex).reversed()); - SingleFileSnapshotInfo latest = i.next(); - for(; i.hasNext(); ) { - final SingleFileSnapshotInfo info = i.next(); - if (info.getIndex() > latest.getIndex()) { - latest = info; + for (SingleFileSnapshotInfo latest : infos) { + if (latest.hasMd5()) { + return latest; } } - // read md5 - final Path path = latest.getFile().getPath(); - final MD5Hash md5 = MD5FileUtil.readStoredMd5ForFile(path.toFile()); - final FileInfo info = new FileInfo(path, md5); - return new SingleFileSnapshotInfo(info, latest.getTerm(), latest.getIndex()); + return infos.get(0); // all snapshots do not have MD5 } public SingleFileSnapshotInfo updateLatestSnapshot(SingleFileSnapshotInfo info) { diff --git a/ratis-server/src/main/java/org/apache/ratis/statemachine/impl/SingleFileSnapshotInfo.java b/ratis-server/src/main/java/org/apache/ratis/statemachine/impl/SingleFileSnapshotInfo.java index 14d501a4af..5ecc59a102 100644 --- a/ratis-server/src/main/java/org/apache/ratis/statemachine/impl/SingleFileSnapshotInfo.java +++ b/ratis-server/src/main/java/org/apache/ratis/statemachine/impl/SingleFileSnapshotInfo.java @@ -36,6 +36,11 @@ public SingleFileSnapshotInfo(FileInfo fileInfo, long term, long endIndex) { this(fileInfo, TermIndex.valueOf(term, endIndex)); } + /** @return true iff the MD5 exists. */ + public boolean hasMd5() { + return getFile().getFileDigest() != null; + } + /** @return the file associated with the snapshot. */ public FileInfo getFile() { return getFiles().get(0); diff --git a/ratis-test/src/test/java/org/apache/ratis/server/storage/TestRaftStorage.java b/ratis-test/src/test/java/org/apache/ratis/server/storage/TestRaftStorage.java index 12cd771315..093e9add7f 100644 --- a/ratis-test/src/test/java/org/apache/ratis/server/storage/TestRaftStorage.java +++ b/ratis-test/src/test/java/org/apache/ratis/server/storage/TestRaftStorage.java @@ -18,6 +18,7 @@ package org.apache.ratis.server.storage; import static java.util.stream.Collectors.toList; +import static org.apache.ratis.statemachine.impl.SimpleStateMachineStorage.SNAPSHOT_MD5_REGEX; import static org.apache.ratis.statemachine.impl.SimpleStateMachineStorage.SNAPSHOT_REGEX; import static org.apache.ratis.util.MD5FileUtil.MD5_SUFFIX; @@ -29,7 +30,9 @@ import org.apache.ratis.server.storage.RaftStorageDirectoryImpl.StorageState; import org.apache.ratis.statemachine.impl.SimpleStateMachineStorage; import org.apache.ratis.statemachine.SnapshotRetentionPolicy; +import org.apache.ratis.statemachine.impl.SingleFileSnapshotInfo; import org.apache.ratis.util.FileUtils; +import org.apache.ratis.util.MD5FileUtil; import org.apache.ratis.util.Preconditions; import org.apache.ratis.util.SizeInBytes; import org.junit.jupiter.api.AfterEach; @@ -40,10 +43,13 @@ import java.io.File; import java.io.IOException; +import java.nio.charset.StandardCharsets; +import java.nio.file.Files; import java.util.Arrays; import java.util.Collections; import java.util.HashSet; import java.util.List; +import java.util.Objects; import java.util.Set; import java.util.concurrent.ThreadLocalRandom; import java.util.concurrent.atomic.AtomicReference; @@ -228,7 +234,7 @@ public void testSnapshotCleanup() throws IOException { SnapshotRetentionPolicy snapshotRetentionPolicy = new SnapshotRetentionPolicy() { @Override public int getNumSnapshotsRetained() { - return 3; + return 2; } }; @@ -239,15 +245,24 @@ public int getNumSnapshotsRetained() { Set termIndexSet = new HashSet<>(); - //Create 5 snapshot files in storage dir. - while (termIndexSet.size() < 5) { + //Create 3 snapshot files in storage dir. + while (termIndexSet.size() < 3) { final long term = ThreadLocalRandom.current().nextLong(1, 10L); - final long index = ThreadLocalRandom.current().nextLong(100, 1000L); + final long index = ThreadLocalRandom.current().nextLong(100, 500L); if (termIndexSet.add(TermIndex.valueOf(term, index))) { - File file = simpleStateMachineStorage.getSnapshotFile(term, index); - Assertions.assertTrue(file.createNewFile()); + createSnapshot(simpleStateMachineStorage, term, index, true); + } + } + + // Create 2 more snapshot files in storage dir without MD5 files + while (termIndexSet.size() < 5) { + final long term = ThreadLocalRandom.current().nextLong(11, 20L); + final long index = ThreadLocalRandom.current().nextLong(501, 1000L); + if (termIndexSet.add(TermIndex.valueOf(term, index))) { + createSnapshot(simpleStateMachineStorage, term, index, false); } } + // create MD5 files that will not be deleted in older version while (termIndexSet.size() < 7) { final long term = 1; @@ -260,16 +275,18 @@ public int getNumSnapshotsRetained() { } File stateMachineDir = storage.getStorageDir().getStateMachineDir(); - assertFileCount(stateMachineDir, 7); + assertFileCount(stateMachineDir, 10); simpleStateMachineStorage.cleanupOldSnapshots(snapshotRetentionPolicy); - File[] remainingFiles = assertFileCount(stateMachineDir, 3); + // Since the MD5 files are not matching the snapshot files they are cleaned up. + // So we still have 6 files - 4 snapshots and 2 MD5 files. + File[] remainingFiles = assertFileCount(stateMachineDir, 6); List remainingIndices = termIndexSet.stream() .map(TermIndex::getIndex) .sorted(Collections.reverseOrder()) - .limit(3) + .limit(4) .collect(toList()); for (File file : remainingFiles) { System.out.println(file.getName()); @@ -281,21 +298,262 @@ public int getNumSnapshotsRetained() { // Attempt to clean up again should not delete any more files. simpleStateMachineStorage.cleanupOldSnapshots(snapshotRetentionPolicy); - assertFileCount(stateMachineDir, 3); + assertFileCount(stateMachineDir, 6); //Test with Retention disabled. //Create 2 snapshot files in storage dir. for (int i = 0; i < 2; i++) { - final long term = ThreadLocalRandom.current().nextLong(1, 10L); + final long term = ThreadLocalRandom.current().nextLong(21, 30L); final long index = ThreadLocalRandom.current().nextLong(1000L); - File file = simpleStateMachineStorage.getSnapshotFile(term, index); - Assertions.assertTrue(file.createNewFile()); + createSnapshot(simpleStateMachineStorage, term, index, false); } simpleStateMachineStorage.cleanupOldSnapshots(new SnapshotRetentionPolicy() { }); + assertFileCount(stateMachineDir, 8); + } + + @Test + public void testSnapshotCleanupWithMissingMd5File() throws IOException { + + SnapshotRetentionPolicy snapshotRetentionPolicy = new SnapshotRetentionPolicy() { + @Override + public int getNumSnapshotsRetained() { + return 2; + } + }; + + + SimpleStateMachineStorage simpleStateMachineStorage = new SimpleStateMachineStorage(); + final RaftStorage storage = newRaftStorage(storageDir); + simpleStateMachineStorage.init(storage); + + Set termIndexSet = new HashSet<>(); + + // Create one snapshot file without MD5 file + if (termIndexSet.add(TermIndex.valueOf(1, 100))) { + createSnapshot(simpleStateMachineStorage, 1, 100, false); + } + + //Create 4 snapshot files in storage dir + while (termIndexSet.size() < 5) { + final long term = ThreadLocalRandom.current().nextLong(2, 10L); + final long index = ThreadLocalRandom.current().nextLong(100, 1000L); + if (termIndexSet.add(TermIndex.valueOf(term, index))) { + createSnapshot(simpleStateMachineStorage, term, index, true); + } + } + + // 1 snapshot file without MD5 hash, 4 snapshots + 4 md5 hash files = 9 files + File stateMachineDir = storage.getStorageDir().getStateMachineDir(); + assertFileCount(stateMachineDir, 9); + + simpleStateMachineStorage.cleanupOldSnapshots(snapshotRetentionPolicy); + + // We should have 4 files remaining, and 2 snapshots with MD5 hash + assertFileCount(stateMachineDir, 4); + } + + @Test + public void testSnapshotCleanupWithLatestSnapshotMissingMd5File() throws IOException { + + SnapshotRetentionPolicy snapshotRetentionPolicy = new SnapshotRetentionPolicy() { + @Override + public int getNumSnapshotsRetained() { + return 2; + } + }; + + + SimpleStateMachineStorage simpleStateMachineStorage = new SimpleStateMachineStorage(); + final RaftStorage storage = newRaftStorage(storageDir); + simpleStateMachineStorage.init(storage); + + Set termIndexSet = new HashSet<>(); + + //Create 4 snapshot files in storage dir + while (termIndexSet.size() < 4) { + final long term = ThreadLocalRandom.current().nextLong(1, 10L); + final long index = ThreadLocalRandom.current().nextLong(100, 1000L); + if (termIndexSet.add(TermIndex.valueOf(term, index))) { + createSnapshot(simpleStateMachineStorage, term, index, true); + } + } + + // Create a snapshot file with a missing MD5 file and having the highest term index + if (termIndexSet.add(TermIndex.valueOf(99, 1001))) { + createSnapshot(simpleStateMachineStorage, 99, 1001, false); + } + + // 1 snapshot file without MD5 hash, 4 snapshots + 4 md5 hash files = 9 files + File stateMachineDir = storage.getStorageDir().getStateMachineDir(); + assertFileCount(stateMachineDir, 9); + + simpleStateMachineStorage.cleanupOldSnapshots(snapshotRetentionPolicy); + + // We should have 5 files remaining, and 2 snapshots with MD5 hash and 1 snapshot file without MD5 hash assertFileCount(stateMachineDir, 5); } + @Test + public void testCleanupOldSnapshotsDeletesOlderSnapshotsWithMd5() throws Exception { + SnapshotRetentionPolicy snapshotRetentionPolicy = new SnapshotRetentionPolicy() { + @Override + public int getNumSnapshotsRetained() { + return 2; + } + }; + + SimpleStateMachineStorage simpleStateMachineStorage = new SimpleStateMachineStorage(); + final RaftStorage storage = newRaftStorage(storageDir); + simpleStateMachineStorage.init(storage); + try { + createSnapshot(simpleStateMachineStorage, 1, 100, true); + createSnapshot(simpleStateMachineStorage, 1, 200, true); + createSnapshot(simpleStateMachineStorage, 1, 300, true); + createSnapshot(simpleStateMachineStorage, 1, 400, true); + + File stateMachineDir = storage.getStorageDir().getStateMachineDir(); + simpleStateMachineStorage.cleanupOldSnapshots(snapshotRetentionPolicy); + + List snapshotNames = listMatchingFileNames(stateMachineDir, SNAPSHOT_REGEX); + Assertions.assertEquals(2, snapshotNames.size()); + Assertions.assertTrue(snapshotNames.contains(SimpleStateMachineStorage.getSnapshotFileName(1, 400))); + Assertions.assertTrue(snapshotNames.contains(SimpleStateMachineStorage.getSnapshotFileName(1, 300))); + Assertions.assertFalse(snapshotNames.contains(SimpleStateMachineStorage.getSnapshotFileName(1, 200))); + Assertions.assertFalse(snapshotNames.contains(SimpleStateMachineStorage.getSnapshotFileName(1, 100))); + + List md5Names = listMatchingFileNames(stateMachineDir, SNAPSHOT_MD5_REGEX); + Assertions.assertEquals(2, md5Names.size()); + Assertions.assertTrue(md5Names.contains(SimpleStateMachineStorage.getSnapshotFileName(1, 400) + MD5_SUFFIX)); + Assertions.assertTrue(md5Names.contains(SimpleStateMachineStorage.getSnapshotFileName(1, 300) + MD5_SUFFIX)); + Assertions.assertFalse(md5Names.contains(SimpleStateMachineStorage.getSnapshotFileName(1, 200) + MD5_SUFFIX)); + Assertions.assertFalse(md5Names.contains(SimpleStateMachineStorage.getSnapshotFileName(1, 100) + MD5_SUFFIX)); + } finally { + storage.close(); + } + } + + @Test + public void testCleanupOldSnapshotsWithoutAnyMd5() throws Exception { + SnapshotRetentionPolicy snapshotRetentionPolicy = new SnapshotRetentionPolicy() { + @Override + public int getNumSnapshotsRetained() { + return 2; + } + }; + + SimpleStateMachineStorage simpleStateMachineStorage = new SimpleStateMachineStorage(); + final RaftStorage storage = newRaftStorage(storageDir); + simpleStateMachineStorage.init(storage); + try { + createSnapshot(simpleStateMachineStorage, 1, 100, false); + createSnapshot(simpleStateMachineStorage, 1, 200, false); + createSnapshot(simpleStateMachineStorage, 1, 300, false); + + File stateMachineDir = storage.getStorageDir().getStateMachineDir(); + simpleStateMachineStorage.cleanupOldSnapshots(snapshotRetentionPolicy); + + List snapshotNames = listMatchingFileNames(stateMachineDir, SNAPSHOT_REGEX); + Assertions.assertEquals(3, snapshotNames.size()); + Assertions.assertTrue(listMatchingFileNames(stateMachineDir, SNAPSHOT_MD5_REGEX).isEmpty()); + } finally { + storage.close(); + } + } + + @Test + public void testGetLatestSnapshotReturnsNewest() throws Exception { + SimpleStateMachineStorage simpleStateMachineStorage = new SimpleStateMachineStorage(); + final RaftStorage storage = newRaftStorage(storageDir); + simpleStateMachineStorage.init(storage); + try { + Assertions.assertNull(simpleStateMachineStorage.getLatestSnapshot()); + + createSnapshot(simpleStateMachineStorage, 1, 100, true); + simpleStateMachineStorage.loadLatestSnapshot(); + SingleFileSnapshotInfo first = simpleStateMachineStorage.getLatestSnapshot(); + Assertions.assertNotNull(first); + Assertions.assertEquals(1, first.getTerm()); + Assertions.assertEquals(100, first.getIndex()); + Assertions.assertNotNull(first.getFile().getFileDigest()); + + createSnapshot(simpleStateMachineStorage, 1, 200, true); + simpleStateMachineStorage.loadLatestSnapshot(); + SingleFileSnapshotInfo second = simpleStateMachineStorage.getLatestSnapshot(); + Assertions.assertNotNull(second); + Assertions.assertEquals(1, second.getTerm()); + Assertions.assertEquals(200, second.getIndex()); + Assertions.assertNotNull(second.getFile().getFileDigest()); + } finally { + storage.close(); + } + } + + @Test + public void testGetLatestSnapshotIgnoresSnapshotsWithoutMd5() throws Exception { + SimpleStateMachineStorage simpleStateMachineStorage = new SimpleStateMachineStorage(); + final RaftStorage storage = newRaftStorage(storageDir); + simpleStateMachineStorage.init(storage); + try { + createSnapshot(simpleStateMachineStorage, 1, 100, true); + simpleStateMachineStorage.loadLatestSnapshot(); + + createSnapshot(simpleStateMachineStorage, 1, 200, false); + simpleStateMachineStorage.loadLatestSnapshot(); + + SingleFileSnapshotInfo latest = simpleStateMachineStorage.getLatestSnapshot(); + Assertions.assertNotNull(latest); + Assertions.assertEquals(100, latest.getIndex()); + Assertions.assertEquals(1, latest.getTerm()); + } finally { + storage.close(); + } + } + + @Test + public void testGetLatestSnapshotFallsBackToSnapshotWithoutMd5() throws Exception { + SimpleStateMachineStorage simpleStateMachineStorage = new SimpleStateMachineStorage(); + final RaftStorage storage = newRaftStorage(storageDir); + simpleStateMachineStorage.init(storage); + try { + createSnapshot(simpleStateMachineStorage, 1, 100, false); + simpleStateMachineStorage.loadLatestSnapshot(); + + SingleFileSnapshotInfo latest = simpleStateMachineStorage.getLatestSnapshot(); + Assertions.assertNotNull(latest); + Assertions.assertEquals(100, latest.getIndex()); + Assertions.assertEquals(1, latest.getTerm()); + Assertions.assertNull(latest.getFile().getFileDigest()); + } finally { + storage.close(); + } + } + + @Test + public void testGetLatestSnapshotFallsBackWhenNewestMd5IsInvalid() throws Exception { + SimpleStateMachineStorage simpleStateMachineStorage = new SimpleStateMachineStorage(); + final RaftStorage storage = newRaftStorage(storageDir); + simpleStateMachineStorage.init(storage); + try { + createSnapshot(simpleStateMachineStorage, 1, 100, true); + simpleStateMachineStorage.loadLatestSnapshot(); + + File latestSnapshot = createSnapshot(simpleStateMachineStorage, 1, 200, true); + final File latestMd5File = MD5FileUtil.getDigestFileForFile(latestSnapshot); + Files.write(latestMd5File.toPath(), "null".getBytes(StandardCharsets.UTF_8)); + + simpleStateMachineStorage.loadLatestSnapshot(); + + SingleFileSnapshotInfo latest = simpleStateMachineStorage.getLatestSnapshot(); + Assertions.assertNotNull(latest); + Assertions.assertEquals(100, latest.getIndex()); + Assertions.assertEquals(1, latest.getTerm()); + Assertions.assertNotNull(latest.getFile().getFileDigest()); + } finally { + storage.close(); + } + } + private static File[] assertFileCount(File dir, int expected) { File[] files = dir.listFiles(); Assertions.assertNotNull(files); @@ -303,6 +561,25 @@ private static File[] assertFileCount(File dir, int expected) { return files; } + private File createSnapshot(SimpleStateMachineStorage storage, + long term, long endIndex, + boolean withMd5) throws IOException { + File snapshotFile = storage.getSnapshotFile(term, endIndex); + Assertions.assertTrue(snapshotFile.createNewFile()); + + if (withMd5) { + MD5FileUtil.computeAndSaveMd5ForFile(snapshotFile); + } + + return snapshotFile; + } + + private static List listMatchingFileNames(File dir, java.util.regex.Pattern pattern) { + return Arrays.stream(Objects.requireNonNull(dir.list())) + .filter(name -> pattern.matcher(name).matches()) + .collect(toList()); + } + @Test public void testNotEnoughSpace() throws IOException { File mockStorageDir = Mockito.spy(storageDir); From 3d71a3ff35fde4b23c1c8040c311addc032743c8 Mon Sep 17 00:00:00 2001 From: "Doroszlai, Attila" <6454655+adoroszlai@users.noreply.github.com> Date: Sat, 28 Mar 2026 08:09:31 +0100 Subject: [PATCH 293/397] RATIS-2455. Ignore Spotbugs 4.9+ due to Java 11 requirement (#1395) --- .github/dependabot.yml | 6 ++++++ 1 file changed, 6 insertions(+) diff --git a/.github/dependabot.yml b/.github/dependabot.yml index feb09054aa..3e76d886fe 100644 --- a/.github/dependabot.yml +++ b/.github/dependabot.yml @@ -32,3 +32,9 @@ updates: cronjob: "15 10 * * *" cooldown: default-days: 4 + ignore: + # requires Java 11 + - dependency-name: "com.github.spotbugs:spotbugs" + versions: [">=4.9.0"] + - dependency-name: "com.github.spotbugs:spotbugs-maven-plugin" + versions: [">=4.9.0.0"] From 688c5865f59cbe138e79251cb54a9376eafb3264 Mon Sep 17 00:00:00 2001 From: "dependabot[bot]" <49699333+dependabot[bot]@users.noreply.github.com> Date: Sat, 28 Mar 2026 16:28:24 +0100 Subject: [PATCH 294/397] RATIS-2463. Bump slf4j to 2.0.17 (#1399) --- pom.xml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/pom.xml b/pom.xml index d08f874fed..46abee78c4 100644 --- a/pom.xml +++ b/pom.xml @@ -192,7 +192,7 @@ 4 - 2.0.7 + 2.0.17 5.13.3 4.11.0 0.8.12 From 1afb303f53fc100e8545c525b831fb46dd08f923 Mon Sep 17 00:00:00 2001 From: "Doroszlai, Attila" <6454655+adoroszlai@users.noreply.github.com> Date: Sat, 28 Mar 2026 17:00:53 +0100 Subject: [PATCH 295/397] RATIS-2461. Ignore Mockito 5+ due to Java 11 requirement (#1401) --- .github/dependabot.yml | 2 ++ 1 file changed, 2 insertions(+) diff --git a/.github/dependabot.yml b/.github/dependabot.yml index 3e76d886fe..2df7a383ab 100644 --- a/.github/dependabot.yml +++ b/.github/dependabot.yml @@ -38,3 +38,5 @@ updates: versions: [">=4.9.0"] - dependency-name: "com.github.spotbugs:spotbugs-maven-plugin" versions: [">=4.9.0.0"] + - dependency-name: "org.mockito:mockito-core" + versions: [">=5.0.0"] From 23304b414d1d5a509ae8e9e1d8cf415074a79a40 Mon Sep 17 00:00:00 2001 From: "dependabot[bot]" <49699333+dependabot[bot]@users.noreply.github.com> Date: Sat, 28 Mar 2026 17:50:43 +0100 Subject: [PATCH 296/397] RATIS-2466. Bump sonar-maven-plugin to 5.5.0.6356 (#1397) --- pom.xml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/pom.xml b/pom.xml index 46abee78c4..39be9e27ae 100644 --- a/pom.xml +++ b/pom.xml @@ -150,7 +150,7 @@ 5.1.8 3.5.3 3.4.3 - 5.3.0.6276 + 5.5.0.6356 3.6.1 From b0c714ca2bf4eb3ac251828db064639e50cc0a93 Mon Sep 17 00:00:00 2001 From: "dependabot[bot]" <49699333+dependabot[bot]@users.noreply.github.com> Date: Sat, 28 Mar 2026 18:27:42 +0100 Subject: [PATCH 297/397] RATIS-2467. Bump jacoco-maven-plugin to 0.8.14 (#1402) --- pom.xml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/pom.xml b/pom.xml index 39be9e27ae..598706e616 100644 --- a/pom.xml +++ b/pom.xml @@ -195,7 +195,7 @@ 2.0.17 5.13.3 4.11.0 - 0.8.12 + 0.8.14 1.3.5 flaky | org.apache.ratis.test.tag.FlakyTest From cdb7e467f9138cf5d85503a025e34723f4e95498 Mon Sep 17 00:00:00 2001 From: "Doroszlai, Attila" <6454655+adoroszlai@users.noreply.github.com> Date: Sun, 29 Mar 2026 08:41:02 +0200 Subject: [PATCH 298/397] RATIS-2469. Remove Command#close() (#1404) --- .../apache/ratis/shell/cli/AbstractShell.java | 18 +----------------- .../org/apache/ratis/shell/cli/Command.java | 9 +-------- .../ratis/shell/cli/sh/command/Context.java | 14 ++------------ .../ratis/shell/cli/sh/TestRatisShell.java | 10 +++++----- .../shell/cli/sh/TestSecureRatisShell.java | 4 ++-- 5 files changed, 11 insertions(+), 44 deletions(-) diff --git a/ratis-shell/src/main/java/org/apache/ratis/shell/cli/AbstractShell.java b/ratis-shell/src/main/java/org/apache/ratis/shell/cli/AbstractShell.java index e2679045e5..a59e03cf8e 100644 --- a/ratis-shell/src/main/java/org/apache/ratis/shell/cli/AbstractShell.java +++ b/ratis-shell/src/main/java/org/apache/ratis/shell/cli/AbstractShell.java @@ -19,12 +19,9 @@ import org.apache.commons.cli.CommandLine; import org.apache.ratis.shell.cli.sh.command.Context; -import org.apache.ratis.thirdparty.com.google.common.io.Closer; import org.slf4j.Logger; import org.slf4j.LoggerFactory; -import java.io.Closeable; -import java.io.IOException; import java.util.Arrays; import java.util.Collection; import java.util.Map; @@ -34,20 +31,16 @@ /** * Abstract class for handling command line inputs. */ -public abstract class AbstractShell implements Closeable { +public abstract class AbstractShell { private static final Logger LOG = LoggerFactory.getLogger(AbstractShell.class); private final Map mCommands; - private final Closer closer; /** * Creates a new instance of {@link AbstractShell}. */ protected AbstractShell(Context context) { - closer = Closer.create(); mCommands = loadCommands(context); - // Register all loaded commands under closer. - mCommands.values().forEach(closer::register); } /** @@ -116,11 +109,6 @@ public Collection getCommands() { return mCommands.values(); } - @Override - public void close() throws IOException { - closer.close(); - } - /** * @return name of the shell */ @@ -133,10 +121,6 @@ public void close() throws IOException { */ protected abstract Map loadCommands(Context context); - protected Closer getCloser() { - return closer; - } - /** * Prints usage for all commands. */ diff --git a/ratis-shell/src/main/java/org/apache/ratis/shell/cli/Command.java b/ratis-shell/src/main/java/org/apache/ratis/shell/cli/Command.java index ae4e701077..bc2882bfef 100644 --- a/ratis-shell/src/main/java/org/apache/ratis/shell/cli/Command.java +++ b/ratis-shell/src/main/java/org/apache/ratis/shell/cli/Command.java @@ -23,7 +23,6 @@ import org.apache.commons.cli.Options; import org.apache.commons.cli.ParseException; -import java.io.Closeable; import java.io.IOException; import java.util.Collections; import java.util.Map; @@ -32,7 +31,7 @@ /** * An interface for all the commands that can be run from a shell. */ -public interface Command extends Comparable, Closeable { +public interface Command extends Comparable { /** * Gets the command name as input from the shell. @@ -119,10 +118,4 @@ default int run(CommandLine cl) throws IOException { */ String getDescription(); - /** - * Used to close resources created by commands. - * - * @throws IOException if closing resources fails - */ - default void close() throws IOException {} } diff --git a/ratis-shell/src/main/java/org/apache/ratis/shell/cli/sh/command/Context.java b/ratis-shell/src/main/java/org/apache/ratis/shell/cli/sh/command/Context.java index 6f80256d44..a29cbd0265 100644 --- a/ratis-shell/src/main/java/org/apache/ratis/shell/cli/sh/command/Context.java +++ b/ratis-shell/src/main/java/org/apache/ratis/shell/cli/sh/command/Context.java @@ -24,11 +24,8 @@ import org.apache.ratis.protocol.RaftGroup; import org.apache.ratis.retry.ExponentialBackoffRetry; import org.apache.ratis.retry.RetryPolicy; -import org.apache.ratis.thirdparty.com.google.common.io.Closer; import org.apache.ratis.util.TimeDuration; -import java.io.Closeable; -import java.io.IOException; import java.io.PrintStream; import java.util.Objects; import java.util.Properties; @@ -37,7 +34,7 @@ /** * A context for ratis-shell. */ -public final class Context implements Closeable { +public final class Context { private static final TimeDuration DEFAULT_REQUEST_TIMEOUT = TimeDuration.valueOf(15, TimeUnit.SECONDS); private static final RetryPolicy DEFAULT_RETRY_POLICY = ExponentialBackoffRetry.newBuilder() .setBaseSleepTime(TimeDuration.valueOf(1000, TimeUnit.MILLISECONDS)) @@ -46,7 +43,6 @@ public final class Context implements Closeable { .build(); private final PrintStream mPrintStream; - private final Closer mCloser; private final boolean cli; private final RetryPolicy retryPolicy; @@ -63,8 +59,7 @@ public Context(PrintStream printStream) { public Context(PrintStream printStream, boolean cli, RetryPolicy retryPolicy, RaftProperties properties, Parameters parameters) { - mCloser = Closer.create(); - mPrintStream = mCloser.register(Objects.requireNonNull(printStream, "printStream == null")); + mPrintStream = Objects.requireNonNull(printStream, "printStream == null"); this.cli = cli; this.retryPolicy = retryPolicy != null? retryPolicy : DEFAULT_RETRY_POLICY; @@ -115,9 +110,4 @@ public RaftClient newRaftClient(RaftGroup group) { .setRetryPolicy(getRetryPolicy()) .build(); } - - @Override - public void close() throws IOException { - mCloser.close(); - } } diff --git a/ratis-test/src/test/java/org/apache/ratis/shell/cli/sh/TestRatisShell.java b/ratis-test/src/test/java/org/apache/ratis/shell/cli/sh/TestRatisShell.java index 21f4685121..ccf9702f7f 100644 --- a/ratis-test/src/test/java/org/apache/ratis/shell/cli/sh/TestRatisShell.java +++ b/ratis-test/src/test/java/org/apache/ratis/shell/cli/sh/TestRatisShell.java @@ -44,6 +44,7 @@ /** * Test {@link RatisShell} */ +@SuppressWarnings({"rawtypes"}) public class TestRatisShell extends BaseTest { static final PrintStream OUT = System.out; static final Class[] ARG_CLASSES = new Class[] {Context.class}; @@ -65,12 +66,11 @@ public void testFullParentCommandList() throws Exception { final List expected = new ArrayList<>(loadCommands(RatisShell.class.getPackage().getName() + ".command")); Collections.sort(expected); - try(RatisShell shell = new RatisShell(OUT)) { - final List computed = new ArrayList<>(shell.getCommands()); - Collections.sort(computed); + RatisShell shell = new RatisShell(OUT); + final List computed = new ArrayList<>(shell.getCommands()); + Collections.sort(computed); - assertCommands(expected, computed); - } + assertCommands(expected, computed); } @Test diff --git a/ratis-test/src/test/java/org/apache/ratis/shell/cli/sh/TestSecureRatisShell.java b/ratis-test/src/test/java/org/apache/ratis/shell/cli/sh/TestSecureRatisShell.java index 7f11490670..21e9fe229a 100644 --- a/ratis-test/src/test/java/org/apache/ratis/shell/cli/sh/TestSecureRatisShell.java +++ b/ratis-test/src/test/java/org/apache/ratis/shell/cli/sh/TestSecureRatisShell.java @@ -85,8 +85,8 @@ public void testRatisShell() throws Exception { } void runTestRatisShell(MiniRaftClusterWithGrpc cluster, boolean secure) throws Exception { - try(ByteArrayOutputStream out = new ByteArrayOutputStream(1 << 16); - RatisShell shell = newRatisShell(out, cluster.getProperties(), secure)) { + try (ByteArrayOutputStream out = new ByteArrayOutputStream(1 << 16)) { + RatisShell shell = newRatisShell(out, cluster.getProperties(), secure); shell.run("group", "info", "-peers", toCliArg(cluster.getPeers())); final String output = out.toString(); LOG.info("output (secure? {}):\n{}", secure, output); From d1188e21f89f4606304b878ae9a8be45d4bda831 Mon Sep 17 00:00:00 2001 From: "dependabot[bot]" <49699333+dependabot[bot]@users.noreply.github.com> Date: Sun, 29 Mar 2026 12:11:36 +0200 Subject: [PATCH 299/397] RATIS-2471. Bump commons-cli to 1.11.0 (#1398) --- ratis-shell/pom.xml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/ratis-shell/pom.xml b/ratis-shell/pom.xml index efa96a7866..495187b2a8 100644 --- a/ratis-shell/pom.xml +++ b/ratis-shell/pom.xml @@ -43,7 +43,7 @@ commons-cli commons-cli - 1.5.0 + 1.11.0 org.slf4j From 28dc87aed0140ca22dc34006957a6705866e95cb Mon Sep 17 00:00:00 2001 From: "Doroszlai, Attila" <6454655+adoroszlai@users.noreply.github.com> Date: Mon, 30 Mar 2026 12:16:33 +0200 Subject: [PATCH 300/397] RATIS-2473. Ignore JUnit 6+ due to Java 17 requirement (#1408) --- .github/dependabot.yml | 3 +++ 1 file changed, 3 insertions(+) diff --git a/.github/dependabot.yml b/.github/dependabot.yml index 2df7a383ab..47dcd35eca 100644 --- a/.github/dependabot.yml +++ b/.github/dependabot.yml @@ -40,3 +40,6 @@ updates: versions: [">=4.9.0.0"] - dependency-name: "org.mockito:mockito-core" versions: [">=5.0.0"] + # requires Java 17 + - dependency-name: "org.junit:junit-bom" + versions: [">=6.0.0"] From ba11fa366ed8d4b42f83c61bf9bbafcfb67eac78 Mon Sep 17 00:00:00 2001 From: "dependabot[bot]" <49699333+dependabot[bot]@users.noreply.github.com> Date: Mon, 30 Mar 2026 14:19:04 +0200 Subject: [PATCH 301/397] RATIS-2476. Bump develocity-maven-extension to 2.4.0 (#1410) --- .mvn/extensions.xml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/.mvn/extensions.xml b/.mvn/extensions.xml index a32ab684b6..960b8cd9ef 100644 --- a/.mvn/extensions.xml +++ b/.mvn/extensions.xml @@ -24,7 +24,7 @@ com.gradle develocity-maven-extension - 1.22.2 + 2.4.0 com.gradle From d59bd0d8f542f2a7291808e0091d259189290d1d Mon Sep 17 00:00:00 2001 From: "dependabot[bot]" <49699333+dependabot[bot]@users.noreply.github.com> Date: Mon, 30 Mar 2026 15:17:12 +0200 Subject: [PATCH 302/397] RATIS-2479. Bump junit to 5.14.3 (#1409) --- pom.xml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/pom.xml b/pom.xml index 598706e616..7217c51078 100644 --- a/pom.xml +++ b/pom.xml @@ -193,7 +193,7 @@ 4 2.0.17 - 5.13.3 + 5.14.3 4.11.0 0.8.14 1.3.5 From 67a3717fe5381219d2c39e51c7c7446e41ba57ed Mon Sep 17 00:00:00 2001 From: "dependabot[bot]" <49699333+dependabot[bot]@users.noreply.github.com> Date: Mon, 30 Mar 2026 15:43:43 +0200 Subject: [PATCH 303/397] RATIS-2480. Bump os-maven-plugin to 1.7.1 (#1406) --- pom.xml | 2 +- ratis-proto/pom.xml | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/pom.xml b/pom.xml index 7217c51078..8a823485c1 100644 --- a/pom.xml +++ b/pom.xml @@ -431,7 +431,7 @@ kr.motd.maven os-maven-plugin - 1.5.0.Final + 1.7.1 diff --git a/ratis-proto/pom.xml b/ratis-proto/pom.xml index 9eb9229603..95a49dcd74 100644 --- a/ratis-proto/pom.xml +++ b/ratis-proto/pom.xml @@ -36,7 +36,7 @@ kr.motd.maven os-maven-plugin - 1.5.0.Final + 1.7.1 From f61a9c82726867be774e2551f73eef480f112f11 Mon Sep 17 00:00:00 2001 From: "Doroszlai, Attila" <6454655+adoroszlai@users.noreply.github.com> Date: Mon, 30 Mar 2026 21:34:31 +0200 Subject: [PATCH 304/397] RATIS-2478. compile check does not set target version (#1411) --- .github/workflows/ci.yaml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/.github/workflows/ci.yaml b/.github/workflows/ci.yaml index 50b88212a5..272cb33d90 100644 --- a/.github/workflows/ci.yaml +++ b/.github/workflows/ci.yaml @@ -45,7 +45,7 @@ jobs: java-version: ${{ matrix.java }} needs-source-tarball: true script: compile - script-args: -Djavac.version=${{ matrix.java }} + script-args: -Dmaven.compiler.release=${{ matrix.java }} split: ${{ matrix.java }} timeout-minutes: 30 secrets: inherit From 9b48f5b4dfd435907ba9d31f4f03ea7d233827e1 Mon Sep 17 00:00:00 2001 From: "dependabot[bot]" <49699333+dependabot[bot]@users.noreply.github.com> Date: Mon, 30 Mar 2026 21:57:29 +0200 Subject: [PATCH 305/397] RATIS-2472. Bump apache parent to 37 (#1403) Co-authored-by: Doroszlai, Attila --- pom.xml | 8 +++----- ratis-bom/pom.xml | 2 +- 2 files changed, 4 insertions(+), 6 deletions(-) diff --git a/pom.xml b/pom.xml index 8a823485c1..f78f3589bb 100644 --- a/pom.xml +++ b/pom.xml @@ -18,7 +18,7 @@ org.apache apache - 35 + 37 @@ -171,10 +171,8 @@ bash - - 8 - - ${maven.compiler.release} + + ${javaVersion} 1.0.11 diff --git a/ratis-bom/pom.xml b/ratis-bom/pom.xml index aac81f0170..a848571fbf 100644 --- a/ratis-bom/pom.xml +++ b/ratis-bom/pom.xml @@ -18,7 +18,7 @@ org.apache apache - 35 + 37 From 86f2aabe66365eff4454a79b9b0a6e506ae2ae64 Mon Sep 17 00:00:00 2001 From: "dependabot[bot]" <49699333+dependabot[bot]@users.noreply.github.com> Date: Tue, 31 Mar 2026 15:11:50 +0200 Subject: [PATCH 306/397] RATIS-2481. Bump license-maven-plugin to 2.7.1 (#1412) --- pom.xml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/pom.xml b/pom.xml index f78f3589bb..a893aacd73 100644 --- a/pom.xml +++ b/pom.xml @@ -157,7 +157,7 @@ 3.6.2 1.12.0 3.0.0 - 2.7.0 + 2.7.1 0.6.1 2.9.1 From 5f6c6bfa7d307db634ae4bf1f0f0e70a1e354147 Mon Sep 17 00:00:00 2001 From: "Doroszlai, Attila" <6454655+adoroszlai@users.noreply.github.com> Date: Tue, 31 Mar 2026 21:22:31 +0200 Subject: [PATCH 307/397] RATIS-2484. Remove unused jakarta.annotation-api (#1416) --- pom.xml | 6 ------ ratis-proto/pom.xml | 4 ---- 2 files changed, 10 deletions(-) diff --git a/pom.xml b/pom.xml index a893aacd73..6574d460c1 100644 --- a/pom.xml +++ b/pom.xml @@ -194,7 +194,6 @@ 5.14.3 4.11.0 0.8.14 - 1.3.5 flaky | org.apache.ratis.test.tag.FlakyTest @@ -388,11 +387,6 @@ mockito-core ${mockito.version} - - jakarta.annotation - jakarta.annotation-api - ${jakarta.annotation.version} - diff --git a/ratis-proto/pom.xml b/ratis-proto/pom.xml index 95a49dcd74..322bf56778 100644 --- a/ratis-proto/pom.xml +++ b/ratis-proto/pom.xml @@ -170,10 +170,6 @@ org.apache.ratis ratis-thirdparty-misc - - jakarta.annotation - jakarta.annotation-api - org.junit.jupiter junit-jupiter-engine From f9f4b0263d264f1e5f734af82e10be5fb0971e6d Mon Sep 17 00:00:00 2001 From: "dependabot[bot]" <49699333+dependabot[bot]@users.noreply.github.com> Date: Thu, 2 Apr 2026 14:41:55 +0200 Subject: [PATCH 308/397] RATIS-2490. Bump spotbugs-maven-plugin to 4.8.6.8 (#1421) --- pom.xml | 2 +- ratis-bom/pom.xml | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/pom.xml b/pom.xml index 6574d460c1..099fd6bfd5 100644 --- a/pom.xml +++ b/pom.xml @@ -163,7 +163,7 @@ 2.9.1 4.8.6 - 4.8.6.2 + 4.8.6.8 ${distMgmtReleasesId} ${distMgmtReleasesName} diff --git a/ratis-bom/pom.xml b/ratis-bom/pom.xml index a848571fbf..3a046d9ff3 100644 --- a/ratis-bom/pom.xml +++ b/ratis-bom/pom.xml @@ -143,7 +143,7 @@ com.github.spotbugs spotbugs-maven-plugin - 4.8.6.2 + 4.8.6.8 true From e95f94edb65b43777277784c6910432e6c053361 Mon Sep 17 00:00:00 2001 From: "dependabot[bot]" <49699333+dependabot[bot]@users.noreply.github.com> Date: Thu, 2 Apr 2026 15:33:43 +0200 Subject: [PATCH 309/397] RATIS-2482. Bump maven-bundle-plugin to 6.0.2 (#1413) --- pom.xml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/pom.xml b/pom.xml index 099fd6bfd5..df0f71ca52 100644 --- a/pom.xml +++ b/pom.xml @@ -147,7 +147,7 @@ false - 5.1.8 + 6.0.2 3.5.3 3.4.3 5.5.0.6356 From a6b3c19ee001285762e23e62126e1af36cec65c0 Mon Sep 17 00:00:00 2001 From: "Doroszlai, Attila" <6454655+adoroszlai@users.noreply.github.com> Date: Fri, 3 Apr 2026 15:28:41 +0200 Subject: [PATCH 310/397] RATIS-2454. Findbugs summary is empty despite violations (#1396) --- .gitignore | 1 + dev-support/checks/_lib.sh | 88 ++++++++++++++++++++++++++ dev-support/checks/findbugs.sh | 24 +++---- dev-support/checks/install/spotbugs.sh | 34 ++++++++++ dev-support/find_maven.sh | 2 +- 5 files changed, 137 insertions(+), 12 deletions(-) create mode 100644 dev-support/checks/_lib.sh create mode 100644 dev-support/checks/install/spotbugs.sh diff --git a/.gitignore b/.gitignore index ecaf1f2df9..cf5943b0df 100644 --- a/.gitignore +++ b/.gitignore @@ -8,6 +8,7 @@ *.sdf *.suo *.vcxproj.user +.dev-tools/ .hugo_build.lock .idea .classpath diff --git a/dev-support/checks/_lib.sh b/dev-support/checks/_lib.sh new file mode 100644 index 0000000000..fd30f756fb --- /dev/null +++ b/dev-support/checks/_lib.sh @@ -0,0 +1,88 @@ +#!/usr/bin/env bash +# 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. + +check_name="$(basename "${BASH_SOURCE[1]}")" +check_name="${check_name%.sh}" + +: ${TOOLS_DIR:=$(pwd)/.dev-tools} # directory for tools +: ${RATIS_PREFER_LOCAL_TOOL:=true} # skip install if tools are already available (eg. via package manager) + +## @description Install a dependency. Only first argument is mandatory. +## @param name of the tool +## @param the directory for binaries, relative to the tool directory; added to PATH. +## @param the directory for the tool, relative to TOOLS_DIR +## @param name of the executable, for testing if it is already installed +## @param name of the function that performs actual installation steps +_install_tool() { + local tool bindir dir bin func + + tool="$1" + bindir="${2:-}" + dir="${TOOLS_DIR}"/"${3:-"${tool}"}" + bin="${4:-"${tool}"}" + func="${5:-"_install_${tool}"}" + + if [[ "${RATIS_PREFER_LOCAL_TOOL}" == "true" ]] && which "${bin}" >& /dev/null; then + echo "Skip installing $bin, as it's already available on PATH." + return + fi + + if [[ ! -d "${dir}" ]]; then + mkdir -pv "${dir}" + _do_install "${tool}" "${dir}" "${func}" + fi + + if [[ -n "${bindir}" ]]; then + _add_to_path "${dir}"/"${bindir}" + + if ! which "${bin}" >& /dev/null; then + _do_install "${tool}" "${dir}" "${func}" + _add_to_path "${dir}"/"${bindir}" + fi + fi +} + +_do_install() { + local tool="$1" + local dir="$2" + local func="$3" + + pushd "${dir}" + if eval "${func}"; then + echo "Installed ${tool} in ${dir}" + popd + else + popd + msg="Failed to install ${tool}" + echo "${msg}" >&2 + if [[ -n "${REPORT_FILE}" ]]; then + echo "${msg}" >> "${REPORT_FILE}" + fi + exit 1 + fi +} + +_add_to_path() { + local bindir="$1" + + if [[ -d "${bindir}" ]]; then + if [[ "${RATIS_PREFER_LOCAL_TOOL}" == "true" ]]; then + export PATH="${PATH}:${bindir}" + else + export PATH="${bindir}:${PATH}" + fi + fi +} diff --git a/dev-support/checks/findbugs.sh b/dev-support/checks/findbugs.sh index 3a063b3fa1..93d3ef936f 100755 --- a/dev-support/checks/findbugs.sh +++ b/dev-support/checks/findbugs.sh @@ -13,6 +13,9 @@ # 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. + +set -u -o pipefail + DIR="$( cd "$( dirname "${BASH_SOURCE[0]}" )" >/dev/null 2>&1 && pwd )" cd "$DIR/../.." || exit 1 @@ -20,28 +23,27 @@ source "${DIR}/../find_maven.sh" : ${WITH_COVERAGE:="false"} -MAVEN_OPTIONS='-B -fae' +REPORT_DIR=${OUTPUT_DIR:-"$DIR/../../target/findbugs"} +mkdir -p "$REPORT_DIR" +REPORT_FILE="$REPORT_DIR/summary.txt" -if ! type unionBugs >/dev/null 2>&1 || ! type convertXmlToText >/dev/null 2>&1; then - #shellcheck disable=SC2086 - ${MVN} ${MAVEN_OPTIONS} test-compile spotbugs:check - exit $? -fi +source "${DIR}/_lib.sh" +source "${DIR}/install/spotbugs.sh" + +MAVEN_OPTIONS='-B -fae' if [[ "${WITH_COVERAGE}" != "true" ]]; then MAVEN_OPTIONS="${MAVEN_OPTIONS} -Djacoco.skip" fi #shellcheck disable=SC2086 -${MVN} ${MAVEN_OPTIONS} test-compile spotbugs:spotbugs +${MVN} ${MAVEN_OPTIONS} test-compile spotbugs:spotbugs "$@" | tee "${REPORT_DIR}/output.log" rc=$? -REPORT_DIR=${OUTPUT_DIR:-"$DIR/../../target/findbugs"} -mkdir -p "$REPORT_DIR" -REPORT_FILE="$REPORT_DIR/summary.txt" +touch "$REPORT_FILE" find ratis* -name spotbugsXml.xml -print0 | xargs -0 unionBugs -output "${REPORT_DIR}"/summary.xml -convertXmlToText "${REPORT_DIR}"/summary.xml | tee "${REPORT_FILE}" +convertXmlToText "${REPORT_DIR}"/summary.xml | tee -a "${REPORT_FILE}" convertXmlToText -html:fancy-hist.xsl "${REPORT_DIR}"/summary.xml "${REPORT_DIR}"/summary.html wc -l "$REPORT_FILE" | awk '{print $1}'> "$REPORT_DIR/failures" diff --git a/dev-support/checks/install/spotbugs.sh b/dev-support/checks/install/spotbugs.sh new file mode 100644 index 0000000000..337ba2b94d --- /dev/null +++ b/dev-support/checks/install/spotbugs.sh @@ -0,0 +1,34 @@ +#!/usr/bin/env bash +# 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. + +# This script installs SpotBugs. +# Requires _install_tool from _lib.sh. Use `source` for both scripts, because it modifies $PATH. + +_get_spotbugs_version() { + MAVEN_ARGS='' ${MVN} -q -DforceStdout -Dscan=false help:evaluate -Dexpression=spotbugs.version 2>/dev/null || echo '4.8.6' +} + +if [[ -z "${SPOTBUGS_VERSION:-}" ]]; then + SPOTBUGS_VERSION="$(_get_spotbugs_version)" +fi + +_install_spotbugs() { + echo "https://repo.maven.apache.org/maven2/com/github/spotbugs/spotbugs/${SPOTBUGS_VERSION}/spotbugs-${SPOTBUGS_VERSION}.tgz" + curl -LSs "https://repo.maven.apache.org/maven2/com/github/spotbugs/spotbugs/${SPOTBUGS_VERSION}/spotbugs-${SPOTBUGS_VERSION}.tgz" | tar -xz -f - || exit 1 + find "spotbugs-${SPOTBUGS_VERSION}"/bin -type f -print0 | xargs -0 --no-run-if-empty chmod +x +} + +_install_tool spotbugs "spotbugs-${SPOTBUGS_VERSION}/bin" diff --git a/dev-support/find_maven.sh b/dev-support/find_maven.sh index 20b6462b1e..2067ff5152 100644 --- a/dev-support/find_maven.sh +++ b/dev-support/find_maven.sh @@ -17,7 +17,7 @@ # limitations under the License. function find_maven() { - if [ "$MAVEN" != "" ]; then + if [[ -n "${MAVEN:-}" ]]; then echo "${MAVEN}" else local DIR From 94da526093679d7f51110825d5e393f1952f4a13 Mon Sep 17 00:00:00 2001 From: "Doroszlai, Attila" <6454655+adoroszlai@users.noreply.github.com> Date: Fri, 3 Apr 2026 19:41:23 +0200 Subject: [PATCH 311/397] RATIS-2494. Ignore Hadoop 3.5+ due to Java 17 requirement (#1423) --- .github/dependabot.yml | 2 ++ 1 file changed, 2 insertions(+) diff --git a/.github/dependabot.yml b/.github/dependabot.yml index 47dcd35eca..d343c5a93d 100644 --- a/.github/dependabot.yml +++ b/.github/dependabot.yml @@ -41,5 +41,7 @@ updates: - dependency-name: "org.mockito:mockito-core" versions: [">=5.0.0"] # requires Java 17 + - dependency-name: "org.apache.hadoop:*" + versions: [">=3.5.0"] - dependency-name: "org.junit:junit-bom" versions: [">=6.0.0"] From 0f954a6e1c987d9c1e4fb4aa9a58230a9b9fa87e Mon Sep 17 00:00:00 2001 From: Ivan Andika Date: Sat, 4 Apr 2026 02:01:22 +0800 Subject: [PATCH 312/397] RATIS-2487. Trigger installSnapshot if leader cannot get previous entry (#1420) --- .../ratis/grpc/server/GrpcLogAppender.java | 38 -------- .../ratis/server/leader/LogAppender.java | 83 +++++++++++++---- .../ratis/server/leader/LogAppenderBase.java | 29 ++---- .../org/apache/ratis/LogAppenderTests.java | 92 +++++++++++++++++++ 4 files changed, 167 insertions(+), 75 deletions(-) diff --git a/ratis-grpc/src/main/java/org/apache/ratis/grpc/server/GrpcLogAppender.java b/ratis-grpc/src/main/java/org/apache/ratis/grpc/server/GrpcLogAppender.java index b4d78c207a..69421e9f0f 100644 --- a/ratis-grpc/src/main/java/org/apache/ratis/grpc/server/GrpcLogAppender.java +++ b/ratis-grpc/src/main/java/org/apache/ratis/grpc/server/GrpcLogAppender.java @@ -832,44 +832,6 @@ private void notifyInstallSnapshot(TermIndex firstAvailable) { responseHandler.waitForResponse(); } - /** - * Should the Leader notify the Follower to install the snapshot through - * its own State Machine. - * @return the first available log's start term index - */ - private TermIndex shouldNotifyToInstallSnapshot() { - final FollowerInfo follower = getFollower(); - final long leaderNextIndex = getRaftLog().getNextIndex(); - final boolean isFollowerBootstrapping = getLeaderState().isFollowerBootstrapping(follower); - final long leaderStartIndex = getRaftLog().getStartIndex(); - final TermIndex firstAvailable = Optional.ofNullable(getRaftLog().getTermIndex(leaderStartIndex)) - .orElseGet(() -> TermIndex.valueOf(getServer().getInfo().getCurrentTerm(), leaderNextIndex)); - if (isFollowerBootstrapping && !follower.hasAttemptedToInstallSnapshot()) { - // If the follower is bootstrapping and has not yet installed any snapshot from leader, then the follower should - // be notified to install a snapshot. Every follower should try to install at least one snapshot during - // bootstrapping, if available. - LOG.debug("{}: follower is bootstrapping, notify to install snapshot to {}.", this, firstAvailable); - return firstAvailable; - } - - final long followerNextIndex = follower.getNextIndex(); - if (followerNextIndex >= leaderNextIndex) { - return null; - } - - if (followerNextIndex < leaderStartIndex) { - // The Leader does not have the logs from the Follower's last log - // index onwards. And install snapshot is disabled. So the Follower - // should be notified to install the latest snapshot through its - // State Machine. - return firstAvailable; - } else if (leaderStartIndex == RaftLog.INVALID_LOG_INDEX) { - // Leader has no logs to check from, hence return next index. - return firstAvailable; - } - - return null; - } static class AppendEntriesRequest { private final Timekeeper timer; diff --git a/ratis-server-api/src/main/java/org/apache/ratis/server/leader/LogAppender.java b/ratis-server-api/src/main/java/org/apache/ratis/server/leader/LogAppender.java index cff5425d32..33914fde7f 100644 --- a/ratis-server-api/src/main/java/org/apache/ratis/server/leader/LogAppender.java +++ b/ratis-server-api/src/main/java/org/apache/ratis/server/leader/LogAppender.java @@ -134,38 +134,89 @@ default RaftPeerId getFollowerId() { /** @return an {@link Iterable} of {@link InstallSnapshotRequestProto} for sending the given snapshot. */ Iterable newInstallSnapshotRequests(String requestId, SnapshotInfo snapshot); + /** + * Get the previous {@link TermIndex} for the given next index. + * This is used to set the previous log entry in AppendEntries requests. + * + * @return the previous {@link TermIndex}, or null if unavailable + * (e.g. the entry has been purged and the snapshot does not cover it). + */ + default TermIndex getPrevious(long nextIndex) { + if (nextIndex == RaftLog.LEAST_VALID_LOG_INDEX) { + return null; + } + + final long previousIndex = nextIndex - 1; + final TermIndex previous = getRaftLog().getTermIndex(previousIndex); + if (previous != null) { + return previous; + } + + final SnapshotInfo snapshot = getServer().getStateMachine().getLatestSnapshot(); + if (snapshot != null) { + final TermIndex snapshotTermIndex = snapshot.getTermIndex(); + if (snapshotTermIndex.getIndex() == previousIndex) { + return snapshotTermIndex; + } + } + + return null; + } + /** * Should this {@link LogAppender} send a snapshot to the follower? * * @return the snapshot if it should install a snapshot; otherwise, return null. */ default SnapshotInfo shouldInstallSnapshot() { - // we should install snapshot if the follower needs to catch up and: - // 1. there is no local log entry but there is snapshot - // 2. or the follower's next index is smaller than the log start index - // 3. or the follower is bootstrapping (i.e. not yet caught up) and has not installed any snapshot yet - final FollowerInfo follower = getFollower(); - final boolean isFollowerBootstrapping = getLeaderState().isFollowerBootstrapping(follower); final SnapshotInfo snapshot = getServer().getStateMachine().getLatestSnapshot(); + return shouldInstallSnapshot(snapshot != null) ? snapshot : null; + } + + /** + * Should this {@link LogAppender} send a snapshot notification to the follower? + * + * @return the first available log {@link TermIndex} if it should install a snapshot; otherwise, return null. + */ + default TermIndex shouldNotifyToInstallSnapshot() { + if (!shouldInstallSnapshot(true)) { + return null; + } + final TermIndex start = getRaftLog().getTermIndex(getRaftLog().getStartIndex()); + if (start != null) { + return start; + } + // No log is currently available; return the next, which will become available in the future. + return TermIndex.valueOf(getServer().getInfo().getCurrentTerm(), getRaftLog().getNextIndex()); + } - if (isFollowerBootstrapping && !follower.hasAttemptedToInstallSnapshot()) { - if (snapshot == null) { + default boolean shouldInstallSnapshot(boolean hasSnapshot) { + final FollowerInfo follower = getFollower(); + if (getLeaderState().isFollowerBootstrapping(follower) + && !follower.hasAttemptedToInstallSnapshot()) { + if (!hasSnapshot) { // Leader cannot send null snapshot to follower. Hence, acknowledge InstallSnapshot attempt (even though it // was not attempted) so that follower can come out of staging state after appending log entries. follower.setAttemptedToInstallSnapshot(); - } else { - return snapshot; } + return true; } + final long leaderNextIndex = getRaftLog().getNextIndex(); final long followerNextIndex = getFollower().getNextIndex(); - if (followerNextIndex < getRaftLog().getNextIndex()) { - final long logStartIndex = getRaftLog().getStartIndex(); - if (followerNextIndex < logStartIndex || (logStartIndex == RaftLog.INVALID_LOG_INDEX && snapshot != null)) { - return snapshot; - } + if (followerNextIndex >= leaderNextIndex) { + // follower caught up already + return false; } - return null; + final long leaderStartIndex = getRaftLog().getStartIndex(); + if (followerNextIndex < leaderStartIndex || leaderStartIndex == RaftLog.INVALID_LOG_INDEX) { + // leader does not have follower's next log + return true; + } + // leader does not have the previous log for appendEntries + return followerNextIndex == leaderStartIndex && + followerNextIndex > RaftLog.LEAST_VALID_LOG_INDEX && + getPrevious(followerNextIndex) == null; } /** Define how this {@link LogAppender} should run. */ diff --git a/ratis-server/src/main/java/org/apache/ratis/server/leader/LogAppenderBase.java b/ratis-server/src/main/java/org/apache/ratis/server/leader/LogAppenderBase.java index be0404da36..4f558e0c7c 100644 --- a/ratis-server/src/main/java/org/apache/ratis/server/leader/LogAppenderBase.java +++ b/ratis-server/src/main/java/org/apache/ratis/server/leader/LogAppenderBase.java @@ -165,27 +165,6 @@ public boolean hasPendingDataRequests() { return false; } - private TermIndex getPrevious(long nextIndex) { - if (nextIndex == RaftLog.LEAST_VALID_LOG_INDEX) { - return null; - } - - final long previousIndex = nextIndex - 1; - final TermIndex previous = getRaftLog().getTermIndex(previousIndex); - if (previous != null) { - return previous; - } - - final SnapshotInfo snapshot = server.getStateMachine().getLatestSnapshot(); - if (snapshot != null) { - final TermIndex snapshotTermIndex = snapshot.getTermIndex(); - if (snapshotTermIndex.getIndex() == previousIndex) { - return snapshotTermIndex; - } - } - - return null; - } protected long getNextIndexForInconsistency(long requestFirstIndex, long replyNextIndex) { long next = replyNextIndex; @@ -238,6 +217,14 @@ public AppendEntriesRequestProto newAppendEntriesRequest(long callId, boolean he final long snapshotIndex = follower.getSnapshotIndex(); final long leaderNext = getRaftLog().getNextIndex(); final long followerNext = follower.getNextIndex(); + + if (previous == null && followerNext > RaftLog.LEAST_VALID_LOG_INDEX && followerNext != snapshotIndex + 1) { + LOG.info("{}: Skipping appendEntries since the previous log entry is unavailable:" + + " follower {} nextIndex={} and snapshotIndex={} but leader startIndex={}", + this, follower.getName(), followerNext, snapshotIndex, getRaftLog().getStartIndex()); + return null; + } + final long halfMs = heartbeatWaitTimeMs/2; for (long next = followerNext; leaderNext > next && getHeartbeatWaitTimeMs() - halfMs > 0; ) { if (!buffer.offer(getRaftLog().getEntryWithData(next++))) { diff --git a/ratis-server/src/test/java/org/apache/ratis/LogAppenderTests.java b/ratis-server/src/test/java/org/apache/ratis/LogAppenderTests.java index c7a7849e6a..c9b19a72a1 100644 --- a/ratis-server/src/test/java/org/apache/ratis/LogAppenderTests.java +++ b/ratis-server/src/test/java/org/apache/ratis/LogAppenderTests.java @@ -32,6 +32,7 @@ import org.apache.ratis.server.RaftServerConfigKeys; import org.apache.ratis.server.leader.LogAppender; import org.apache.ratis.server.impl.MiniRaftCluster; +import org.apache.ratis.server.impl.RaftServerTestUtil; import org.apache.ratis.server.metrics.RaftServerMetricsImpl; import org.apache.ratis.server.raftlog.LogProtoUtils; import org.apache.ratis.server.raftlog.RaftLog; @@ -53,6 +54,7 @@ import java.util.concurrent.atomic.AtomicBoolean; import java.util.concurrent.atomic.AtomicLong; import java.util.concurrent.atomic.AtomicReference; +import java.util.stream.Stream; import org.apache.ratis.thirdparty.com.codahale.metrics.Gauge; import org.slf4j.event.Level; @@ -222,4 +224,94 @@ void runTest(CLUSTER cluster) throws Exception { Assertions.assertNotNull(last); Assertions.assertTrue(last.getIndex() <= leader.getInfo().getLastAppliedIndex()); } + + @Test + public void testNewAppendEntriesRequestAfterPurgeFollowerBehindStartIndex() throws Exception { + final RaftProperties prop = getProperties(); + RaftServerConfigKeys.Log.setPurgeGap(prop, 1); + RaftServerConfigKeys.Log.setSegmentSizeMax(prop, SizeInBytes.valueOf("1KB")); + runWithNewCluster(3, cluster -> { + final long startIndexAfterPurge = setupPurgedLeaderLog(cluster); + // Test when followerNextIndex < leader's logStartIndex + runTestNewAppendEntriesRequestAfterPurge(cluster, startIndexAfterPurge - 1); + }); + } + + @Test + public void testNewAppendEntriesRequestAfterPurgeFollowerAtStartIndex() throws Exception { + final RaftProperties prop = getProperties(); + RaftServerConfigKeys.Log.setPurgeGap(prop, 1); + RaftServerConfigKeys.Log.setSegmentSizeMax(prop, SizeInBytes.valueOf("1KB")); + runWithNewCluster(3, cluster -> { + final long startIndexAfterPurge = setupPurgedLeaderLog(cluster); + // Test when followerNextIndex == leader's logStartIndex, but the previous index is already purged + runTestNewAppendEntriesRequestAfterPurge(cluster, startIndexAfterPurge); + }); + } + + private long setupPurgedLeaderLog(CLUSTER cluster) throws Exception { + final RaftServer.Division leader = waitForLeader(cluster); + final RaftLog leaderLog = leader.getRaftLog(); + + try (RaftClient client = cluster.createClient(leader.getId())) { + for (SimpleMessage msg : generateMsgs(5)) { + client.io().send(msg); + } + } + + final long lastLogIndex = leaderLog.getLastEntryTermIndex().getIndex(); + LOG.info("Leader log lastIndex={}, startIndex={}", lastLogIndex, leaderLog.getStartIndex()); + Assertions.assertTrue(lastLogIndex > 5, "Need enough log entries for the test"); + + // Take a snapshot so that shouldInstallSnapshot() can return it + final long snapshotIndex = SimpleStateMachine4Testing.get(leader).takeSnapshot(); + LOG.info("Snapshot taken at index {}", snapshotIndex); + Assertions.assertTrue(snapshotIndex > 0, "Snapshot should have been taken"); + + final long purgeUpTo = lastLogIndex - 2; + LOG.info("Purging leader log up to index {}", purgeUpTo); + leaderLog.purge(purgeUpTo).get(); + + final long startIndexAfterPurge = leaderLog.getStartIndex(); + LOG.info("Leader log after purge: startIndex={}", startIndexAfterPurge); + Assertions.assertTrue(startIndexAfterPurge > 1, + "Purge should have advanced startIndex, but got " + startIndexAfterPurge); + + return startIndexAfterPurge; + } + + void runTestNewAppendEntriesRequestAfterPurge(CLUSTER cluster, + long targetNextIndex) throws Exception { + final RaftServer.Division leader = waitForLeader(cluster); + final RaftLog leaderLog = leader.getRaftLog(); + final long startIndexAfterPurge = leaderLog.getStartIndex(); + + final Stream appenders = RaftServerTestUtil.getLogAppenders(leader); + Assertions.assertNotNull(appenders, "Leader should have log appenders"); + final LogAppender appender = appenders.findFirst().orElseThrow( + () -> new AssertionError("No log appender found")); + + Assertions.assertTrue(targetNextIndex > RaftLog.LEAST_VALID_LOG_INDEX, + "targetNextIndex should be > LEAST_VALID_LOG_INDEX"); + appender.getFollower().setNextIndex(targetNextIndex); + + LOG.info("Set follower nextIndex={}, startIndexAfterPurge={}, snapshotIndex={}", + targetNextIndex, startIndexAfterPurge, appender.getFollower().getSnapshotIndex()); + Assertions.assertEquals(0, appender.getFollower().getSnapshotIndex(), + "Follower snapshotIndex should be 0 (default, never installed snapshot)"); + + Assertions.assertNull(leaderLog.getTermIndex(targetNextIndex - 1), + "Entry at previousIndex=" + (targetNextIndex - 1) + " should have been purged"); + + // Should return null instead of throwing NPE + Assertions.assertNull(appender.newAppendEntriesRequest(0, false), + "newAppendEntriesRequest should return null when previous TermIndex is not found"); + + Assertions.assertEquals(targetNextIndex, appender.getFollower().getNextIndex(), + "Follower nextIndex should remain unchanged"); + + Assertions.assertNotNull(appender.shouldInstallSnapshot(), + "shouldInstallSnapshot should return non-null when followerNextIndex (" + + targetNextIndex + ") and previous entry has been purged"); + } } From 6e40497333fed219f0ca5e8f11beae747d58eba7 Mon Sep 17 00:00:00 2001 From: "dependabot[bot]" <49699333+dependabot[bot]@users.noreply.github.com> Date: Fri, 3 Apr 2026 20:26:26 +0200 Subject: [PATCH 313/397] RATIS-2489. Bump exec-maven-plugin to 3.6.3 (#1418) Co-authored-by: Doroszlai, Attila --- dev-support/make_rc.sh | 2 +- pom.xml | 2 +- ratis-experiments/pom.xml | 1 - 3 files changed, 2 insertions(+), 3 deletions(-) diff --git a/dev-support/make_rc.sh b/dev-support/make_rc.sh index 42cadb07d6..7317729acb 100755 --- a/dev-support/make_rc.sh +++ b/dev-support/make_rc.sh @@ -36,7 +36,7 @@ fi mvnGet() { ${MVN} -q -Dexec.executable="echo" -Dexec.args="\${${1}}" --non-recursive \ - org.codehaus.mojo:exec-maven-plugin:1.6.0:exec 2>/dev/null + org.codehaus.mojo:exec-maven-plugin:exec 2>/dev/null } diff --git a/pom.xml b/pom.xml index df0f71ca52..97d5841279 100644 --- a/pom.xml +++ b/pom.xml @@ -154,7 +154,7 @@ 3.6.1 - 3.6.2 + 3.6.3 1.12.0 3.0.0 2.7.1 diff --git a/ratis-experiments/pom.xml b/ratis-experiments/pom.xml index ed13c993da..e4a8451db2 100644 --- a/ratis-experiments/pom.xml +++ b/ratis-experiments/pom.xml @@ -54,7 +54,6 @@ org.codehaus.mojo exec-maven-plugin - 1.6.0 compile-flatbufs From 885d1b7f8e09e2541d098d3ca5566e1cbc76c5e3 Mon Sep 17 00:00:00 2001 From: Ivan Andika Date: Mon, 6 Apr 2026 12:42:03 +0800 Subject: [PATCH 314/397] RATIS-2403. Support leader batch write to improve linearizable follower read throughput (#1362) --- .../ratis/client/impl/OrderedAsync.java | 2 +- .../src/site/markdown/configurations.md | 37 ++++- .../ratis/server/RaftServerConfigKeys.java | 33 +++- .../ratis/server/impl/LeaderStateImpl.java | 62 ++++++-- .../ratis/server/impl/PendingRequests.java | 5 +- .../ratis/server/impl/RaftServerImpl.java | 8 +- .../ratis/server/impl/ReplyFlusher.java | 137 +++++++++++++++++ .../apache/ratis/LinearizableReadTests.java | 47 ++++-- .../apache/ratis/OutputStreamBaseTest.java | 4 +- .../apache/ratis/ReadOnlyRequestTests.java | 17 +- ...stLinearizableLeaderLeaseReadWithGrpc.java | 18 +-- ...adAppliedIndexLeaderLeaseReadWithGrpc.java | 6 +- ...tLinearizableReadAppliedIndexWithGrpc.java | 6 +- ...leReadRepliedIndexLeaderLeaseWithGrpc.java | 27 ++++ ...tLinearizableReadRepliedIndexWithGrpc.java | 145 ++++++++++++++++++ .../grpc/TestLinearizableReadWithGrpc.java | 18 +-- 16 files changed, 492 insertions(+), 80 deletions(-) create mode 100644 ratis-server/src/main/java/org/apache/ratis/server/impl/ReplyFlusher.java create mode 100644 ratis-test/src/test/java/org/apache/ratis/grpc/TestLinearizableReadRepliedIndexLeaderLeaseWithGrpc.java create mode 100644 ratis-test/src/test/java/org/apache/ratis/grpc/TestLinearizableReadRepliedIndexWithGrpc.java diff --git a/ratis-client/src/main/java/org/apache/ratis/client/impl/OrderedAsync.java b/ratis-client/src/main/java/org/apache/ratis/client/impl/OrderedAsync.java index fbeb4b992a..ecf4db3dce 100644 --- a/ratis-client/src/main/java/org/apache/ratis/client/impl/OrderedAsync.java +++ b/ratis-client/src/main/java/org/apache/ratis/client/impl/OrderedAsync.java @@ -198,7 +198,7 @@ private void sendRequestWithRetry(PendingOrderedRequest pending) { return; } - if (getSlidingWindow((RaftPeerId) null).isFirst(pending.getSeqNum())) { + if (getSlidingWindow(request).isFirst(pending.getSeqNum())) { pending.setFirstRequest(); } LOG.debug("{}: send* {}", client.getId(), request); diff --git a/ratis-docs/src/site/markdown/configurations.md b/ratis-docs/src/site/markdown/configurations.md index 67e988348c..f5189ed862 100644 --- a/ratis-docs/src/site/markdown/configurations.md +++ b/ratis-docs/src/site/markdown/configurations.md @@ -220,11 +220,38 @@ if it fails to receive any RPC responses from this peer within this specified ti ### Read Index - Configurations related to ReadIndex used in linearizable read -| **Property** | `raft.server.read.read-index.applied-index.enabled` | -|:----------------|:----------------------------------------------------------------------| -| **Description** | whether applied index (instead of commit index) is used for ReadIndex | -| **Type** | boolean | -| **Default** | false | +| **Property** | `raft.server.read.read-index.type` | +|:----------------|:-----------------------------------------------------------------------------| +| **Description** | type of read index returned | +| **Type** | enum `Read.ReadIndex.Type` [`COMMIT_INDEX`, `APPLIED_INDEX`, `REPLIED_INDEX` | +| **Default** | `Read.ReadIndex.Type.COMMIT_INDEX` | + +* `Read.ReadIndex.Type.COMMIT_INDEX` - Use leader's CommitIndex (see Raft Paper section 6.4) + * The safest type as it is specified in the Raft dissertation + * This ReadIndex type can be chosen if the base linearizable read from followers performance already meets expectations. + +* `Read.ReadIndex.Type.APPLIED_INDEX` - Use leader's AppliedIndex + * Allow leader to return AppliedIndex (instead of CommitIndex) as the ReadIndex + * This reduces the time follower applying logs up to ReadIndex since AppliedIndex ≤ CommitIndex + * This ReadIndex type can be chosen `Read.ReadIndex.Type.COMMIT_INDEX` read latency is too high. + +* `Read.ReadIndex.Type.REPLIED_INDEX` - Use leader's RepliedIndex + * RepliedIndex is defined as the last AppliedIndex of the leader when returning the last batch. + * Leader delays replying write requests and only reply them every write batch boundary configurable by `raft.server.read.read-index.replied-index.batch-interval`. + * This allows the ReadIndex to advance in a coarser, less frequent steps, so followers are more likely to have already applied past the ReadIndex when a read arrives. + * This is most effective on read-heavy, follower-read workloads which prioritizes overall read throughput without consistency sacrifice. + * There is a trade-off in increased write latency (up to one `raft.server.read.read-index.replied-index.batch-interval`) per write. + * RepliedIndex still guarantees linearizability (no stale read) since by definition each ReadIndex returns the index of the last replied request. + * If the RepliedIndex is set to 0, the behavior is identical to `Read.ReadIndex.Type.APPLIED_INDEX` + +Note that theoretically all the ReadIndex types still guarantee linearizability, +but there are tradeoffs (e.g. Write and Read performance) between different types. + +| **Property** | `raft.server.read.read-index.replied-index.batch-interval` | +|:----------------|:--------------------------------------------------------------------------------------------------------------------------------------------| +| **Description** | if `Read.ReadIndex.Type` is `REPLIED_INDEX`, the interval at which held write replies are flushed to clients and `repliedIndex` is advanced | +| **Type** | TimeDuration | +| **Default** | 10ms | | **Property** | `raft.server.read.leader.heartbeat-check.enabled` | |:----------------|:--------------------------------------------------| diff --git a/ratis-server-api/src/main/java/org/apache/ratis/server/RaftServerConfigKeys.java b/ratis-server-api/src/main/java/org/apache/ratis/server/RaftServerConfigKeys.java index ef16f67f67..2d55594782 100644 --- a/ratis-server-api/src/main/java/org/apache/ratis/server/RaftServerConfigKeys.java +++ b/ratis-server-api/src/main/java/org/apache/ratis/server/RaftServerConfigKeys.java @@ -280,15 +280,34 @@ static void setWriteIndexCacheExpiryTime(RaftProperties properties, TimeDuration interface ReadIndex { String PREFIX = Read.PREFIX + ".read-index"; - String APPLIED_INDEX_ENABLED_KEY = PREFIX + ".applied-index.enabled"; - boolean APPLIED_INDEX_ENABLED_DEFAULT = false; - static boolean appliedIndexEnabled(RaftProperties properties) { - return getBoolean(properties::getBoolean, APPLIED_INDEX_ENABLED_KEY, - APPLIED_INDEX_ENABLED_DEFAULT, getDefaultLog()); + enum Type { + /** ReadIndex returns leader's commitIndex (see Raft Paper section 6.4). */ + COMMIT_INDEX, + + /** ReadIndex returns leader's appliedIndex to reduce the ReadIndex latency. */ + APPLIED_INDEX, + + /** ReadIndex returns leader's repliedIndex, the index of the last replied request. */ + REPLIED_INDEX + } + + String TYPE_KEY = PREFIX + ".type"; + Type TYPE_DEFAULT = Type.COMMIT_INDEX; + static Type type(RaftProperties properties) { + return get(properties::getEnum, TYPE_KEY, TYPE_DEFAULT, getDefaultLog()); + } + static void setType(RaftProperties properties, Type type) { + set(properties::setEnum, TYPE_KEY, type); } - static void setAppliedIndexEnabled(RaftProperties properties, boolean enabled) { - setBoolean(properties::setBoolean, APPLIED_INDEX_ENABLED_KEY, enabled); + String REPLIED_INDEX_BATCH_INTERVAL_KEY = PREFIX + ".replied-index.batch-interval"; + TimeDuration REPLIED_INDEX_BATCH_INTERVAL_DEFAULT = TimeDuration.valueOf(10, TimeUnit.MILLISECONDS); + static TimeDuration repliedIndexBatchInterval(RaftProperties properties) { + return getTimeDuration(properties.getTimeDuration(REPLIED_INDEX_BATCH_INTERVAL_DEFAULT.getUnit()), + REPLIED_INDEX_BATCH_INTERVAL_KEY, REPLIED_INDEX_BATCH_INTERVAL_DEFAULT, getDefaultLog()); + } + static void setRepliedIndexBatchInterval(RaftProperties properties, TimeDuration interval) { + setTimeDuration(properties::setTimeDuration, REPLIED_INDEX_BATCH_INTERVAL_KEY, interval); } } } diff --git a/ratis-server/src/main/java/org/apache/ratis/server/impl/LeaderStateImpl.java b/ratis-server/src/main/java/org/apache/ratis/server/impl/LeaderStateImpl.java index ef0bb6b700..1c986ca638 100644 --- a/ratis-server/src/main/java/org/apache/ratis/server/impl/LeaderStateImpl.java +++ b/ratis-server/src/main/java/org/apache/ratis/server/impl/LeaderStateImpl.java @@ -39,6 +39,7 @@ import org.apache.ratis.protocol.exceptions.ReadIndexException; import org.apache.ratis.protocol.exceptions.ReconfigurationTimeoutException; import org.apache.ratis.server.RaftServerConfigKeys; +import org.apache.ratis.server.RaftServerConfigKeys.Read.ReadIndex.Type; import org.apache.ratis.server.impl.ReadIndexHeartbeats.AppendEntriesListener; import org.apache.ratis.server.leader.FollowerInfo; import org.apache.ratis.server.leader.LeaderState; @@ -82,6 +83,7 @@ import java.util.concurrent.atomic.AtomicBoolean; import java.util.function.LongSupplier; import java.util.function.Predicate; +import java.util.function.Supplier; import java.util.function.ToLongFunction; import java.util.stream.Collectors; import java.util.stream.Stream; @@ -353,10 +355,13 @@ boolean isApplied() { private final PendingStepDown pendingStepDown; private final ReadIndexHeartbeats readIndexHeartbeats; - private final boolean readIndexAppliedIndexEnabled; + private final RaftServerConfigKeys.Read.ReadIndex.Type readIndexType; + private final Supplier readIndexSupplier; private final boolean leaderHeartbeatCheckEnabled; private final LeaderLease lease; + private ReplyFlusher replyFlusher; + LeaderStateImpl(RaftServerImpl server) { this.name = ServerStringUtils.generateUnifiedName(server.getMemberId(), getClass()); this.server = server; @@ -391,8 +396,21 @@ boolean isApplied() { } else { this.followerMaxGapThreshold = (long) (followerGapRatioMax * maxPendingRequests); } - this.readIndexAppliedIndexEnabled = RaftServerConfigKeys.Read.ReadIndex - .appliedIndexEnabled(properties); + + this.readIndexType = RaftServerConfigKeys.Read.ReadIndex.type(properties); + switch (readIndexType) { + case REPLIED_INDEX: + this.replyFlusher = new ReplyFlusher(server.getId(), state.getLastAppliedIndex(), + RaftServerConfigKeys.Read.ReadIndex.repliedIndexBatchInterval(properties)); + readIndexSupplier = replyFlusher::getRepliedIndex; + break; + case APPLIED_INDEX: + readIndexSupplier = () -> server.getState().getLastAppliedIndex(); + break; + case COMMIT_INDEX: + default: + readIndexSupplier = () -> server.getRaftLog().getLastCommittedIndex(); + } this.leaderHeartbeatCheckEnabled = RaftServerConfigKeys.Read .leaderHeartbeatCheckEnabled(properties); @@ -418,6 +436,11 @@ void start() { // Initialize startup log entry and append it to the RaftLog startupLogEntry.get(); processor.start(); + + if (replyFlusher != null) { + replyFlusher.start(startupLogEntry.get().startIndex); + } + senders.forEach(LogAppender::start); } @@ -453,6 +476,9 @@ CompletableFuture stop() { startupLogEntry.get().getAppliedIndexFuture().completeExceptionally( new ReadIndexException("failed to obtain read index since: ", nle)); server.getServerRpc().notifyNotLeader(server.getMemberId().getGroupId()); + if (replyFlusher != null) { + replyFlusher.stop(); + } logAppenderMetrics.unregister(); raftServerMetrics.unregister(); pendingRequests.close(); @@ -620,7 +646,7 @@ public AppendEntriesRequestProto newAppendEntriesRequestProto(FollowerInfo follo final boolean initializing = !isCaughtUp(follower); final RaftPeerId targetId = follower.getId(); return ServerProtoUtils.toAppendEntriesRequestProto(server.getMemberId(), targetId, getCurrentTerm(), entries, - ServerImplUtils.effectiveCommitIndex(raftLog.getLastCommittedIndex(), previous, entries.size()), + ServerImplUtils.effectiveCommitIndex(readIndexSupplier.get(), previous, entries.size()), initializing, previous, server.getCommitInfos(), callId); } @@ -1140,23 +1166,21 @@ public boolean checkLeadership() { /** * Obtain the current readIndex for read only requests. See Raft paper section 6.4. * 1. Leader makes sure at least one log from current term is committed. - * 2. Leader record last committed index or applied index (depending on configuration) as readIndex. + * 2. Leader record last committed index or applied index or replied index (depending on configuration) as readIndex. * 3. Leader broadcast heartbeats to followers and waits for acknowledgements. * 4. If majority respond success, returns readIndex. * @return current readIndex. */ CompletableFuture getReadIndex(Long readAfterWriteConsistentIndex) { - final long index = readIndexAppliedIndexEnabled ? - server.getState().getLastAppliedIndex() : server.getRaftLog().getLastCommittedIndex(); + final long index = readIndexSupplier.get(); final long readIndex; if (readAfterWriteConsistentIndex != null && readAfterWriteConsistentIndex > index) { readIndex = readAfterWriteConsistentIndex; } else { readIndex = index; } - LOG.debug("readIndex={} ({}Index={}, readAfterWriteConsistentIndex={})", - readIndex, readIndexAppliedIndexEnabled ? "applied" : "commit", - index, readAfterWriteConsistentIndex); + LOG.debug("readIndex={} ({}={}, readAfterWriteConsistentIndex={})", + readIndex, readIndexType, index, readAfterWriteConsistentIndex); // if group contains only one member, fast path if (server.getRaftConf().isSingleton()) { @@ -1217,8 +1241,22 @@ private boolean checkLeaderLease() { && (server.getRaftConf().isSingleton() || lease.isValid()); } - void replyPendingRequest(TermIndex termIndex, RaftClientReply reply) { - pendingRequests.replyPendingRequest(termIndex, reply); + void replyPendingRequest(TermIndex termIndex, RaftClientReply reply, RetryCacheImpl.CacheEntry cacheEntry) { + final PendingRequest pending = pendingRequests.remove(termIndex); + + final LongSupplier replyMethod = () -> { + cacheEntry.updateResult(reply); + if (pending != null) { + pending.setReply(reply); + } + return termIndex.getIndex(); + }; + + if (readIndexType == Type.REPLIED_INDEX) { + replyFlusher.hold(replyMethod); + } else { + replyMethod.getAsLong(); + } } TransactionContext getTransactionContext(TermIndex termIndex) { diff --git a/ratis-server/src/main/java/org/apache/ratis/server/impl/PendingRequests.java b/ratis-server/src/main/java/org/apache/ratis/server/impl/PendingRequests.java index c6a9dd2794..f89d354e6a 100644 --- a/ratis-server/src/main/java/org/apache/ratis/server/impl/PendingRequests.java +++ b/ratis-server/src/main/java/org/apache/ratis/server/impl/PendingRequests.java @@ -264,12 +264,13 @@ TransactionContext getTransactionContext(TermIndex termIndex) { return pendingRequest != null ? pendingRequest.getEntry() : null; } - void replyPendingRequest(TermIndex termIndex, RaftClientReply reply) { + /** @return the removed the {@link PendingRequest} for the given {@link TermIndex}. */ + PendingRequest remove(TermIndex termIndex) { final PendingRequest pending = pendingRequests.remove(termIndex); if (pending != null) { Preconditions.assertEquals(termIndex, pending.getTermIndex(), "termIndex"); - pending.setReply(reply); } + return pending; } /** diff --git a/ratis-server/src/main/java/org/apache/ratis/server/impl/RaftServerImpl.java b/ratis-server/src/main/java/org/apache/ratis/server/impl/RaftServerImpl.java index 958da846d2..d4c6f164e3 100644 --- a/ratis-server/src/main/java/org/apache/ratis/server/impl/RaftServerImpl.java +++ b/ratis-server/src/main/java/org/apache/ratis/server/impl/RaftServerImpl.java @@ -1837,8 +1837,12 @@ private CompletableFuture replyPendingRequest( } // update pending request - role.getLeaderState().ifPresent(leader -> leader.replyPendingRequest(termIndex, r)); - cacheEntry.updateResult(r); + final LeaderStateImpl leader = role.getLeaderState().orElse(null); + if (leader != null) { + leader.replyPendingRequest(termIndex, r, cacheEntry); + } else { + cacheEntry.updateResult(r); + } }); } diff --git a/ratis-server/src/main/java/org/apache/ratis/server/impl/ReplyFlusher.java b/ratis-server/src/main/java/org/apache/ratis/server/impl/ReplyFlusher.java new file mode 100644 index 0000000000..47e9967c11 --- /dev/null +++ b/ratis-server/src/main/java/org/apache/ratis/server/impl/ReplyFlusher.java @@ -0,0 +1,137 @@ +/* + * 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.ratis.server.impl; + +import org.apache.ratis.server.raftlog.RaftLogIndex; +import org.apache.ratis.util.CodeInjectionForTesting; +import org.apache.ratis.util.Daemon; +import org.apache.ratis.util.JavaUtils; +import org.apache.ratis.util.LifeCycle; +import org.apache.ratis.util.TimeDuration; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.util.LinkedList; +import java.util.concurrent.TimeUnit; +import java.util.function.LongSupplier; + +/** + * Implements the reply flush logic as part of the leader batch write when RepliedIndex is used. + */ +public class ReplyFlusher { + static final Logger LOG = LoggerFactory.getLogger(ReplyFlusher.class); + + private static final String CLASS_NAME = JavaUtils.getClassSimpleName(RaftServerImpl.class); + public static final String FLUSH = CLASS_NAME + ".flush"; + + static class Replies { + /** When a {@link LongSupplier} is invoked, it completes a write reply and return the log index. */ + private LinkedList list = new LinkedList<>(); + + synchronized void add(LongSupplier replyMethod) { + list.add(replyMethod); + } + + synchronized LinkedList getAndSetNewList() { + final LinkedList old = list; + list = new LinkedList<>(); + return old; + } + } + + private final Object id; + private final LifeCycle lifeCycle; + private final Daemon daemon; + private final Replies replies = new Replies(); + private final RaftLogIndex repliedIndex; + /** The interval at which held write replies are flushed. */ + private final TimeDuration batchInterval; + + ReplyFlusher(Object id, long repliedIndex, TimeDuration batchInterval) { + this.id = id; + final String name = id + "-ReplyFlusher"; + this.lifeCycle = new LifeCycle(name); + this.daemon = Daemon.newBuilder() + .setName(name) + .setRunnable(this::run) + .build(); + this.repliedIndex = new RaftLogIndex("repliedIndex", repliedIndex); + this.batchInterval = batchInterval; + } + + long getRepliedIndex() { + return repliedIndex.get(); + } + + /** Hold a write reply for later batch flushing */ + void hold(LongSupplier replyMethod) { + replies.add(replyMethod); + } + + void start(long startIndex) { + repliedIndex.updateToMax(startIndex, s -> LOG.debug("{}: {}", id, s)); + lifeCycle.transition(LifeCycle.State.STARTING); + // We need to transition to RUNNING first so that ReplyFlusher#run always + // see that the lifecycle state is in RUNNING state. + lifeCycle.transition(LifeCycle.State.RUNNING); + daemon.start(); + } + + /** The reply flusher daemon loop. */ + private void run() { + try { + while (lifeCycle.getCurrentState() == LifeCycle.State.RUNNING) { + batchInterval.sleep(); + flush(); + } + } catch (InterruptedException e) { + Thread.currentThread().interrupt(); + LOG.warn("{}: Interrupted ", daemon.getName(), e); + } finally { + // Flush remaining on exit + flush(); + } + } + + /** Flush all held replies and advance {@link #repliedIndex}. */ + private void flush() { + CodeInjectionForTesting.execute(FLUSH, id, null); + + final LinkedList toFlush = replies.getAndSetNewList(); + if (toFlush.isEmpty()) { + return; + } + long maxIndex = toFlush.removeLast().getAsLong(); + for (LongSupplier held : toFlush) { + maxIndex = Math.max(maxIndex, held.getAsLong()); + } + repliedIndex.updateToMax(maxIndex, s -> + LOG.debug("{}: flushed {} replies, {}", id, toFlush.size(), s)); + } + + /** Stop the reply flusher daemon. */ + void stop() { + lifeCycle.checkStateAndClose(); + daemon.interrupt(); + try { + daemon.join(batchInterval.toLong(TimeUnit.MILLISECONDS )* 2); + } catch (InterruptedException e) { + Thread.currentThread().interrupt(); + } + } +} diff --git a/ratis-server/src/test/java/org/apache/ratis/LinearizableReadTests.java b/ratis-server/src/test/java/org/apache/ratis/LinearizableReadTests.java index b15ae3067f..dd536508ce 100644 --- a/ratis-server/src/test/java/org/apache/ratis/LinearizableReadTests.java +++ b/ratis-server/src/test/java/org/apache/ratis/LinearizableReadTests.java @@ -18,6 +18,7 @@ package org.apache.ratis; import org.apache.ratis.client.RaftClient; +import org.apache.ratis.client.RaftClientConfigKeys; import org.apache.ratis.conf.RaftProperties; import org.apache.ratis.protocol.RaftClientReply; import org.apache.ratis.protocol.RaftPeerId; @@ -27,6 +28,7 @@ import org.apache.ratis.retry.RetryPolicy; import org.apache.ratis.server.RaftServer; import org.apache.ratis.server.RaftServerConfigKeys; +import org.apache.ratis.server.RaftServerConfigKeys.Read.ReadIndex.Type; import org.apache.ratis.server.impl.MiniRaftCluster; import org.apache.ratis.util.Slf4jUtils; import org.apache.ratis.util.TimeDuration; @@ -45,9 +47,12 @@ import static org.apache.ratis.ReadOnlyRequestTests.INCREMENT; import static org.apache.ratis.ReadOnlyRequestTests.QUERY; import static org.apache.ratis.ReadOnlyRequestTests.WAIT_AND_INCREMENT; +import static org.apache.ratis.ReadOnlyRequestTests.assertOption; import static org.apache.ratis.ReadOnlyRequestTests.assertReplyAtLeast; import static org.apache.ratis.ReadOnlyRequestTests.assertReplyExact; import static org.apache.ratis.server.RaftServerConfigKeys.Read.Option.LINEARIZABLE; +import static org.junit.jupiter.api.Assertions.assertEquals; +import static org.junit.jupiter.api.Assertions.assertSame; /** Test for the {@link RaftServerConfigKeys.Read.Option#LINEARIZABLE} feature. */ public abstract class LinearizableReadTests @@ -56,15 +61,20 @@ public abstract class LinearizableReadTests { Slf4jUtils.setLogLevel(RaftServer.Division.LOG, Level.DEBUG); + Slf4jUtils.setLogLevel(RaftClient.LOG, Level.DEBUG); } public abstract boolean isLeaderLeaseEnabled(); - public abstract boolean readIndexAppliedIndexEnabled(); + public abstract Type readIndexType(); - public abstract void assertRaftProperties(RaftProperties properties); + public final void assertRaftProperties(RaftProperties p) { + assertOption(LINEARIZABLE, p); + assertEquals(isLeaderLeaseEnabled(), RaftServerConfigKeys.Read.leaderLeaseEnabled(p)); + assertSame(readIndexType(), RaftServerConfigKeys.Read.ReadIndex.type(p)); + } - void runWithNewCluster(CheckedConsumer testCase) throws Exception { + protected void runWithNewCluster(CheckedConsumer testCase) throws Exception { runWithNewCluster(3, 0, true, cluster -> { assertRaftProperties(cluster.getProperties()); testCase.accept(cluster); @@ -77,7 +87,11 @@ public void setup() { CounterStateMachine.setProperties(p); RaftServerConfigKeys.Read.setOption(p, LINEARIZABLE); RaftServerConfigKeys.Read.setLeaderLeaseEnabled(p, isLeaderLeaseEnabled()); - RaftServerConfigKeys.Read.ReadIndex.setAppliedIndexEnabled(p, readIndexAppliedIndexEnabled()); + RaftServerConfigKeys.Read.ReadIndex.setType(p, readIndexType()); + // Disable dummy request since currently the request is implemented as a watch request + // which can cause follower client to trigger failover to leader which will cause the + // all reads to be sent to the leader, making the follower read moot. + RaftClientConfigKeys.Async.Experimental.setSendDummyRequest(p, false); } @Test @@ -95,22 +109,34 @@ public void testFollowerLinearizableRead() throws Exception { runWithNewCluster(LinearizableReadTests::runTestFollowerLinearizableRead); } - static class Reply { + public static class Reply { private final int count; private final CompletableFuture future; - Reply(int count, CompletableFuture future) { + public Reply(int count, CompletableFuture future) { this.count = count; this.future = future; } - void assertExact() { + public boolean isDone() { + return future.isDone(); + } + + public void assertExact() { assertReplyExact(count, future.join()); } - void assertAtLeast() { + public void assertAtLeast() { assertReplyAtLeast(count, future.join()); } + + @Override + public String toString() { + return "Reply{" + + "count=" + count + + ", reply=" + (isDone() ? future.join() : "pending") + + '}'; + } } static void runTestFollowerLinearizableRead(C cluster) throws Exception { @@ -167,8 +193,9 @@ static void runTestFollowerReadOnlyParallel(C cluste count++; writeReplies.add(new Reply(count, leaderClient.async().send(WAIT_AND_INCREMENT))); + // sleep to let the commitIndex/appliedIndex get updated. Thread.sleep(100); - + // WAIT_AND_INCREMENT will delay 500ms to update the count, the read must wait for it. assertReplyExact(count, f0Client.io().sendReadOnly(QUERY, f0)); f1Replies.add(new Reply(count, f1Client.async().sendReadOnly(QUERY, f1))); } @@ -189,7 +216,7 @@ static void runTestLinearizableReadFailWhenLeaderDow final RaftPeerId leaderId = RaftTestUtil.waitForLeader(cluster).getId(); final List followers = cluster.getFollowers(); - Assertions.assertEquals(2, followers.size()); + assertEquals(2, followers.size()); final RaftPeerId f0 = followers.get(0).getId(); try (RaftClient leaderClient = cluster.createClient(leaderId); diff --git a/ratis-server/src/test/java/org/apache/ratis/OutputStreamBaseTest.java b/ratis-server/src/test/java/org/apache/ratis/OutputStreamBaseTest.java index a17cdb0d58..9821126ce6 100644 --- a/ratis-server/src/test/java/org/apache/ratis/OutputStreamBaseTest.java +++ b/ratis-server/src/test/java/org/apache/ratis/OutputStreamBaseTest.java @@ -300,8 +300,10 @@ private void runTestKillLeader(CLUSTER cluster) throws Exception { Thread.sleep(500); running.set(false); - latch.await(5, TimeUnit.SECONDS); + final boolean latchCompleted = latch.await(5, TimeUnit.SECONDS); + Assertions.assertTrue(latchCompleted, "Writer thread did not finish within the timeout"); LOG.info("Writer success? " + success.get()); + Assertions.assertNotNull(success.get(), "Writer thread completed but success was not set"); Assertions.assertTrue(success.get()); // total number of tx should be >= result + 2, where 2 means two NoOp from // leaders. It may be larger than result+2 because the client may resend diff --git a/ratis-server/src/test/java/org/apache/ratis/ReadOnlyRequestTests.java b/ratis-server/src/test/java/org/apache/ratis/ReadOnlyRequestTests.java index aa77ee5c77..94e9433b15 100644 --- a/ratis-server/src/test/java/org/apache/ratis/ReadOnlyRequestTests.java +++ b/ratis-server/src/test/java/org/apache/ratis/ReadOnlyRequestTests.java @@ -58,9 +58,9 @@ public abstract class ReadOnlyRequestTests static final String WAIT_AND_INCREMENT_STRING = "WAIT_AND_INCREMENT"; static final String QUERY_STRING = "QUERY"; - static final Message INCREMENT = new RaftTestUtil.SimpleMessage(INCREMENT_STRING); - static final Message WAIT_AND_INCREMENT = new RaftTestUtil.SimpleMessage(WAIT_AND_INCREMENT_STRING); - static final Message QUERY = new RaftTestUtil.SimpleMessage(QUERY_STRING); + public static final Message INCREMENT = new RaftTestUtil.SimpleMessage(INCREMENT_STRING); + public static final Message WAIT_AND_INCREMENT = new RaftTestUtil.SimpleMessage(WAIT_AND_INCREMENT_STRING); + public static final Message QUERY = new RaftTestUtil.SimpleMessage(QUERY_STRING); @BeforeEach public void setup() { @@ -144,7 +144,7 @@ static int retrieve(RaftClientReply reply) { return Integer.parseInt(reply.getMessage().getContent().toString(StandardCharsets.UTF_8)); } - static void assertReplyExact(int expectedCount, RaftClientReply reply) { + public static void assertReplyExact(int expectedCount, RaftClientReply reply) { Assertions.assertTrue(reply.isSuccess()); final int retrieved = retrieve(reply); Assertions.assertEquals(expectedCount, retrieved, () -> "reply=" + reply); @@ -163,7 +163,7 @@ static void assertReplyAtLeast(int minCount, RaftClientReply reply) { * 2. get * 3. waitAndIncrement */ - static class CounterStateMachine extends BaseStateMachine { + public static class CounterStateMachine extends BaseStateMachine { static void setProperties(RaftProperties properties) { properties.setClass(MiniRaftCluster.STATEMACHINE_CLASS_KEY, CounterStateMachine.class, StateMachine.class); } @@ -193,6 +193,10 @@ private void sleepQuietly(int millis) { } } + public long getCount() { + return counter.get(); + } + private long increment() { return counter.incrementAndGet(); } @@ -213,6 +217,7 @@ public CompletableFuture applyTransaction(TransactionContext trx) { final LogEntryProto logEntry = trx.getLogEntry(); final TermIndex ti = TermIndex.valueOf(logEntry); updateLastAppliedTermIndex(ti); + LOG.info("{}: updateLastAppliedTermIndex {}", getId(), ti); final String command = logEntry.getStateMachineLogEntry().getLogData().toString(StandardCharsets.UTF_8); @@ -224,7 +229,7 @@ public CompletableFuture applyTransaction(TransactionContext trx) { } else { updatedCount = timeoutIncrement(); } - LOG.info("Applied {} command {}, updatedCount={}", ti, command, updatedCount); + LOG.info("{}: Applied {} command {}, updatedCount={}", getId(), ti, command, updatedCount); return toMessageFuture(updatedCount); } diff --git a/ratis-test/src/test/java/org/apache/ratis/grpc/TestLinearizableLeaderLeaseReadWithGrpc.java b/ratis-test/src/test/java/org/apache/ratis/grpc/TestLinearizableLeaderLeaseReadWithGrpc.java index d637498d73..120cce48cc 100644 --- a/ratis-test/src/test/java/org/apache/ratis/grpc/TestLinearizableLeaderLeaseReadWithGrpc.java +++ b/ratis-test/src/test/java/org/apache/ratis/grpc/TestLinearizableLeaderLeaseReadWithGrpc.java @@ -18,12 +18,7 @@ package org.apache.ratis.grpc; import org.apache.ratis.LinearizableReadTests; -import org.apache.ratis.conf.RaftProperties; -import org.apache.ratis.server.RaftServerConfigKeys; - -import static org.apache.ratis.ReadOnlyRequestTests.assertOption; -import static org.apache.ratis.server.RaftServerConfigKeys.Read.Option.LINEARIZABLE; -import static org.junit.jupiter.api.Assertions.assertTrue; +import org.apache.ratis.server.RaftServerConfigKeys.Read.ReadIndex.Type; public class TestLinearizableLeaderLeaseReadWithGrpc extends LinearizableReadTests @@ -35,14 +30,7 @@ public boolean isLeaderLeaseEnabled() { } @Override - public boolean readIndexAppliedIndexEnabled() { - return false; - } - - @Override - public void assertRaftProperties(RaftProperties p) { - assertOption(LINEARIZABLE, p); - assertTrue(RaftServerConfigKeys.Read.leaderLeaseEnabled(p)); - assertTrue(isLeaderLeaseEnabled()); + public Type readIndexType() { + return Type.COMMIT_INDEX; } } diff --git a/ratis-test/src/test/java/org/apache/ratis/grpc/TestLinearizableReadAppliedIndexLeaderLeaseReadWithGrpc.java b/ratis-test/src/test/java/org/apache/ratis/grpc/TestLinearizableReadAppliedIndexLeaderLeaseReadWithGrpc.java index 9bf3e307be..3705fb3ffc 100644 --- a/ratis-test/src/test/java/org/apache/ratis/grpc/TestLinearizableReadAppliedIndexLeaderLeaseReadWithGrpc.java +++ b/ratis-test/src/test/java/org/apache/ratis/grpc/TestLinearizableReadAppliedIndexLeaderLeaseReadWithGrpc.java @@ -17,11 +17,13 @@ */ package org.apache.ratis.grpc; +import org.apache.ratis.server.RaftServerConfigKeys.Read.ReadIndex.Type; + public class TestLinearizableReadAppliedIndexLeaderLeaseReadWithGrpc extends TestLinearizableLeaderLeaseReadWithGrpc { @Override - public boolean readIndexAppliedIndexEnabled() { - return true; + public Type readIndexType() { + return Type.APPLIED_INDEX; } } diff --git a/ratis-test/src/test/java/org/apache/ratis/grpc/TestLinearizableReadAppliedIndexWithGrpc.java b/ratis-test/src/test/java/org/apache/ratis/grpc/TestLinearizableReadAppliedIndexWithGrpc.java index c019aac166..b119f32a6f 100644 --- a/ratis-test/src/test/java/org/apache/ratis/grpc/TestLinearizableReadAppliedIndexWithGrpc.java +++ b/ratis-test/src/test/java/org/apache/ratis/grpc/TestLinearizableReadAppliedIndexWithGrpc.java @@ -17,11 +17,13 @@ */ package org.apache.ratis.grpc; +import org.apache.ratis.server.RaftServerConfigKeys.Read.ReadIndex.Type; + public class TestLinearizableReadAppliedIndexWithGrpc extends TestLinearizableReadWithGrpc { @Override - public boolean readIndexAppliedIndexEnabled() { - return true; + public Type readIndexType() { + return Type.APPLIED_INDEX; } } diff --git a/ratis-test/src/test/java/org/apache/ratis/grpc/TestLinearizableReadRepliedIndexLeaderLeaseWithGrpc.java b/ratis-test/src/test/java/org/apache/ratis/grpc/TestLinearizableReadRepliedIndexLeaderLeaseWithGrpc.java new file mode 100644 index 0000000000..bb50eafbfc --- /dev/null +++ b/ratis-test/src/test/java/org/apache/ratis/grpc/TestLinearizableReadRepliedIndexLeaderLeaseWithGrpc.java @@ -0,0 +1,27 @@ +/* + * 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.ratis.grpc; + +public class TestLinearizableReadRepliedIndexLeaderLeaseWithGrpc + extends TestLinearizableReadRepliedIndexWithGrpc { + + @Override + public boolean isLeaderLeaseEnabled() { + return true; + } +} diff --git a/ratis-test/src/test/java/org/apache/ratis/grpc/TestLinearizableReadRepliedIndexWithGrpc.java b/ratis-test/src/test/java/org/apache/ratis/grpc/TestLinearizableReadRepliedIndexWithGrpc.java new file mode 100644 index 0000000000..f08346fc02 --- /dev/null +++ b/ratis-test/src/test/java/org/apache/ratis/grpc/TestLinearizableReadRepliedIndexWithGrpc.java @@ -0,0 +1,145 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.ratis.grpc; + +import org.apache.ratis.RaftTestUtil; +import org.apache.ratis.ReadOnlyRequestTests.CounterStateMachine; +import org.apache.ratis.client.RaftClient; +import org.apache.ratis.protocol.RaftPeerId; +import org.apache.ratis.server.RaftServer; +import org.apache.ratis.server.RaftServerConfigKeys.Read.ReadIndex.Type; +import org.apache.ratis.server.impl.MiniRaftCluster; +import org.apache.ratis.server.impl.ReplyFlusher; +import org.apache.ratis.util.CodeInjectionForTesting; +import org.junit.jupiter.api.Assertions; +import org.junit.jupiter.api.Test; + +import java.util.ArrayList; +import java.util.List; +import java.util.concurrent.CompletableFuture; + +import static org.apache.ratis.ReadOnlyRequestTests.INCREMENT; +import static org.apache.ratis.ReadOnlyRequestTests.QUERY; +import static org.apache.ratis.ReadOnlyRequestTests.assertReplyExact; +import static org.junit.jupiter.api.Assertions.assertEquals; +import static org.junit.jupiter.api.Assertions.assertFalse; +import static org.junit.jupiter.api.Assertions.assertTrue; + +public class TestLinearizableReadRepliedIndexWithGrpc + extends TestLinearizableReadWithGrpc { + + @Override + public Type readIndexType() { + return Type.REPLIED_INDEX; + } + + @Test + @Override + public void testFollowerLinearizableReadParallel() throws Exception { + runWithNewCluster(TestLinearizableReadRepliedIndexWithGrpc::runTestFollowerReadOnlyParallelRepliedIndex); + } + + static void runTestFollowerReadOnlyParallelRepliedIndex(C cluster) + throws Exception { + final RaftServer.Division leader = RaftTestUtil.waitForLeader(cluster); + final CounterStateMachine leaderStateMachine = (CounterStateMachine)leader.getStateMachine(); + + final List followers = cluster.getFollowers(); + Assertions.assertEquals(2, followers.size()); + final RaftPeerId f0 = followers.get(0).getId(); + final RaftPeerId f1 = followers.get(1).getId(); + + final BlockingCode blockingReplyFlusher = new BlockingCode(); + + try (RaftClient leaderClient = cluster.createClient(leader.getId()); + RaftClient f0Client = cluster.createClient(f0); + RaftClient f1Client = cluster.createClient(f1)) { + // Warm up the clients first before blocking the reply flusher + assertReplyExact(0, leaderClient.async().sendReadOnly(QUERY).get()); + assertReplyExact(0, f0Client.async().sendReadOnly(QUERY, f0).get()); + assertReplyExact(0, f1Client.async().sendReadOnly(QUERY, f1).get()); + + CodeInjectionForTesting.put(ReplyFlusher.FLUSH, blockingReplyFlusher); + + final int n = 10; + final List writeReplies = new ArrayList<>(n); + final List f0Replies = new ArrayList<>(n); + final List f1Replies = new ArrayList<>(n); + for (int i = 0; i < n; i++) { + final int count = i + 1; + writeReplies.add(new Reply(count, leaderClient.async().send(INCREMENT))); + + // Read reply returns immediately, but they all should return 0 since the repliedIndex has not been updated + // and write operations should not been applied by the followers + f0Replies.add(new Reply(0, f0Client.async().sendReadOnly(QUERY, f0))); + f1Replies.add(new Reply(0, f1Client.async().sendReadOnly(QUERY, f1))); + + // sleep in order to make sure + // (1) the count is incremented, and + // (2) the reads will wait for the repliedIndex. + Thread.sleep(100); + assertEquals(count, leaderStateMachine.getCount()); + } + + for (int i = 0; i < n; i++) { + // Write reply should not yet complete since ReplyFlusher remains blocked. + assertFalse(writeReplies.get(i).isDone(), "Received unexpected Write reply " + writeReplies.get(i)); + + // Follower reads should be immediately served, but the read value should return the value before the + // replyFlusher is blocked + assertTrue(f0Replies.get(i).isDone(), "Follower read should return immediately"); + f0Replies.get(i).assertExact(); + assertTrue(f1Replies.get(i).isDone(), "Follower read should return immediately"); + f1Replies.get(i).assertExact(); + } + + // unblock ReplyFlusher + blockingReplyFlusher.complete(); + assertReplyExact(n, f0Client.io().sendReadOnly(QUERY, f0)); + assertReplyExact(n, f1Client.io().sendReadOnly(QUERY, f0)); + + for (int i = 0; i < n; i++) { + //write reply should get the exact count at the write time + writeReplies.get(i).assertExact(); + } + } + } + + static class BlockingCode implements CodeInjectionForTesting.Code { + private final CompletableFuture future = new CompletableFuture<>(); + + void complete() { + future.complete(null); + } + + @Override + public boolean execute(Object localId, Object remoteId, Object... args) { + final boolean blocked = !future.isDone(); + if (blocked) { + LOG.info("{}: ReplyFlusher is blocked", localId, new Throwable()); + } + future.join(); + if (blocked) { + LOG.info("{}: ReplyFlusher is unblocked", localId); + } + return true; + } + } + + +} diff --git a/ratis-test/src/test/java/org/apache/ratis/grpc/TestLinearizableReadWithGrpc.java b/ratis-test/src/test/java/org/apache/ratis/grpc/TestLinearizableReadWithGrpc.java index 3e8860dd19..77593ff85e 100644 --- a/ratis-test/src/test/java/org/apache/ratis/grpc/TestLinearizableReadWithGrpc.java +++ b/ratis-test/src/test/java/org/apache/ratis/grpc/TestLinearizableReadWithGrpc.java @@ -18,12 +18,7 @@ package org.apache.ratis.grpc; import org.apache.ratis.LinearizableReadTests; -import org.apache.ratis.conf.RaftProperties; -import org.apache.ratis.server.RaftServerConfigKeys; - -import static org.apache.ratis.ReadOnlyRequestTests.assertOption; -import static org.apache.ratis.server.RaftServerConfigKeys.Read.Option.LINEARIZABLE; -import static org.junit.jupiter.api.Assertions.assertFalse; +import org.apache.ratis.server.RaftServerConfigKeys.Read.ReadIndex.Type; public class TestLinearizableReadWithGrpc extends LinearizableReadTests @@ -35,14 +30,7 @@ public boolean isLeaderLeaseEnabled() { } @Override - public boolean readIndexAppliedIndexEnabled() { - return false; - } - - @Override - public void assertRaftProperties(RaftProperties p) { - assertOption(LINEARIZABLE, p); - assertFalse(RaftServerConfigKeys.Read.leaderLeaseEnabled(p)); - assertFalse(isLeaderLeaseEnabled()); + public Type readIndexType() { + return Type.COMMIT_INDEX; } } From e0d6a32fb95ac6decee89fa295fb0d56669c7b60 Mon Sep 17 00:00:00 2001 From: XiChen <32928346+xichen01@users.noreply.github.com> Date: Wed, 8 Apr 2026 19:49:24 +0800 Subject: [PATCH 315/397] RATIS-2499. Allow the LogAppender restart when LogAppenderDaemon exception (#1425) --- .../ratis/server/leader/LogAppenderBase.java | 14 ++++- .../server/leader/LogAppenderDaemon.java | 4 ++ .../org/apache/ratis/LogAppenderTests.java | 1 + .../ratis/grpc/TestLogAppenderWithGrpc.java | 63 +++++++++++++++++++ 4 files changed, 79 insertions(+), 3 deletions(-) diff --git a/ratis-server/src/main/java/org/apache/ratis/server/leader/LogAppenderBase.java b/ratis-server/src/main/java/org/apache/ratis/server/leader/LogAppenderBase.java index 4f558e0c7c..f65ac1863c 100644 --- a/ratis-server/src/main/java/org/apache/ratis/server/leader/LogAppenderBase.java +++ b/ratis-server/src/main/java/org/apache/ratis/server/leader/LogAppenderBase.java @@ -125,7 +125,11 @@ public void start() { @Override public boolean isRunning() { return daemon.isWorking() - && server.getInfo().isAlive() + && isLeaderAlive(); + } + + private boolean isLeaderAlive() { + return server.getInfo().isAlive() && server.getInfo().isLeader() && getRaftLog().isOpened(); } @@ -136,8 +140,12 @@ public CompletableFuture stopAsync() { } void restart() { - if (!isRunning()) { - LOG.warn("{} is not running: skipping restart", this); + if (daemon.isClosingOrClosed()) { + LOG.warn("{}: daemon is closing or closed, skipping restart", this); + return; + } + if (!isLeaderAlive()) { + LOG.warn("{}: leader is not ready, skipping restart", this); return; } getLeaderState().restart(this); diff --git a/ratis-server/src/main/java/org/apache/ratis/server/leader/LogAppenderDaemon.java b/ratis-server/src/main/java/org/apache/ratis/server/leader/LogAppenderDaemon.java index 5de3f3b4da..c779d007ba 100644 --- a/ratis-server/src/main/java/org/apache/ratis/server/leader/LogAppenderDaemon.java +++ b/ratis-server/src/main/java/org/apache/ratis/server/leader/LogAppenderDaemon.java @@ -58,6 +58,10 @@ public boolean isWorking() { return !LifeCycle.States.CLOSING_OR_CLOSED_OR_EXCEPTION.contains(lifeCycle.getCurrentState()); } + public boolean isClosingOrClosed() { + return LifeCycle.States.CLOSING_OR_CLOSED.contains(lifeCycle.getCurrentState()); + } + public void tryToStart() { if (lifeCycle.compareAndTransition(NEW, STARTING)) { daemon.start(); diff --git a/ratis-server/src/test/java/org/apache/ratis/LogAppenderTests.java b/ratis-server/src/test/java/org/apache/ratis/LogAppenderTests.java index c9b19a72a1..8a8731daf4 100644 --- a/ratis-server/src/test/java/org/apache/ratis/LogAppenderTests.java +++ b/ratis-server/src/test/java/org/apache/ratis/LogAppenderTests.java @@ -171,6 +171,7 @@ public void testFollowerHeartbeatMetric() throws IOException, InterruptedExcepti assertTrue(t.getTimer().getCount() > 0L); } } + cluster.shutdown(); } void runTest(CLUSTER cluster) throws Exception { diff --git a/ratis-test/src/test/java/org/apache/ratis/grpc/TestLogAppenderWithGrpc.java b/ratis-test/src/test/java/org/apache/ratis/grpc/TestLogAppenderWithGrpc.java index 107cd7ba9a..318ed5e6bc 100644 --- a/ratis-test/src/test/java/org/apache/ratis/grpc/TestLogAppenderWithGrpc.java +++ b/ratis-test/src/test/java/org/apache/ratis/grpc/TestLogAppenderWithGrpc.java @@ -18,7 +18,10 @@ package org.apache.ratis.grpc; import org.apache.ratis.LogAppenderTests; +import org.apache.ratis.grpc.server.GrpcServicesImpl; import org.apache.ratis.proto.RaftProtos; +import org.apache.ratis.protocol.RaftGroupId; +import org.apache.ratis.protocol.RaftPeerId; import org.apache.ratis.server.impl.MiniRaftCluster; import org.apache.ratis.RaftTestUtil; import org.apache.ratis.client.RaftClient; @@ -29,11 +32,14 @@ import org.apache.ratis.server.RaftServerConfigKeys; import org.apache.ratis.server.leader.FollowerInfo; import org.apache.ratis.server.impl.RaftServerTestUtil; +import org.apache.ratis.server.leader.LogAppender; import org.apache.ratis.statemachine.impl.SimpleStateMachine4Testing; import org.apache.ratis.statemachine.StateMachine; +import org.apache.ratis.util.CodeInjectionForTesting; import org.apache.ratis.util.JavaUtils; import org.apache.ratis.util.Slf4jUtils; import org.junit.jupiter.api.Assertions; +import org.junit.jupiter.api.Test; import org.junit.jupiter.params.ParameterizedTest; import org.junit.jupiter.params.provider.MethodSource; import org.slf4j.event.Level; @@ -42,7 +48,10 @@ import java.util.ArrayList; import java.util.Arrays; import java.util.Collection; +import java.util.Set; import java.util.concurrent.CompletableFuture; +import java.util.concurrent.atomic.AtomicInteger; +import java.util.stream.Collectors; import static org.apache.ratis.RaftTestUtil.waitForLeader; @@ -148,4 +157,58 @@ private void runTestRestartLogAppender(MiniRaftClusterWithGrpc cluster) throws E Assertions.assertTrue(newleaderMetrics.getRegistry().counter(counter).getCount() >= 1L); } } + + @Test + public void testLogAppenderAutoRestartOnException() throws Exception { + runWithNewCluster(3, this::runTestAutoRestartOnException); + } + + private void runTestAutoRestartOnException(MiniRaftClusterWithGrpc cluster) throws Exception { + final RaftServer.Division leader = waitForLeader(cluster); + final RaftPeerId leaderId = leader.getId(); + + try (RaftClient client = cluster.createClient(leaderId)) { + for (int i = 0; i < 5; i++) { + Assertions.assertTrue(client.io().send(new RaftTestUtil.SimpleMessage("init-" + i)).isSuccess()); + } + } + + final Set before = RaftServerTestUtil.getLogAppenders(leader).collect(Collectors.toSet()); + Assertions.assertEquals(2, before.size()); + + // Inject a one-time IllegalStateException into the leader's AppendEntries send path. + // This causes the LogAppenderDaemon to enter EXCEPTION state and call restart(). + final RaftGroupId groupId = cluster.getGroupId(); + final AtomicInteger failCount = new AtomicInteger(0); + try { + CodeInjectionForTesting.put(GrpcServicesImpl.GRPC_SEND_SERVER_REQUEST, (localId, remoteId, args) -> { + if (leaderId.equals(localId) + && args.length > 0 && args[0] instanceof RaftProtos.AppendEntriesRequestProto) { + final RaftProtos.AppendEntriesRequestProto proto = (RaftProtos.AppendEntriesRequestProto) args[0]; + if (RaftGroupId.valueOf(proto.getServerRequest().getRaftGroupId().getId()).equals(groupId) + && failCount.getAndIncrement() < 1) { + throw new IllegalStateException("Injected failure for restart test"); + } + } + return false; + }); + + JavaUtils.attempt(() -> { + final Set current = RaftServerTestUtil.getLogAppenders(leader) + .collect(Collectors.toSet()); + Assertions.assertEquals(2, current.size()); + Assertions.assertTrue(current.stream().anyMatch(a -> !before.contains(a)), + "Expected at least one new LogAppender instance after daemon exception restart"); + }, 30, ONE_SECOND, "LogAppender auto-restart after exception", LOG); + } finally { + CodeInjectionForTesting.remove(GrpcServicesImpl.GRPC_SEND_SERVER_REQUEST); + } + + try (RaftClient client = cluster.createClient(leaderId)) { + for (int i = 0; i < 5; i++) { + Assertions.assertTrue( + client.io().send(new RaftTestUtil.SimpleMessage("after-restart-" + i)).isSuccess()); + } + } + } } From 93314c3e78720efcfb033f197ce3b2c4d0a53e94 Mon Sep 17 00:00:00 2001 From: Abhishek Pal Date: Thu, 9 Apr 2026 23:27:44 +0530 Subject: [PATCH 316/397] RATIS-2433. Cancel transaction in case of failure to append (#1382) --- .../ratis/server/impl/RaftServerImpl.java | 89 +++++++++++++------ .../ratis/server/impl/RetryCacheImpl.java | 10 --- .../impl/TransactionContextImpl.java | 2 - .../impl/RaftStateMachineExceptionTests.java | 35 ++++++++ 4 files changed, 98 insertions(+), 38 deletions(-) diff --git a/ratis-server/src/main/java/org/apache/ratis/server/impl/RaftServerImpl.java b/ratis-server/src/main/java/org/apache/ratis/server/impl/RaftServerImpl.java index d4c6f164e3..a9c80d000b 100644 --- a/ratis-server/src/main/java/org/apache/ratis/server/impl/RaftServerImpl.java +++ b/ratis-server/src/main/java/org/apache/ratis/server/impl/RaftServerImpl.java @@ -768,17 +768,18 @@ private CompletableFuture checkLeaderState(RaftClientRequest re } catch (GroupMismatchException e) { return JavaUtils.completeExceptionally(e); } - return checkLeaderState(request, null); + return checkLeaderState(request, null, null); } /** * @return null if the server is in leader state. */ - private CompletableFuture checkLeaderState(RaftClientRequest request, CacheEntry entry) { + private CompletableFuture checkLeaderState( + RaftClientRequest request, CacheEntry entry, TransactionContextImpl context) { if (!getInfo().isLeader()) { NotLeaderException exception = generateNotLeaderException(); final RaftClientReply reply = newExceptionReply(request, exception); - return RetryCacheImpl.failWithReply(reply, entry); + return failWithReply(reply, entry, context); } if (!getInfo().isLeaderReady()) { final CacheEntry cacheEntry = retryCache.getIfPresent(ClientInvocationId.valueOf(request)); @@ -787,13 +788,13 @@ private CompletableFuture checkLeaderState(RaftClientRequest re } final LeaderNotReadyException lnre = new LeaderNotReadyException(getMemberId()); final RaftClientReply reply = newExceptionReply(request, lnre); - return RetryCacheImpl.failWithReply(reply, entry); + return failWithReply(reply, entry, context); } if (!request.isReadOnly() && isSteppingDown()) { final LeaderSteppingDownException lsde = new LeaderSteppingDownException(getMemberId() + " is stepping down"); final RaftClientReply reply = newExceptionReply(request, lsde); - return RetryCacheImpl.failWithReply(reply, entry); + return failWithReply(reply, entry, context); } return null; @@ -819,11 +820,44 @@ void assertLifeCycleState(Set expected) throws ServerNotReadyEx getMemberId() + " is not in " + expected + ": current state is " + c), expected); } - private CompletableFuture getResourceUnavailableReply(RaftClientRequest request, CacheEntry entry) { - return entry.failWithException(new ResourceUnavailableException( - getMemberId() + ": Failed to acquire a pending write request for " + request)); + private CompletableFuture getResourceUnavailableReply(String op, + RaftClientRequest request, CacheEntry entry, TransactionContextImpl context) { + final ResourceUnavailableException e = new ResourceUnavailableException(getMemberId() + + ": Failed to " + op + " for " + request); + cancelTransaction(context, e); + return entry.failWithException(e); + } + + private CompletableFuture failWithReply( + RaftClientReply reply, CacheEntry entry, TransactionContextImpl context) { + if (context != null) { + cancelTransaction(context, reply.getException()); + } + + if (entry == null) { + return CompletableFuture.completedFuture(reply); + } + entry.failWithReply(reply); + return entry.getReplyFuture(); + } + + /** Cancel a transaction and notify the state machine. Set exception if provided to the transaction context. */ + private void cancelTransaction(TransactionContextImpl context, Exception exception) { + if (context == null) { + return; } + if (exception != null) { + context.setException(exception); + } + + try { + context.cancelTransaction(); + } catch (IOException ioe) { + LOG.warn("{}: Failed to cancel transaction {}", getMemberId(), context, ioe); + } + } + /** * Handle a normal update request from client. */ @@ -836,27 +870,28 @@ private CompletableFuture appendTransaction( final LeaderStateImpl unsyncedLeaderState = role.getLeaderState().orElse(null); if (unsyncedLeaderState == null) { - final RaftClientReply reply = newExceptionReply(request, generateNotLeaderException()); - return RetryCacheImpl.failWithReply(reply, cacheEntry); + final NotLeaderException nle = generateNotLeaderException(); + final RaftClientReply reply = newExceptionReply(request, nle); + return failWithReply(reply, cacheEntry, context); } final PendingRequests.Permit unsyncedPermit = unsyncedLeaderState.tryAcquirePendingRequest(request.getMessage()); if (unsyncedPermit == null) { - return getResourceUnavailableReply(request, cacheEntry); + return getResourceUnavailableReply("acquire a pending write request", request, cacheEntry, context); } final LeaderStateImpl leaderState; final PendingRequest pending; synchronized (this) { - final CompletableFuture reply = checkLeaderState(request, cacheEntry); + final CompletableFuture reply = checkLeaderState(request, cacheEntry, context); if (reply != null) { return reply; } leaderState = role.getLeaderStateNonNull(); - final PendingRequests.Permit permit = leaderState == unsyncedLeaderState? unsyncedPermit + final PendingRequests.Permit permit = leaderState == unsyncedLeaderState ? unsyncedPermit : leaderState.tryAcquirePendingRequest(request.getMessage()); if (permit == null) { - return getResourceUnavailableReply(request, cacheEntry); + return getResourceUnavailableReply("acquire a pending write request", request, cacheEntry, context); } // append the message to its local log @@ -866,20 +901,18 @@ private CompletableFuture appendTransaction( } catch (StateMachineException e) { // the StateMachineException is thrown by the SM in the preAppend stage. // Return the exception in a RaftClientReply. - RaftClientReply exceptionReply = newExceptionReply(request, e); - cacheEntry.failWithReply(exceptionReply); + final RaftClientReply exceptionReply = newExceptionReply(request, e); // leader will step down here if (e.leaderShouldStepDown() && getInfo().isLeader()) { leaderState.submitStepDownEvent(StepDownReason.STATE_MACHINE_EXCEPTION); } - return CompletableFuture.completedFuture(exceptionReply); + return failWithReply(exceptionReply, cacheEntry, null); } // put the request into the pending queue pending = leaderState.addPendingRequest(permit, request, context); if (pending == null) { - return cacheEntry.failWithException(new ResourceUnavailableException( - getMemberId() + ": Failed to add a pending write request for " + request)); + return getResourceUnavailableReply("add a pending write request", request, cacheEntry, context); } } leaderState.notifySenders(); @@ -1011,19 +1044,23 @@ private CompletableFuture writeAsyncImpl(RaftClientRequest requ // return the cached future. return cacheEntry.getReplyFuture(); } - // TODO: this client request will not be added to pending requests until - // later which means that any failure in between will leave partial state in - // the state machine. We should call cancelTransaction() for failed requests + // This request will be added to pending requests later in appendTransaction. + // Any failure in between must invoke cancelTransaction. final TransactionContextImpl context = (TransactionContextImpl) stateMachine.startTransaction( filterDataStreamRaftClientRequest(request)); if (context.getException() != null) { - final StateMachineException e = new StateMachineException(getMemberId(), context.getException()); + final Exception exception = context.getException(); + final StateMachineException e = new StateMachineException(getMemberId(), exception); final RaftClientReply exceptionReply = newExceptionReply(request, e); - cacheEntry.failWithReply(exceptionReply); - return CompletableFuture.completedFuture(exceptionReply); + return failWithReply(exceptionReply, cacheEntry, context); } - return appendTransaction(request, context, cacheEntry); + try { + return appendTransaction(request, context, cacheEntry); + } catch (Exception e) { + cancelTransaction(context, e); + throw e; + } } private CompletableFuture watchAsync(RaftClientRequest request) { diff --git a/ratis-server/src/main/java/org/apache/ratis/server/impl/RetryCacheImpl.java b/ratis-server/src/main/java/org/apache/ratis/server/impl/RetryCacheImpl.java index 4da459ae9e..96ad62a531 100644 --- a/ratis-server/src/main/java/org/apache/ratis/server/impl/RetryCacheImpl.java +++ b/ratis-server/src/main/java/org/apache/ratis/server/impl/RetryCacheImpl.java @@ -257,14 +257,4 @@ public synchronized void close() { cache.invalidateAll(); statistics.set(null); } - - static CompletableFuture failWithReply( - RaftClientReply reply, CacheEntry entry) { - if (entry != null) { - entry.failWithReply(reply); - return entry.getReplyFuture(); - } else { - return CompletableFuture.completedFuture(reply); - } - } } diff --git a/ratis-server/src/main/java/org/apache/ratis/statemachine/impl/TransactionContextImpl.java b/ratis-server/src/main/java/org/apache/ratis/statemachine/impl/TransactionContextImpl.java index d92f3a1c82..8497b12f4d 100644 --- a/ratis-server/src/main/java/org/apache/ratis/statemachine/impl/TransactionContextImpl.java +++ b/ratis-server/src/main/java/org/apache/ratis/statemachine/impl/TransactionContextImpl.java @@ -191,8 +191,6 @@ public TransactionContext preAppendTransaction() throws IOException { @Override public TransactionContext cancelTransaction() throws IOException { - // TODO: This is not called from Raft server / log yet. When an IOException happens, we should - // call this to let the SM know that Transaction cannot be synced return stateMachine.cancelTransaction(this); } } diff --git a/ratis-server/src/test/java/org/apache/ratis/server/impl/RaftStateMachineExceptionTests.java b/ratis-server/src/test/java/org/apache/ratis/server/impl/RaftStateMachineExceptionTests.java index 3a58f4e7c6..1e46907d10 100644 --- a/ratis-server/src/test/java/org/apache/ratis/server/impl/RaftStateMachineExceptionTests.java +++ b/ratis-server/src/test/java/org/apache/ratis/server/impl/RaftStateMachineExceptionTests.java @@ -40,6 +40,7 @@ import java.io.IOException; import java.util.Objects; import java.util.concurrent.CompletableFuture; +import java.util.concurrent.atomic.AtomicInteger; import static org.junit.jupiter.api.Assertions.fail; @@ -52,6 +53,7 @@ public abstract class RaftStateMachineExceptionTests { + Assertions.assertEquals(0, numCancelTransaction.get(), + () -> "Expected cancelTransaction() not to be called but got " + numCancelTransaction.get()); + return null; + }, 10, ONE_SECOND, "wait for cancelTransaction", LOG); + } finally { + failPreAppend = false; + } + } } From 6b7d16bf0fa72f006c5bbfb37e54e432c825465c Mon Sep 17 00:00:00 2001 From: "dependabot[bot]" <49699333+dependabot[bot]@users.noreply.github.com> Date: Sun, 12 Apr 2026 17:07:49 +0200 Subject: [PATCH 317/397] RATIS-2483. Bump dependency-check-maven to 12.2.0 (#1414) --- pom.xml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/pom.xml b/pom.xml index 97d5841279..4e70dd29c4 100644 --- a/pom.xml +++ b/pom.xml @@ -749,7 +749,7 @@ org.owasp dependency-check-maven - 12.1.9 + 12.2.0 From 6eb952e69acb826cb59aa3cb48bfdbe483f34c7e Mon Sep 17 00:00:00 2001 From: "dependabot[bot]" <49699333+dependabot[bot]@users.noreply.github.com> Date: Sun, 12 Apr 2026 21:50:03 +0200 Subject: [PATCH 318/397] RATIS-2453. Bump opentelemetry to 1.60.1 (#1387) --- pom.xml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/pom.xml b/pom.xml index 4e70dd29c4..4ffed3fdc2 100644 --- a/pom.xml +++ b/pom.xml @@ -182,7 +182,7 @@ 1.77.1 - 1.59.0 + 1.60.1 1.40.0 true From 0ac3277b577fa3e43091c5b009e3d7ada4a31cd3 Mon Sep 17 00:00:00 2001 From: "Doroszlai, Attila" <6454655+adoroszlai@users.noreply.github.com> Date: Wed, 15 Apr 2026 11:57:13 +0200 Subject: [PATCH 319/397] RATIS-2493. Check actions with zizmor (#1426) --- .github/dependabot.yml | 4 +-- .github/workflows/check.yaml | 29 ++++++++++------- .github/workflows/ci.yaml | 38 +++++++++++++++------- .github/workflows/close-stale-pr.yaml | 2 +- .github/workflows/post-commit.yaml | 6 +++- .github/workflows/repeat-test.yaml | 14 +++++--- .github/workflows/vulnerability-check.yaml | 15 ++++++--- .github/workflows/zizmor.yml | 36 ++++++++++++++++++++ 8 files changed, 108 insertions(+), 36 deletions(-) create mode 100644 .github/workflows/zizmor.yml diff --git a/.github/dependabot.yml b/.github/dependabot.yml index d343c5a93d..4ada4d9821 100644 --- a/.github/dependabot.yml +++ b/.github/dependabot.yml @@ -23,7 +23,7 @@ updates: interval: "cron" cronjob: "15 9 * * *" cooldown: - default-days: 4 + default-days: 7 - package-ecosystem: "maven" directory: "/" schedule: @@ -31,7 +31,7 @@ updates: interval: "cron" cronjob: "15 10 * * *" cooldown: - default-days: 4 + default-days: 7 ignore: # requires Java 11 - dependency-name: "com.github.spotbugs:spotbugs" diff --git a/.github/workflows/check.yaml b/.github/workflows/check.yaml index cb3d31d1fe..7d38342068 100644 --- a/.github/workflows/check.yaml +++ b/.github/workflows/check.yaml @@ -76,6 +76,11 @@ on: default: 30 required: false + secrets: + DEVELOCITY_ACCESS_KEY: + description: 'Token for submitting build scan to Develocity' + required: false + env: MAVEN_ARGS: --batch-mode --show-version MAVEN_OPTS: -Dhttp.keepAlive=false -Dmaven.wagon.http.pool=false -Dmaven.wagon.http.retryHandler.class=standard -Dmaven.wagon.http.retryHandler.count=3 @@ -90,11 +95,13 @@ jobs: steps: - name: Checkout project if: ${{ !inputs.needs-source-tarball }} - uses: actions/checkout@v6 + uses: actions/checkout@de0fac2e4500dabe0009e67214ff5f5447ce83dd # v6.0.2 + with: + persist-credentials: false - name: Download source tarball if: ${{ inputs.needs-source-tarball }} - uses: actions/download-artifact@v8 + uses: actions/download-artifact@3e5f45b2cfb9172054b4087a40e8e0b5a5461e7c # v8.0.1 with: name: ratis-src @@ -105,7 +112,7 @@ jobs: - name: Create cache for Maven dependencies if: ${{ inputs.script == 'build' }} - uses: actions/cache@v5 + uses: actions/cache@668228422ae6a00e4ad889ee87cd7109ec5666a7 # v5.0.4 with: path: | ~/.m2/repository/*/*/* @@ -116,7 +123,7 @@ jobs: - name: Restore cache for Maven dependencies if: ${{ inputs.script != 'build' }} - uses: actions/cache/restore@v5 + uses: actions/cache/restore@668228422ae6a00e4ad889ee87cd7109ec5666a7 # v5.0.4 with: path: | ~/.m2/repository/*/*/* @@ -128,7 +135,7 @@ jobs: - name: Download Maven repo id: download-maven-repo if: ${{ inputs.needs-maven-repo }} - uses: actions/download-artifact@v8 + uses: actions/download-artifact@3e5f45b2cfb9172054b4087a40e8e0b5a5461e7c # v8.0.1 with: name: maven-repo path: | @@ -136,7 +143,7 @@ jobs: - name: Download binary tarball if: ${{ inputs.needs-binary-tarball }} - uses: actions/download-artifact@v8 + uses: actions/download-artifact@3e5f45b2cfb9172054b4087a40e8e0b5a5461e7c # v8.0.1 with: name: ratis-bin @@ -148,7 +155,7 @@ jobs: - name: Setup java ${{ inputs.java-version }} if: ${{ inputs.java-version }} - uses: actions/setup-java@v5 + uses: actions/setup-java@be666c2fcd27ec809703dec50e508c2fdc7f6654 # v5.2.0 with: distribution: 'temurin' java-version: ${{ inputs.java-version }} @@ -169,7 +176,7 @@ jobs: - name: Archive build results if: ${{ !cancelled() }} - uses: actions/upload-artifact@v7 + uses: actions/upload-artifact@bbbca2ddaa5d8feaa63e36b76fdaad77386f024f # v7.0.0 with: name: ${{ (inputs.split && format('{0}-{1}', inputs.script, inputs.split)) || inputs.script }} path: target/${{ inputs.script }} @@ -179,7 +186,7 @@ jobs: # to avoid the need for 3 more inputs. - name: Store binaries for tests if: ${{ inputs.script == 'build' && !cancelled() }} - uses: actions/upload-artifact@v7 + uses: actions/upload-artifact@bbbca2ddaa5d8feaa63e36b76fdaad77386f024f # v7.0.0 with: name: ratis-bin path: | @@ -188,7 +195,7 @@ jobs: - name: Store source tarball for compilation if: ${{ inputs.script == 'build' && !cancelled() }} - uses: actions/upload-artifact@v7 + uses: actions/upload-artifact@bbbca2ddaa5d8feaa63e36b76fdaad77386f024f # v7.0.0 with: name: ratis-src path: | @@ -197,7 +204,7 @@ jobs: - name: Store Maven repo for tests if: ${{ inputs.script == 'build' && !cancelled() }} - uses: actions/upload-artifact@v7 + uses: actions/upload-artifact@bbbca2ddaa5d8feaa63e36b76fdaad77386f024f # v7.0.0 with: name: maven-repo path: | diff --git a/.github/workflows/ci.yaml b/.github/workflows/ci.yaml index 272cb33d90..d8f9f8365d 100644 --- a/.github/workflows/ci.yaml +++ b/.github/workflows/ci.yaml @@ -23,6 +23,15 @@ on: description: Ratis git ref (branch, tag or commit hash) default: '' required: false + secrets: + DEVELOCITY_ACCESS_KEY: + description: 'Token for submitting build scan to Develocity' + required: false + SONARCLOUD_TOKEN: + description: 'Token for submitting coverage data to SonarCloud' + required: false + +permissions: { } jobs: build: @@ -31,7 +40,8 @@ jobs: script: build script-args: -Prelease timeout-minutes: 30 - secrets: inherit + secrets: + DEVELOCITY_ACCESS_KEY: ${{ secrets.DEVELOCITY_ACCESS_KEY }} compile: needs: @@ -48,14 +58,16 @@ jobs: script-args: -Dmaven.compiler.release=${{ matrix.java }} split: ${{ matrix.java }} timeout-minutes: 30 - secrets: inherit + secrets: + DEVELOCITY_ACCESS_KEY: ${{ secrets.DEVELOCITY_ACCESS_KEY }} release: uses: ./.github/workflows/check.yaml with: script: release timeout-minutes: 30 - secrets: inherit + secrets: + DEVELOCITY_ACCESS_KEY: ${{ secrets.DEVELOCITY_ACCESS_KEY }} repro: needs: @@ -66,7 +78,8 @@ jobs: script: repro script-args: -Prelease timeout-minutes: 30 - secrets: inherit + secrets: + DEVELOCITY_ACCESS_KEY: ${{ secrets.DEVELOCITY_ACCESS_KEY }} basic: strategy: @@ -81,7 +94,8 @@ jobs: with: script: ${{ matrix.check }} timeout-minutes: 30 - secrets: inherit + secrets: + DEVELOCITY_ACCESS_KEY: ${{ secrets.DEVELOCITY_ACCESS_KEY }} unit: strategy: @@ -98,7 +112,8 @@ jobs: script-args: -P${{ matrix.profile }}-tests split: ${{ matrix.profile }} timeout-minutes: 60 - secrets: inherit + secrets: + DEVELOCITY_ACCESS_KEY: ${{ secrets.DEVELOCITY_ACCESS_KEY }} coverage: needs: @@ -109,11 +124,12 @@ jobs: if: github.event_name != 'pull_request' steps: - name: Checkout project - uses: actions/checkout@v6 + uses: actions/checkout@de0fac2e4500dabe0009e67214ff5f5447ce83dd # v6.0.2 with: fetch-depth: 0 + persist-credentials: false - name: Cache for maven dependencies - uses: actions/cache/restore@v5 + uses: actions/cache/restore@668228422ae6a00e4ad889ee87cd7109ec5666a7 # v5.0.4 with: path: | ~/.m2/repository @@ -122,12 +138,12 @@ jobs: restore-keys: | maven-repo- - name: Setup java 17 - uses: actions/setup-java@v5 + uses: actions/setup-java@be666c2fcd27ec809703dec50e508c2fdc7f6654 # v5.2.0 with: distribution: 'temurin' java-version: 17 - name: Download artifacts - uses: actions/download-artifact@v8 + uses: actions/download-artifact@3e5f45b2cfb9172054b4087a40e8e0b5a5461e7c # v8.0.1 with: path: target/artifacts - name: Untar binaries @@ -143,7 +159,7 @@ jobs: SONAR_TOKEN: ${{ secrets.SONARCLOUD_TOKEN }} GITHUB_TOKEN: ${{ secrets.GITHUB_TOKEN }} - name: Archive build results - uses: actions/upload-artifact@v7 + uses: actions/upload-artifact@bbbca2ddaa5d8feaa63e36b76fdaad77386f024f # v7.0.0 if: always() with: name: ${{ github.job }} diff --git a/.github/workflows/close-stale-pr.yaml b/.github/workflows/close-stale-pr.yaml index 6c24bf0e51..010f0c955d 100644 --- a/.github/workflows/close-stale-pr.yaml +++ b/.github/workflows/close-stale-pr.yaml @@ -26,7 +26,7 @@ jobs: runs-on: ubuntu-slim steps: - name: Close Stale PRs - uses: actions/stale@v10 + uses: actions/stale@b5d41d4e1d5dceea10e7104786b73624c18a190f # v10.2.0 with: stale-pr-label: 'stale' exempt-draft-pr: false diff --git a/.github/workflows/post-commit.yaml b/.github/workflows/post-commit.yaml index 1d1ba06fd9..4a946f8621 100644 --- a/.github/workflows/post-commit.yaml +++ b/.github/workflows/post-commit.yaml @@ -27,10 +27,14 @@ concurrency: group: ci-${{ github.event.pull_request.number || case(github.repository == 'apache/ratis', github.sha, github.ref_name) }} cancel-in-progress: ${{ github.event_name == 'pull_request' || github.repository != 'apache/ratis' }} +permissions: { } + jobs: CI: if: github.event_name == 'pull_request' || github.repository == 'apache/ratis' || github.ref_name != 'master' uses: ./.github/workflows/ci.yaml - secrets: inherit + secrets: + DEVELOCITY_ACCESS_KEY: ${{ secrets.DEVELOCITY_ACCESS_KEY }} + SONARCLOUD_TOKEN: ${{ secrets.SONARCLOUD_TOKEN }} diff --git a/.github/workflows/repeat-test.yaml b/.github/workflows/repeat-test.yaml index 9a985fd23c..26fef26f75 100644 --- a/.github/workflows/repeat-test.yaml +++ b/.github/workflows/repeat-test.yaml @@ -49,6 +49,9 @@ env: FAIL_FAST: ${{ github.event.inputs.fail-fast }} SPLITS: ${{ github.event.inputs.splits }} run-name: ${{ github.event_name == 'workflow_dispatch' && format('{0}#{1}[{2}]-{3}x{4}', inputs.test-class, inputs.test-method, inputs.ref, inputs.splits, inputs.iterations) || '' }} + +permissions: { } + jobs: prepare: runs-on: ubuntu-24.04 @@ -95,11 +98,12 @@ jobs: split: ${{ fromJson(needs.prepare.outputs.matrix) }} fail-fast: ${{ fromJson(github.event.inputs.fail-fast) }} steps: - - uses: actions/checkout@v6 + - uses: actions/checkout@de0fac2e4500dabe0009e67214ff5f5447ce83dd # v6.0.2 with: + persist-credentials: false ref: ${{ needs.prepare.outputs.ref }} - name: Cache for maven dependencies - uses: actions/cache@v5 + uses: actions/cache@668228422ae6a00e4ad889ee87cd7109ec5666a7 # v5.0.4 with: path: | ~/.m2/repository @@ -108,7 +112,7 @@ jobs: restore-keys: | maven-repo- - name: Setup java - uses: actions/setup-java@v5 + uses: actions/setup-java@be666c2fcd27ec809703dec50e508c2fdc7f6654 # v5.2.0 with: distribution: 'temurin' java-version: 8 @@ -121,7 +125,7 @@ jobs: run: dev-support/checks/_summary.sh target/unit/summary.txt if: ${{ !cancelled() }} - name: Archive build results - uses: actions/upload-artifact@v7 + uses: actions/upload-artifact@bbbca2ddaa5d8feaa63e36b76fdaad77386f024f # v7.0.0 if: ${{ failure() }} with: name: result-${{ github.run_number }}-${{ github.run_id }}-split-${{ matrix.split }} @@ -132,7 +136,7 @@ jobs: runs-on: ubuntu-24.04 steps: - name: Download build results - uses: actions/download-artifact@v8 + uses: actions/download-artifact@3e5f45b2cfb9172054b4087a40e8e0b5a5461e7c # v8.0.1 - name: Count failures run: | failures=$(find . -name 'summary.txt' | grep -v 'iteration' | xargs grep -v 'exit code: 0' | wc -l) diff --git a/.github/workflows/vulnerability-check.yaml b/.github/workflows/vulnerability-check.yaml index a0146f14de..336332a845 100644 --- a/.github/workflows/vulnerability-check.yaml +++ b/.github/workflows/vulnerability-check.yaml @@ -29,15 +29,19 @@ env: MAVEN_OPTS: -Dhttp.keepAlive=false -Dmaven.wagon.http.pool=false -Dmaven.wagon.http.retryHandler.class=standard -Dmaven.wagon.http.retryHandler.count=3 MAVEN_ARGS: --batch-mode --no-transfer-progress +permissions: { } + jobs: dependency-check: if: ${{ github.event_name == 'workflow_dispatch' || github.repository == 'apache/ratis' }} runs-on: ubuntu-latest steps: - - uses: actions/checkout@v6 + - uses: actions/checkout@de0fac2e4500dabe0009e67214ff5f5447ce83dd # v6.0.2 + with: + persist-credentials: false - name: Set up JDK 11 - uses: actions/setup-java@v5 + uses: actions/setup-java@be666c2fcd27ec809703dec50e508c2fdc7f6654 # v5.2.0 with: distribution: corretto java-version: 11 @@ -52,13 +56,14 @@ jobs: - name: Generate report date for artifact name run: | - utc_time="${{ github.run_started_at }}" target_time=$(TZ=Asia/Shanghai date -d "$utc_time" +"%Y-%m-%d") echo "REPORT_DATE=$target_time" >> $GITHUB_ENV + env: + utc_time: ${{ github.run_started_at }} - name: Upload Artifact - uses: actions/upload-artifact@v7 + uses: actions/upload-artifact@bbbca2ddaa5d8feaa63e36b76fdaad77386f024f # v7.0.0 with: name: vulnerability-check-result-${{ env.REPORT_DATE }} path: target/dependency-check-report.html - retention-days: 15 \ No newline at end of file + retention-days: 15 diff --git a/.github/workflows/zizmor.yml b/.github/workflows/zizmor.yml new file mode 100644 index 0000000000..6b7263f91d --- /dev/null +++ b/.github/workflows/zizmor.yml @@ -0,0 +1,36 @@ +# 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. + +name: zizmor + +on: + push: + pull_request: + +permissions: { } + +jobs: + zizmor: + runs-on: ubuntu-latest + permissions: + security-events: write + steps: + - name: Checkout project + uses: actions/checkout@de0fac2e4500dabe0009e67214ff5f5447ce83dd # v6.0.2 + with: + persist-credentials: false + + - name: Run zizmor + uses: zizmorcore/zizmor-action@b1d7e1fb5de872772f31590499237e7cce841e8e # v0.5.3 From f3da8f522102de3f7d6f5bbdcbd984e33182df2b Mon Sep 17 00:00:00 2001 From: Tsz-Wo Nicholas Sze Date: Thu, 23 Apr 2026 06:04:06 -0700 Subject: [PATCH 320/397] RATIS-2497. Pass server to the dummy watch request in OrderedAsync (#1438) --- .../apache/ratis/client/impl/AsyncImpl.java | 2 +- .../ratis/client/impl/OrderedAsync.java | 6 +- .../ratis/client/impl/RaftClientImpl.java | 9 ++- .../org/apache/ratis/protocol/Message.java | 5 +- .../org/apache/ratis/util/MemoizedBase.java | 73 +++++++++++++++++++ .../ratis/util/MemoizedCheckedSupplier.java | 31 ++------ .../apache/ratis/util/MemoizedFunction.java | 69 ++++++++++++++++++ .../apache/ratis/util/MemoizedSupplier.java | 32 ++------ .../org/apache/ratis/util/StringUtils.java | 17 +++++ .../ratis/server/impl/RaftServerImpl.java | 7 +- .../apache/ratis/LinearizableReadTests.java | 16 ++-- .../ratis/client/impl/RaftClientTestUtil.java | 3 +- .../ratis/server/impl/MiniRaftCluster.java | 4 + 13 files changed, 204 insertions(+), 70 deletions(-) create mode 100644 ratis-common/src/main/java/org/apache/ratis/util/MemoizedBase.java create mode 100644 ratis-common/src/main/java/org/apache/ratis/util/MemoizedFunction.java diff --git a/ratis-client/src/main/java/org/apache/ratis/client/impl/AsyncImpl.java b/ratis-client/src/main/java/org/apache/ratis/client/impl/AsyncImpl.java index 973b0db0cf..01329fa7d6 100644 --- a/ratis-client/src/main/java/org/apache/ratis/client/impl/AsyncImpl.java +++ b/ratis-client/src/main/java/org/apache/ratis/client/impl/AsyncImpl.java @@ -40,7 +40,7 @@ class AsyncImpl implements AsyncRpcApi { CompletableFuture send( RaftClientRequest.Type type, Message message, RaftPeerId server) { return TraceClient.asyncSend( - () -> client.getOrderedAsync().send(type, message, server), type, server); + () -> client.getOrderedAsync(server).send(type, message, server), type, server); } @Override diff --git a/ratis-client/src/main/java/org/apache/ratis/client/impl/OrderedAsync.java b/ratis-client/src/main/java/org/apache/ratis/client/impl/OrderedAsync.java index ecf4db3dce..791adc8982 100644 --- a/ratis-client/src/main/java/org/apache/ratis/client/impl/OrderedAsync.java +++ b/ratis-client/src/main/java/org/apache/ratis/client/impl/OrderedAsync.java @@ -58,6 +58,8 @@ public final class OrderedAsync { public static final Logger LOG = LoggerFactory.getLogger(OrderedAsync.class); + public static final Message DUMMY = Message.valueOf("DUMMY"); + private enum BatchLogKey implements BatchLogger.Key { SEND_REQUEST_EXCEPTION } @@ -116,12 +118,12 @@ public String toString() { } } - static OrderedAsync newInstance(RaftClientImpl client, RaftProperties properties) { + static OrderedAsync newInstance(RaftClientImpl client, RaftPeerId server, RaftProperties properties) { final OrderedAsync ordered = new OrderedAsync(client, properties); // send a dummy watch request to establish the connection // TODO: this is a work around, it is better to fix the underlying RPC implementation if (RaftClientConfigKeys.Async.Experimental.sendDummyRequest(properties)) { - ordered.send(RaftClientRequest.watchRequestType(), null, null); + ordered.send(RaftClientRequest.watchRequestType(), DUMMY, server); } return ordered; } diff --git a/ratis-client/src/main/java/org/apache/ratis/client/impl/RaftClientImpl.java b/ratis-client/src/main/java/org/apache/ratis/client/impl/RaftClientImpl.java index 27ae2e6bab..f24360f62b 100644 --- a/ratis-client/src/main/java/org/apache/ratis/client/impl/RaftClientImpl.java +++ b/ratis-client/src/main/java/org/apache/ratis/client/impl/RaftClientImpl.java @@ -47,6 +47,7 @@ import org.apache.ratis.util.CollectionUtils; import org.apache.ratis.util.IOUtils; import org.apache.ratis.util.JavaUtils; +import org.apache.ratis.util.MemoizedFunction; import org.apache.ratis.util.MemoizedSupplier; import org.apache.ratis.util.Preconditions; import org.apache.ratis.util.TimeDuration; @@ -180,7 +181,7 @@ private synchronized Set getAndReset() { private final TimeoutExecutor scheduler = TimeoutExecutor.getInstance(); - private final Supplier orderedAsync; + private final MemoizedFunction orderedAsync; private final Supplier asyncApi; private final Supplier blockingApi; private final Supplier messageStreamApi; @@ -209,7 +210,7 @@ private synchronized Set getAndReset() { clientRpc.addRaftPeers(group.getPeers()); this.clientRpc = clientRpc; - this.orderedAsync = JavaUtils.memoize(() -> OrderedAsync.newInstance(this, properties)); + this.orderedAsync = MemoizedFunction.valueOf(server -> OrderedAsync.newInstance(this, server, properties)); this.messageStreamApi = JavaUtils.memoize(() -> MessageStreamImpl.newInstance(this, properties)); this.asyncApi = JavaUtils.memoize(() -> new AsyncImpl(this)); this.blockingApi = JavaUtils.memoize(() -> new BlockingImpl(this)); @@ -277,8 +278,8 @@ TimeoutExecutor getScheduler() { return scheduler; } - OrderedAsync getOrderedAsync() { - return orderedAsync.get(); + OrderedAsync getOrderedAsync(RaftPeerId server) { + return orderedAsync.apply(server); } RaftClientRequest newRaftClientRequest( diff --git a/ratis-common/src/main/java/org/apache/ratis/protocol/Message.java b/ratis-common/src/main/java/org/apache/ratis/protocol/Message.java index e7ea97ca4e..55fcd064d2 100644 --- a/ratis-common/src/main/java/org/apache/ratis/protocol/Message.java +++ b/ratis-common/src/main/java/org/apache/ratis/protocol/Message.java @@ -19,6 +19,7 @@ import org.apache.ratis.thirdparty.com.google.protobuf.AbstractMessage; import org.apache.ratis.thirdparty.com.google.protobuf.ByteString; +import org.apache.ratis.thirdparty.com.google.protobuf.TextFormat; import org.apache.ratis.util.MemoizedSupplier; import org.apache.ratis.util.StringUtils; @@ -47,11 +48,11 @@ public String toString() { } static Message valueOf(AbstractMessage abstractMessage) { - return valueOf(abstractMessage.toByteString(), abstractMessage::toString); + return valueOf(abstractMessage.toByteString(), () -> TextFormat.shortDebugString(abstractMessage)); } static Message valueOf(ByteString bytes) { - return valueOf(bytes, () -> "Message:" + StringUtils.bytes2HexShortString(bytes)); + return valueOf(bytes, () -> "Message:" + StringUtils.bytes2ShortString(bytes)); } static Message valueOf(String string) { diff --git a/ratis-common/src/main/java/org/apache/ratis/util/MemoizedBase.java b/ratis-common/src/main/java/org/apache/ratis/util/MemoizedBase.java new file mode 100644 index 0000000000..1c78c05cfc --- /dev/null +++ b/ratis-common/src/main/java/org/apache/ratis/util/MemoizedBase.java @@ -0,0 +1,73 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.ratis.util; + +import org.apache.ratis.util.function.CheckedSupplier; + +import java.util.Objects; + +/** + * This is the base class for the memoized subclass such as + * {@link MemoizedSupplier}, {@link MemoizedFunction}, {@link MemoizedCheckedSupplier}, etc, + * The subclasses provide its own method to retrieve the value, + * such as {@link MemoizedSupplier#get()} and {@link MemoizedFunction#apply(Object)}. + * The subclass method returns a value by invoking its initializer once at the first call + * and then keeps returning the same value for the subsequent calls. + *

+ * All the subclasses are thread safe. + * + * @param The value type. + * @param The throwable type of the initializer. + */ +abstract class MemoizedBase { + @SuppressWarnings({"squid:S3077"}) // Suppress volatile for generic type + private volatile RETURN value = null; + + final RETURN init(CheckedSupplier initializer) throws THROW { + final RETURN initialized = value; + if (initialized != null) { + return initialized; + } + + synchronized (this) { + if (value == null) { + value = initializer.get(); + Objects.requireNonNull(value, "initializer.get() returns null"); + } + return value; + } + } + + /** @return is the object initialized? */ + public final boolean isInitialized() { + return value != null; + } + + /** + * @return the value, which must be already initialized. + * @throws NullPointerException if the value is uninitialized. + */ + public RETURN getInitializedValue() { + return Objects.requireNonNull(value, "Uninitialized: value == null"); + } + + @Override + public String toString() { + return value != null ? "Memoized:" + value : "Uninitialized"; + } +} diff --git a/ratis-common/src/main/java/org/apache/ratis/util/MemoizedCheckedSupplier.java b/ratis-common/src/main/java/org/apache/ratis/util/MemoizedCheckedSupplier.java index 0e9ae44fa5..8d4cf9cb83 100644 --- a/ratis-common/src/main/java/org/apache/ratis/util/MemoizedCheckedSupplier.java +++ b/ratis-common/src/main/java/org/apache/ratis/util/MemoizedCheckedSupplier.java @@ -1,4 +1,4 @@ -/** +/* * 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 @@ -25,13 +25,14 @@ * A memoized supplier is a {@link CheckedSupplier} * which gets a value by invoking its initializer once. * and then keeps returning the same value as its supplied results. - * + *

* This class is thread safe. * * @param The return type of the supplier. * @param The throwable type of the supplier. */ public final class MemoizedCheckedSupplier + extends MemoizedBase implements CheckedSupplier { /** * @param supplier to supply at most one non-null value. @@ -45,9 +46,6 @@ public static MemoizedCheckedSupplier initializer; - @SuppressWarnings({"squid:S3077"}) // Suppress volatile for generic type - private volatile RETURN value = null; - /** * Create a memoized supplier. * @param initializer to supply at most one non-null value. @@ -60,16 +58,7 @@ private MemoizedCheckedSupplier(CheckedSupplier initializer) { /** @return the lazily initialized object. */ @Override public RETURN get() throws THROW { - RETURN v = value; - if (v == null) { - synchronized (this) { - v = value; - if (v == null) { - v = value = Objects.requireNonNull(initializer.get(), "initializer.get() returns null"); - } - } - } - return v; + return init(initializer); } /** @@ -77,16 +66,6 @@ public RETURN get() throws THROW { * @throws NullPointerException if the object is uninitialized. */ public RETURN getUnchecked() { - return Objects.requireNonNull(value, "value == null"); - } - - /** @return is the object initialized? */ - public boolean isInitialized() { - return value != null; - } - - @Override - public String toString() { - return isInitialized()? "Memoized:" + value: "UNINITIALIZED"; + return getInitializedValue(); } } diff --git a/ratis-common/src/main/java/org/apache/ratis/util/MemoizedFunction.java b/ratis-common/src/main/java/org/apache/ratis/util/MemoizedFunction.java new file mode 100644 index 0000000000..13272f0f65 --- /dev/null +++ b/ratis-common/src/main/java/org/apache/ratis/util/MemoizedFunction.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.ratis.util; + +import java.util.Objects; +import java.util.function.Function; + +/** + * A memoized function is a {@link Function} + * which returns a value by invoking its initializer once + * and then keeps returning the same value as its result. + *

+ * This class is similar to {@link MemoizedSupplier} except that + * the initializer takes a parameter. + *

+ * This class is thread safe. + * + * @param The function result type. + */ +public final class MemoizedFunction + extends MemoizedBase + implements Function { + /** + * @param function to supply at most one non-null value. + * @return a {@link MemoizedFunction} with the given function. + */ + public static MemoizedFunction valueOf(Function function) { + return function instanceof MemoizedFunction ? + (MemoizedFunction) function : new MemoizedFunction<>(function); + } + + private final Function initializer; + + /** + * Create a memoized function. + * @param initializer to supply at most one non-null value. + */ + private MemoizedFunction(Function initializer) { + Objects.requireNonNull(initializer, "initializer == null"); + this.initializer = initializer; + } + + /** + * @param parameter for passing to the initializer. + * Since the returned function is memoized, the parameter is only used at the first call. + * The parameter is ignored in the subsequent calls. + * + * @return the lazily initialized object. + */ + @Override + public RETURN apply(PARAMETER parameter) { + return init(() -> initializer.apply(parameter)); + } +} diff --git a/ratis-common/src/main/java/org/apache/ratis/util/MemoizedSupplier.java b/ratis-common/src/main/java/org/apache/ratis/util/MemoizedSupplier.java index 5c2754a307..588f7ff796 100644 --- a/ratis-common/src/main/java/org/apache/ratis/util/MemoizedSupplier.java +++ b/ratis-common/src/main/java/org/apache/ratis/util/MemoizedSupplier.java @@ -1,4 +1,4 @@ -/** +/* * 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 @@ -24,12 +24,14 @@ * A memoized supplier is a {@link Supplier} * which gets a value by invoking its initializer once * and then keeps returning the same value as its supplied results. - * + *

* This class is thread safe. * * @param The supplier result type. */ -public final class MemoizedSupplier implements Supplier { +public final class MemoizedSupplier + extends MemoizedBase + implements Supplier { /** * @param supplier to supply at most one non-null value. * @return a {@link MemoizedSupplier} with the given supplier. @@ -40,8 +42,6 @@ public static MemoizedSupplier valueOf(Supplier supplier) { } private final Supplier initializer; - @SuppressWarnings({"squid:S3077"}) // Suppress volatile for generic type - private volatile T value = null; /** * Create a memoized supplier. @@ -55,26 +55,6 @@ private MemoizedSupplier(Supplier initializer) { /** @return the lazily initialized object. */ @Override public T get() { - T v = value; - if (v == null) { - synchronized (this) { - v = value; - if (v == null) { - v = value = Objects.requireNonNull(initializer.get(), - "initializer.get() returns null"); - } - } - } - return v; - } - - /** @return is the object initialized? */ - public boolean isInitialized() { - return value != null; - } - - @Override - public String toString() { - return isInitialized()? "Memoized:" + get(): "UNINITIALIZED"; + return init(initializer::get); } } diff --git a/ratis-common/src/main/java/org/apache/ratis/util/StringUtils.java b/ratis-common/src/main/java/org/apache/ratis/util/StringUtils.java index 0f3266e657..50a256e5f8 100644 --- a/ratis-common/src/main/java/org/apache/ratis/util/StringUtils.java +++ b/ratis-common/src/main/java/org/apache/ratis/util/StringUtils.java @@ -81,6 +81,23 @@ public static String format(final String format, final Object... objects) { return String.format(Locale.ENGLISH, format, objects); } + public static String bytes2ShortString(ByteString bytes) { + return bytes.isValidUtf8() ? bytes2ShortUtf8String(bytes) : StringUtils.bytes2HexShortString(bytes); + } + + public static String bytes2ShortUtf8String(ByteString bytes) { + final String utf8 = bytes.toStringUtf8(); + if (utf8.isEmpty()) { + return ""; + } + final int length = utf8.length(); + if (length <= 10) { + return utf8; + } + // return only the first 10 characters + return utf8.substring(0, 10) + "...(length=" + length + ")"; + } + public static String bytes2HexShortString(ByteString bytes) { final int size = bytes.size(); if (size == 0) { diff --git a/ratis-server/src/main/java/org/apache/ratis/server/impl/RaftServerImpl.java b/ratis-server/src/main/java/org/apache/ratis/server/impl/RaftServerImpl.java index a9c80d000b..af7fd27a1b 100644 --- a/ratis-server/src/main/java/org/apache/ratis/server/impl/RaftServerImpl.java +++ b/ratis-server/src/main/java/org/apache/ratis/server/impl/RaftServerImpl.java @@ -18,6 +18,7 @@ package org.apache.ratis.server.impl; import org.apache.ratis.client.impl.ClientProtoUtils; +import org.apache.ratis.client.impl.OrderedAsync; import org.apache.ratis.conf.RaftProperties; import org.apache.ratis.metrics.Timekeeper; import org.apache.ratis.proto.RaftProtos.AppendEntriesReplyProto; @@ -1064,6 +1065,10 @@ private CompletableFuture writeAsyncImpl(RaftClientRequest requ } private CompletableFuture watchAsync(RaftClientRequest request) { + if (OrderedAsync.DUMMY.getContent().equals(request.getMessage().getContent())) { + return CompletableFuture.completedFuture(RaftClientReply.newBuilder().setRequest(request).build()); + } + final CompletableFuture reply = checkLeaderState(request); if (reply != null) { return reply; @@ -1078,7 +1083,7 @@ private CompletableFuture watchAsync(RaftClientRequest request) private CompletableFuture staleReadAsync(RaftClientRequest request) { final long minIndex = request.getType().getStaleRead().getMinIndex(); final long commitIndex = state.getLog().getLastCommittedIndex(); - LOG.debug("{}: minIndex={}, commitIndex={}", getMemberId(), minIndex, commitIndex); + LOG.debug("{}: minIndex={}, commitIndex={} from {}", getMemberId(), minIndex, commitIndex, request.getClientId()); if (commitIndex < minIndex) { final StaleReadException e = new StaleReadException( "Unable to serve stale-read due to server commit index = " + commitIndex + " < min = " + minIndex); diff --git a/ratis-server/src/test/java/org/apache/ratis/LinearizableReadTests.java b/ratis-server/src/test/java/org/apache/ratis/LinearizableReadTests.java index dd536508ce..09781b546e 100644 --- a/ratis-server/src/test/java/org/apache/ratis/LinearizableReadTests.java +++ b/ratis-server/src/test/java/org/apache/ratis/LinearizableReadTests.java @@ -88,10 +88,9 @@ public void setup() { RaftServerConfigKeys.Read.setOption(p, LINEARIZABLE); RaftServerConfigKeys.Read.setLeaderLeaseEnabled(p, isLeaderLeaseEnabled()); RaftServerConfigKeys.Read.ReadIndex.setType(p, readIndexType()); - // Disable dummy request since currently the request is implemented as a watch request - // which can cause follower client to trigger failover to leader which will cause the - // all reads to be sent to the leader, making the follower read moot. - RaftClientConfigKeys.Async.Experimental.setSendDummyRequest(p, false); + + // Enable dummy request so linearizable-read tests exercise the default ordered-async bootstrap path. + RaftClientConfigKeys.Async.Experimental.setSendDummyRequest(p, true); } @Test @@ -151,13 +150,16 @@ static void runTestFollowerLinearizableRead(C cluste final int n = 100; final List f0Replies = new ArrayList<>(n); final List f1Replies = new ArrayList<>(n); - try (RaftClient client = cluster.createClient(leaderId)) { + try (RaftClient client = cluster.createClient(leaderId); + RaftClient c0 = cluster.createClient(f0); + RaftClient c1 = cluster.createClient(f1); + ) { for (int i = 0; i < n; i++) { final int count = i + 1; assertReplyExact(count, client.io().send(INCREMENT)); - f0Replies.add(new Reply(count, client.async().sendReadOnly(QUERY, f0))); - f1Replies.add(new Reply(count, client.async().sendReadOnly(QUERY, f1))); + f0Replies.add(new Reply(count, c0.async().sendReadOnly(QUERY, f0))); + f1Replies.add(new Reply(count, c1.async().sendReadOnly(QUERY, f1))); } for (int i = 0; i < n; i++) { diff --git a/ratis-server/src/test/java/org/apache/ratis/client/impl/RaftClientTestUtil.java b/ratis-server/src/test/java/org/apache/ratis/client/impl/RaftClientTestUtil.java index d90b0cc53f..886879a472 100644 --- a/ratis-server/src/test/java/org/apache/ratis/client/impl/RaftClientTestUtil.java +++ b/ratis-server/src/test/java/org/apache/ratis/client/impl/RaftClientTestUtil.java @@ -29,7 +29,8 @@ /** Interface for testing raft client. */ public interface RaftClientTestUtil { static void assertAsyncRequestSemaphore(RaftClient client, int expectedAvailablePermits, int expectedQueueLength) { - ((RaftClientImpl) client).getOrderedAsync().assertRequestSemaphore(expectedAvailablePermits, expectedQueueLength); + ((RaftClientImpl) client).getOrderedAsync(null) + .assertRequestSemaphore(expectedAvailablePermits, expectedQueueLength); } static ClientInvocationId getClientInvocationId(RaftClient client) { diff --git a/ratis-server/src/test/java/org/apache/ratis/server/impl/MiniRaftCluster.java b/ratis-server/src/test/java/org/apache/ratis/server/impl/MiniRaftCluster.java index 87ffa8d198..825ae8f55b 100644 --- a/ratis-server/src/test/java/org/apache/ratis/server/impl/MiniRaftCluster.java +++ b/ratis-server/src/test/java/org/apache/ratis/server/impl/MiniRaftCluster.java @@ -67,6 +67,7 @@ import java.util.Map; import java.util.Optional; import java.util.Timer; +import java.util.UUID; import java.util.concurrent.ConcurrentHashMap; import java.util.concurrent.ExecutorService; import java.util.concurrent.Executors; @@ -93,6 +94,8 @@ public abstract class MiniRaftCluster implements Closeable { TimeDuration.valueOf(100, TimeUnit.MILLISECONDS); static final AtomicInteger THREAD_COUNT = new AtomicInteger(0); + static final AtomicInteger CLIENT_ID = new AtomicInteger(0); + public abstract static class Factory { public interface Get { Supplier PROPERTIES = JavaUtils.memoize(RaftProperties::new); @@ -747,6 +750,7 @@ public RaftClient createClient(RaftPeerId leaderId, RaftGroup raftGroup, RetryPo public RaftClient createClient(RaftPeerId leaderId, RaftGroup raftGroup, RetryPolicy retryPolicy, RaftPeer primaryServer) { RaftClient.Builder builder = RaftClient.newBuilder() + .setClientId(ClientId.valueOf(new UUID(0, CLIENT_ID.incrementAndGet()))) .setRaftGroup(raftGroup) .setLeaderId(leaderId) .setProperties(properties) From c8460868d419382b6aa8d4f67004392e427e684b Mon Sep 17 00:00:00 2001 From: Sergey Soldatov Date: Thu, 23 Apr 2026 10:20:36 -0700 Subject: [PATCH 321/397] RATIS-2505. Improve RATIS-2387 with direct synchronous append when compose disabled (#1436) --- .../main/java/org/apache/ratis/server/impl/RaftServerImpl.java | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/ratis-server/src/main/java/org/apache/ratis/server/impl/RaftServerImpl.java b/ratis-server/src/main/java/org/apache/ratis/server/impl/RaftServerImpl.java index af7fd27a1b..c0e93338a6 100644 --- a/ratis-server/src/main/java/org/apache/ratis/server/impl/RaftServerImpl.java +++ b/ratis-server/src/main/java/org/apache/ratis/server/impl/RaftServerImpl.java @@ -1687,7 +1687,7 @@ leaderId, getMemberId(), currentTerm, followerCommit, inconsistencyReplyNextInde future.join(); final CompletableFuture appendFuture = entries.isEmpty()? CompletableFuture.completedFuture(null) : appendLogTermIndices != null ? appendLogTermIndices.append(entries, this::appendLog) - : appendLog(entries); + : JavaUtils.allOf(state.getLog().append(entries)); proto.getCommitInfosList().forEach(commitInfoCache::update); @@ -1719,6 +1719,7 @@ leaderId, getMemberId(), currentTerm, followerCommit, inconsistencyReplyNextInde return reply; }); } + private CompletableFuture appendLog(List entries) { return CompletableFuture.completedFuture(null) .thenComposeAsync(dummy -> JavaUtils.allOf(state.getLog().append(entries)), serverExecutor); From 481906cd178098f8f3d50540ed74dc8cc2c9bda9 Mon Sep 17 00:00:00 2001 From: Sammi Chen Date: Sat, 2 May 2026 04:48:31 +0800 Subject: [PATCH 322/397] RATIS-2507. Fix java.lang.IllegalStateException: gap between entries (#1439) --- .../server/raftlog/segmented/LogSegment.java | 31 ++++--- .../raftlog/segmented/SegmentedRaftLog.java | 1 + .../segmented/SegmentedRaftLogCache.java | 8 +- .../raftlog/segmented/TestLogSegment.java | 10 +-- .../segmented/TestSegmentedRaftLog.java | 83 ++++++++++++++----- .../segmented/TestSegmentedRaftLogCache.java | 2 +- 6 files changed, 96 insertions(+), 39 deletions(-) diff --git a/ratis-server/src/main/java/org/apache/ratis/server/raftlog/segmented/LogSegment.java b/ratis-server/src/main/java/org/apache/ratis/server/raftlog/segmented/LogSegment.java index e9cb2e50f9..bb2bde7edb 100644 --- a/ratis-server/src/main/java/org/apache/ratis/server/raftlog/segmented/LogSegment.java +++ b/ratis-server/src/main/java/org/apache/ratis/server/raftlog/segmented/LogSegment.java @@ -202,7 +202,7 @@ static LogSegment loadSegment(RaftStorage storage, File file, LogSegmentStartEnd final CorruptionPolicy corruptionPolicy = CorruptionPolicy.get(storage, RaftStorage::getLogCorruptionPolicy); final boolean isOpen = startEnd.isOpen(); final int entryCount = readSegmentFile(file, startEnd, maxOpSize, corruptionPolicy, raftLogMetrics, entry -> { - segment.append(keepEntryInCache || isOpen, entry, Op.LOAD_SEGMENT_FILE); + segment.append(keepEntryInCache || isOpen, entry, Op.LOAD_SEGMENT_FILE, true); if (logConsumer != null) { logConsumer.accept(entry); } @@ -353,24 +353,17 @@ CorruptionPolicy getLogCorruptionPolicy() { return CorruptionPolicy.get(storage, RaftStorage::getLogCorruptionPolicy); } - void appendToOpenSegment(LogEntryProto entry, Op op) { + void appendToOpenSegment(LogEntryProto entry, Op op, boolean verifyEntryIndex) { Preconditions.assertTrue(isOpen(), "The log segment %s is not open for append", this); - append(true, entry, op); + append(true, entry, op, verifyEntryIndex); } public static final String APPEND_RECORD = LogSegment.class.getSimpleName() + ".append"; - private void append(boolean keepEntryInCache, LogEntryProto entry, Op op) { + private void append(boolean keepEntryInCache, LogEntryProto entry, Op op, boolean verifyEntryIndex) { Objects.requireNonNull(entry, "entry == null"); - final LogRecord currentLast = records.getLast(); - if (currentLast == null) { - Preconditions.assertTrue(entry.getIndex() == startIndex, - "gap between start index %s and first entry to append %s", - startIndex, entry.getIndex()); - } else { - Preconditions.assertTrue(entry.getIndex() == currentLast.getTermIndex().getIndex() + 1, - "gap between entries %s and %s", entry.getIndex(), currentLast.getTermIndex().getIndex()); + if (verifyEntryIndex) { + verifyEntryIndex(entry.getIndex()); } - final LogRecord record = new LogRecord(totalFileSize, entry); if (keepEntryInCache) { // It is important to put the entry into the cache before appending the @@ -385,6 +378,18 @@ private void append(boolean keepEntryInCache, LogEntryProto entry, Op op) { endIndex = entry.getIndex(); } + void verifyEntryIndex(long entryIndex) { + final LogRecord currentLast = records.getLast(); + if (currentLast == null) { + Preconditions.assertTrue(entryIndex == startIndex, + "gap between start index %s and first entry to append %s", + startIndex, entryIndex); + } else { + Preconditions.assertTrue(entryIndex == currentLast.getTermIndex().getIndex() + 1, + "gap between entries %s and %s", entryIndex, currentLast.getTermIndex().getIndex()); + } + } + LogEntryProto getEntryFromCache(TermIndex ti) { return entryCache.get(ti); } diff --git a/ratis-server/src/main/java/org/apache/ratis/server/raftlog/segmented/SegmentedRaftLog.java b/ratis-server/src/main/java/org/apache/ratis/server/raftlog/segmented/SegmentedRaftLog.java index 6bcc3f8e1c..a6ea6e3caf 100644 --- a/ratis-server/src/main/java/org/apache/ratis/server/raftlog/segmented/SegmentedRaftLog.java +++ b/ratis-server/src/main/java/org/apache/ratis/server/raftlog/segmented/SegmentedRaftLog.java @@ -430,6 +430,7 @@ protected CompletableFuture appendEntryImpl(LogEntryProto entry, Transacti // If the entry has state machine data, then the entry should be inserted // to statemachine first and then to the cache. Not following the order // will leave a spurious entry in the cache. + cache.verifyAppendEntryIndex(entry); CompletableFuture writeFuture = fileLogWorker.writeLogEntry(entry, context).getFuture(); if (stateMachineCachingEnabled) { diff --git a/ratis-server/src/main/java/org/apache/ratis/server/raftlog/segmented/SegmentedRaftLogCache.java b/ratis-server/src/main/java/org/apache/ratis/server/raftlog/segmented/SegmentedRaftLogCache.java index 46acbcc3d8..714943c49c 100644 --- a/ratis-server/src/main/java/org/apache/ratis/server/raftlog/segmented/SegmentedRaftLogCache.java +++ b/ratis-server/src/main/java/org/apache/ratis/server/raftlog/segmented/SegmentedRaftLogCache.java @@ -622,11 +622,17 @@ TermIndex getLastTermIndex() { } } + void verifyAppendEntryIndex(LogEntryProto entry) { + // SegmentedRaftLog does the segment creation/rolling work. + Objects.requireNonNull(openSegment, "openSegment == null"); + openSegment.verifyEntryIndex(entry.getIndex()); + } + void appendEntry(LogEntryProto entry, LogSegment.Op op) { // SegmentedRaftLog does the segment creation/rolling work. Here we just // simply append the entry into the open segment. Objects.requireNonNull(openSegment, "openSegment == null"); - openSegment.appendToOpenSegment(entry, op); + openSegment.appendToOpenSegment(entry, op, false); } /** diff --git a/ratis-test/src/test/java/org/apache/ratis/server/raftlog/segmented/TestLogSegment.java b/ratis-test/src/test/java/org/apache/ratis/server/raftlog/segmented/TestLogSegment.java index a95c683c88..6a75dfb36e 100644 --- a/ratis-test/src/test/java/org/apache/ratis/server/raftlog/segmented/TestLogSegment.java +++ b/ratis-test/src/test/java/org/apache/ratis/server/raftlog/segmented/TestLogSegment.java @@ -206,7 +206,7 @@ public void testAppendEntries() throws Exception { SimpleOperation op = new SimpleOperation("m" + i); LogEntryProto entry = LogProtoUtils.toLogEntryProto(op.getLogEntryContent(), term, i++ + start); size += getEntrySize(entry, LogSegment.Op.WRITE_CACHE_WITHOUT_STATE_MACHINE_CACHE); - segment.appendToOpenSegment(entry, LogSegment.Op.WRITE_CACHE_WITHOUT_STATE_MACHINE_CACHE); + segment.appendToOpenSegment(entry, LogSegment.Op.WRITE_CACHE_WITHOUT_STATE_MACHINE_CACHE, true); } Assertions.assertTrue(segment.getTotalFileSize() >= max); @@ -238,18 +238,18 @@ public void testAppendWithGap() throws Exception { final StateMachineLogEntryProto m = op.getLogEntryContent(); try { LogEntryProto entry = LogProtoUtils.toLogEntryProto(m, 0, 1001); - segment.appendToOpenSegment(entry, LogSegment.Op.WRITE_CACHE_WITHOUT_STATE_MACHINE_CACHE); + segment.appendToOpenSegment(entry, LogSegment.Op.WRITE_CACHE_WITHOUT_STATE_MACHINE_CACHE, true); Assertions.fail("should fail since the entry's index needs to be 1000"); } catch (IllegalStateException e) { // the exception is expected. } LogEntryProto entry = LogProtoUtils.toLogEntryProto(m, 0, 1000); - segment.appendToOpenSegment(entry, LogSegment.Op.WRITE_CACHE_WITHOUT_STATE_MACHINE_CACHE); + segment.appendToOpenSegment(entry, LogSegment.Op.WRITE_CACHE_WITHOUT_STATE_MACHINE_CACHE, true); try { entry = LogProtoUtils.toLogEntryProto(m, 0, 1002); - segment.appendToOpenSegment(entry, LogSegment.Op.WRITE_CACHE_WITHOUT_STATE_MACHINE_CACHE); + segment.appendToOpenSegment(entry, LogSegment.Op.WRITE_CACHE_WITHOUT_STATE_MACHINE_CACHE, true); Assertions.fail("should fail since the entry's index needs to be 1001"); } catch (IllegalStateException e) { // the exception is expected. @@ -264,7 +264,7 @@ public void testTruncate() throws Exception { for (int i = 0; i < 100; i++) { LogEntryProto entry = LogProtoUtils.toLogEntryProto( new SimpleOperation("m" + i).getLogEntryContent(), term, i + start); - segment.appendToOpenSegment(entry, LogSegment.Op.WRITE_CACHE_WITHOUT_STATE_MACHINE_CACHE); + segment.appendToOpenSegment(entry, LogSegment.Op.WRITE_CACHE_WITHOUT_STATE_MACHINE_CACHE, true); } // truncate an open segment (remove 1080~1099) diff --git a/ratis-test/src/test/java/org/apache/ratis/server/raftlog/segmented/TestSegmentedRaftLog.java b/ratis-test/src/test/java/org/apache/ratis/server/raftlog/segmented/TestSegmentedRaftLog.java index 43aafc8967..181d1fa430 100644 --- a/ratis-test/src/test/java/org/apache/ratis/server/raftlog/segmented/TestSegmentedRaftLog.java +++ b/ratis-test/src/test/java/org/apache/ratis/server/raftlog/segmented/TestSegmentedRaftLog.java @@ -67,7 +67,6 @@ import java.util.function.LongSupplier; import java.util.function.Supplier; import java.util.stream.Stream; -import java.util.concurrent.atomic.AtomicBoolean; import org.junit.jupiter.api.AfterEach; import org.junit.jupiter.api.Assertions; @@ -83,6 +82,8 @@ import static java.lang.Boolean.FALSE; import static java.lang.Boolean.TRUE; import static org.apache.ratis.server.raftlog.segmented.SegmentedRaftLogWorker.RUN_WORKER; +import static org.junit.jupiter.api.Assertions.assertEquals; +import static org.junit.jupiter.api.Assertions.assertThrows; import static org.junit.jupiter.api.Assertions.assertTrue; import static org.junit.jupiter.params.provider.Arguments.arguments; @@ -231,7 +232,7 @@ public void testLoadLogSegments(Boolean useAsyncFlush, Boolean smSyncFlush) thro // check if log entries are loaded correctly for (LogEntryProto e : entries) { LogEntryProto entry = raftLog.get(e.getIndex()); - Assertions.assertEquals(e, entry); + assertEquals(e, entry); } final LogEntryHeader[] termIndices = raftLog.getEntries(0, 500); @@ -245,7 +246,7 @@ public void testLoadLogSegments(Boolean useAsyncFlush, Boolean smSyncFlush) thro }) .toArray(LogEntryProto[]::new); Assertions.assertArrayEquals(entries, entriesFromLog); - Assertions.assertEquals(entries[entries.length - 1], getLastEntry(raftLog)); + assertEquals(entries[entries.length - 1], getLastEntry(raftLog)); final RatisMetricRegistry metricRegistryForLogWorker = RaftLogMetricsBase.createRegistry(MEMBER_ID); @@ -400,7 +401,7 @@ public void testAppendAndRoll(Boolean useAsyncFlush, Boolean smSyncFlush) throws raftLog.open(RaftLog.INVALID_LOG_INDEX, null); // check if the raft log is correct checkEntries(raftLog, entries, 0, entries.size()); - Assertions.assertEquals(9, raftLog.getRaftLogCache().getNumOfSegments()); + assertEquals(9, raftLog.getRaftLogCache().getNumOfSegments()); } } @@ -466,12 +467,12 @@ public void testPurgeAfterAppendEntry(Boolean useAsyncFlush, Boolean smSyncFlush if(!tasksAdded.await(FIVE_SECONDS.getDuration(), FIVE_SECONDS.getUnit())) { throw new TimeoutException(); } - Assertions.assertEquals(entries.size() + 1, tasksCount.get()); + assertEquals(entries.size() + 1, tasksCount.get()); // check if the purge task is executed final Long purged = purgeFuture.get().get(); LOG.info("purgeIndex = {}, purged = {}", endIndexOfClosedSegment, purged); - Assertions.assertEquals(endIndexOfClosedSegment, raftLog.getRaftLogCache().getStartIndex()); + assertEquals(endIndexOfClosedSegment, raftLog.getRaftLogCache().getStartIndex()); // check if the appendEntry futures are done JavaUtils.allOf(appendFutures).get(FIVE_SECONDS.getDuration(), FIVE_SECONDS.getUnit()); @@ -515,7 +516,7 @@ private void testTruncate(List entries, long fromIndex) raftLog.open(RaftLog.INVALID_LOG_INDEX, null); // check if the raft log is correct if (fromIndex > 0) { - Assertions.assertEquals(entries.get((int) (fromIndex - 1)), + assertEquals(entries.get((int) (fromIndex - 1)), getLastEntry(raftLog)); } else { Assertions.assertNull(raftLog.getLastEntryTermIndex()); @@ -529,7 +530,7 @@ private void checkEntries(RaftLog raftLog, List expected, if (size > 0) { for (int i = offset; i < size + offset; i++) { LogEntryProto entry = raftLog.get(expected.get(i).getIndex()); - Assertions.assertEquals(expected.get(i), entry); + assertEquals(expected.get(i), entry); } final LogEntryHeader[] termIndices = raftLog.getEntries( expected.get(offset).getIndex(), @@ -637,7 +638,7 @@ private void purgeAndVerify(int startTerm, int endTerm, int segmentSize, int pur final CompletableFuture f = raftLog.purge(purgeIndex); final Long purged = f.get(); LOG.info("purgeIndex = {}, purged = {}", purgeIndex, purged); - Assertions.assertEquals(expectedIndex, raftLog.getRaftLogCache().getStartIndex()); + assertEquals(expectedIndex, raftLog.getRaftLogCache().getStartIndex()); } } @@ -681,9 +682,9 @@ public void testAppendEntriesWithInconsistency(Boolean useAsyncFlush, Boolean sm checkFailedEntries(entries, 650, retryCache); checkEntries(raftLog, entries, 0, 650); checkEntries(raftLog, newEntries, 100, 100); - Assertions.assertEquals(newEntries.get(newEntries.size() - 1), + assertEquals(newEntries.get(newEntries.size() - 1), getLastEntry(raftLog)); - Assertions.assertEquals(newEntries.get(newEntries.size() - 1).getIndex(), + assertEquals(newEntries.get(newEntries.size() - 1).getIndex(), raftLog.getFlushIndex()); } @@ -693,13 +694,57 @@ public void testAppendEntriesWithInconsistency(Boolean useAsyncFlush, Boolean sm raftLog.open(RaftLog.INVALID_LOG_INDEX, null); checkEntries(raftLog, entries, 0, 650); checkEntries(raftLog, newEntries, 100, 100); - Assertions.assertEquals(newEntries.get(newEntries.size() - 1), + assertEquals(newEntries.get(newEntries.size() - 1), getLastEntry(raftLog)); - Assertions.assertEquals(newEntries.get(newEntries.size() - 1).getIndex(), + assertEquals(newEntries.get(newEntries.size() - 1).getIndex(), raftLog.getFlushIndex()); SegmentedRaftLogCache cache = raftLog.getRaftLogCache(); - Assertions.assertEquals(5, cache.getNumOfSegments()); + assertEquals(5, cache.getNumOfSegments()); + } + } + + @ParameterizedTest + @MethodSource("data") + public void testAppendEntriesWithGap(Boolean useAsyncFlush, Boolean smSyncFlush) throws Exception { + RaftServerConfigKeys.Log.setAsyncFlushEnabled(properties, useAsyncFlush); + RaftServerConfigKeys.Log.StateMachineData.setSync(properties, smSyncFlush); + // prepare the log for truncation + List ranges = prepareRanges(0, 5, 200, 0); + List entries = prepareLogEntries(ranges, null); + + final RetryCache retryCache = RetryCacheTestUtil.createRetryCache(); + try (SegmentedRaftLog raftLog = + RetryCacheTestUtil.newSegmentedRaftLog(MEMBER_ID, retryCache, storage, properties)) { + raftLog.open(RaftLog.INVALID_LOG_INDEX, null); + entries.forEach(entry -> RetryCacheTestUtil.createEntry(retryCache, entry)); + // append entries to the raftlog + entries.stream().map(raftLog::appendEntry).forEach(CompletableFuture::join); + } + + long lastIndex = ranges.get(ranges.size() - 1).end; + long snapshotIndex = lastIndex + 100; + LogEntryProto entryProto = prepareLogEntry(4, snapshotIndex + 1, null, false); + final LongSupplier getSnapshotIndexFromStateMachine = new LongSupplier() { + @Override + public long getAsLong() { + return snapshotIndex; + } + }; + try (SegmentedRaftLog raftLog = newSegmentedRaftLog(getSnapshotIndexFromStateMachine)) { + raftLog.open(RaftLog.INVALID_LOG_INDEX, null); + // Assert the wrapped exception + IllegalStateException exception = assertThrows(IllegalStateException.class, + () -> raftLog.appendEntry(entryProto)); + // Assert the original cause + assertTrue(exception.getMessage().contains("gap between entries")); + } + + // load the raftlog again and check + try (SegmentedRaftLog raftLog = + RetryCacheTestUtil.newSegmentedRaftLog(MEMBER_ID, retryCache, storage, properties)) { + raftLog.open(RaftLog.INVALID_LOG_INDEX, null); + Assertions.assertEquals(lastIndex, raftLog.getRaftLogCache().getEndIndex()); } } @@ -795,7 +840,7 @@ public void notifyLogFailed(Throwable cause, LogEntryProto entry) { // SegmentedRaftLogWorker should catch TimeoutIOException CompletableFuture f = raftLog.appendEntry(entry); // Wait for async writeStateMachineData to finish - ex = Assertions.assertThrows(ExecutionException.class, f::get); + ex = assertThrows(ExecutionException.class, f::get); } Assertions.assertSame(LifeCycle.State.PAUSED, sm.getLifeCycleState()); Assertions.assertInstanceOf(TimeoutIOException.class, ex.getCause()); @@ -815,9 +860,9 @@ static Thread startAppendEntryThread(RaftLog raftLog, LogEntryProto entry) { void assertIndices(RaftLog raftLog, long expectedFlushIndex, long expectedNextIndex) { LOG.info("assert expectedFlushIndex={}", expectedFlushIndex); - Assertions.assertEquals(expectedFlushIndex, raftLog.getFlushIndex()); + assertEquals(expectedFlushIndex, raftLog.getFlushIndex()); LOG.info("assert expectedNextIndex={}", expectedNextIndex); - Assertions.assertEquals(expectedNextIndex, raftLog.getNextIndex()); + assertEquals(expectedNextIndex, raftLog.getNextIndex()); } void assertIndicesMultipleAttempts(RaftLog raftLog, long expectedFlushIndex, long expectedNextIndex) @@ -938,10 +983,10 @@ public void testConcurrentGetDuringAppend() throws Exception { // When the reader's get() call completed, the append was fully finished, // so it should have returned the correct entry. - Assertions.assertEquals(newEntry.getIndex(), raftLog.getLastEntryTermIndex().getIndex()); + assertEquals(newEntry.getIndex(), raftLog.getLastEntryTermIndex().getIndex()); readEntry.set(raftLog.get(newEntry.getIndex())); Assertions.assertNotNull(readEntry.get()); - Assertions.assertEquals(newEntry, readEntry.get()); + assertEquals(newEntry, readEntry.get()); } finally { CodeInjectionForTesting.remove(LogSegment.APPEND_RECORD); } diff --git a/ratis-test/src/test/java/org/apache/ratis/server/raftlog/segmented/TestSegmentedRaftLogCache.java b/ratis-test/src/test/java/org/apache/ratis/server/raftlog/segmented/TestSegmentedRaftLogCache.java index 532e32c87d..3133fb36f6 100644 --- a/ratis-test/src/test/java/org/apache/ratis/server/raftlog/segmented/TestSegmentedRaftLogCache.java +++ b/ratis-test/src/test/java/org/apache/ratis/server/raftlog/segmented/TestSegmentedRaftLogCache.java @@ -63,7 +63,7 @@ private LogSegment prepareLogSegment(long start, long end, boolean isOpen) { for (long i = start; i <= end; i++) { SimpleOperation m = new SimpleOperation("m" + i); LogEntryProto entry = LogProtoUtils.toLogEntryProto(m.getLogEntryContent(), 0, i); - s.appendToOpenSegment(entry, LogSegment.Op.WRITE_CACHE_WITHOUT_STATE_MACHINE_CACHE); + s.appendToOpenSegment(entry, LogSegment.Op.WRITE_CACHE_WITHOUT_STATE_MACHINE_CACHE, true); } if (!isOpen) { s.close(); From 803de353c66f8621c56031f3d2241bdc81aaa8d0 Mon Sep 17 00:00:00 2001 From: Tsz-Wo Nicholas Sze Date: Tue, 5 May 2026 00:10:34 -0700 Subject: [PATCH 323/397] RATIS-2508. appendEntries log messages improvement. (#1440) --- .../java/org/apache/ratis/grpc/GrpcUtil.java | 2 +- .../ratis/grpc/server/GrpcLogAppender.java | 14 ++++------ .../ratis/server/impl/FollowerState.java | 4 +-- .../ratis/server/impl/RaftServerImpl.java | 28 +++++++++---------- .../ratis/server/raftlog/LogProtoUtils.java | 20 +++++++++++-- .../ratis/server/util/ServerStringUtils.java | 17 ++++------- 6 files changed, 44 insertions(+), 41 deletions(-) diff --git a/ratis-grpc/src/main/java/org/apache/ratis/grpc/GrpcUtil.java b/ratis-grpc/src/main/java/org/apache/ratis/grpc/GrpcUtil.java index e7bb2b1693..df076875bf 100644 --- a/ratis-grpc/src/main/java/org/apache/ratis/grpc/GrpcUtil.java +++ b/ratis-grpc/src/main/java/org/apache/ratis/grpc/GrpcUtil.java @@ -97,7 +97,7 @@ static Throwable unwrapThrowable(Throwable t) { return unwrapped; } } - return t; + return JavaUtils.unwrapCompletionException(t); } static IOException unwrapException(StatusRuntimeException se) { diff --git a/ratis-grpc/src/main/java/org/apache/ratis/grpc/server/GrpcLogAppender.java b/ratis-grpc/src/main/java/org/apache/ratis/grpc/server/GrpcLogAppender.java index 69421e9f0f..053cc5c0f4 100644 --- a/ratis-grpc/src/main/java/org/apache/ratis/grpc/server/GrpcLogAppender.java +++ b/ratis-grpc/src/main/java/org/apache/ratis/grpc/server/GrpcLogAppender.java @@ -63,6 +63,8 @@ import java.util.concurrent.TimeUnit; import java.util.concurrent.atomic.AtomicLong; +import static org.apache.ratis.server.raftlog.LogProtoUtils.toLogEntryTermIndexString; + /** * A new log appender implementation using grpc bi-directional stream API. */ @@ -301,8 +303,8 @@ private void mayWait() { getEventAwaitForSignal().await(getWaitTimeMs() + errorWaitTimeMs(), TimeUnit.MILLISECONDS); } catch (InterruptedException ie) { - LOG.warn(this + ": Wait interrupted by " + ie); Thread.currentThread().interrupt(); + LOG.warn("{} is interrupted: {}", this, ie.toString()); } } @@ -616,11 +618,11 @@ void removePending(InstallSnapshotReplyProto reply) { if (isNotificationOnly) { Preconditions.assertSame(InstallSnapshotReplyBodyCase.SNAPSHOTINDEX, reply.getInstallSnapshotReplyBodyCase(), "reply case"); - Preconditions.assertSame(INSTALL_SNAPSHOT_NOTIFICATION_INDEX, (int) index, "poll index"); + Preconditions.assertSame(INSTALL_SNAPSHOT_NOTIFICATION_INDEX, index, "poll index"); } else { Preconditions.assertSame(InstallSnapshotReplyBodyCase.REQUESTINDEX, reply.getInstallSnapshotReplyBodyCase(), "reply case"); - Preconditions.assertSame(reply.getRequestIndex(), (int) index, "poll index"); + Preconditions.assertSame(reply.getRequestIndex(), index, "poll index"); } } } @@ -889,13 +891,9 @@ boolean isHeartbeat() { @Override public String toString() { - final String entries = entriesCount == 0? "" - : entriesCount == 1? ",entry=" + firstEntry - : ",entries=" + firstEntry + "..." + lastEntry; return JavaUtils.getClassSimpleName(getClass()) + ":cid=" + callId - + ",entriesCount=" + entriesCount - + entries; + + ":" + toLogEntryTermIndexString(entriesCount, firstEntry, lastEntry); } } diff --git a/ratis-server/src/main/java/org/apache/ratis/server/impl/FollowerState.java b/ratis-server/src/main/java/org/apache/ratis/server/impl/FollowerState.java index b01abcddc0..7a8414ca2d 100644 --- a/ratis-server/src/main/java/org/apache/ratis/server/impl/FollowerState.java +++ b/ratis-server/src/main/java/org/apache/ratis/server/impl/FollowerState.java @@ -158,8 +158,8 @@ private void runImpl() { } synchronized (server) { if (roleChangeChecking(electionTimeout)) { - LOG.info("{}: change to CANDIDATE, lastRpcElapsedTime:{}, electionTimeout:{}", - this, lastRpcTime.elapsedTime(), electionTimeout); + LOG.info("{}: change to CANDIDATE, lastRpcElapsedTime:{}ms, electionTimeout:{}", + this, lastRpcTime.elapsedTimeMs(), electionTimeout); server.getLeaderElectionMetrics().onLeaderElectionTimeout(); // Update timeout metric counters. // election timeout, should become a candidate server.changeToCandidate(false); diff --git a/ratis-server/src/main/java/org/apache/ratis/server/impl/RaftServerImpl.java b/ratis-server/src/main/java/org/apache/ratis/server/impl/RaftServerImpl.java index c0e93338a6..1c9cd3f658 100644 --- a/ratis-server/src/main/java/org/apache/ratis/server/impl/RaftServerImpl.java +++ b/ratis-server/src/main/java/org/apache/ratis/server/impl/RaftServerImpl.java @@ -150,6 +150,7 @@ import static org.apache.ratis.server.impl.ServerProtoUtils.toReadIndexRequestProto; import static org.apache.ratis.server.impl.ServerProtoUtils.toRequestVoteReplyProto; import static org.apache.ratis.server.impl.ServerProtoUtils.toStartLeaderElectionReplyProto; +import static org.apache.ratis.server.raftlog.LogProtoUtils.toLogEntryTermIndexString; import static org.apache.ratis.server.util.ServerStringUtils.toAppendEntriesReplyString; import static org.apache.ratis.server.util.ServerStringUtils.toAppendEntriesRequestString; import static org.apache.ratis.server.util.ServerStringUtils.toRequestVoteReplyString; @@ -239,18 +240,16 @@ public long[] getFollowerMatchIndices() { private final RetryCacheImpl retryCache; private final CommitInfoCache commitInfoCache = new CommitInfoCache(); private final WriteIndexCache writeIndexCache; + private final NavigableIndices appendLogTermIndices; private final RaftServerJmxAdapter jmxAdapter = new RaftServerJmxAdapter(this); private final LeaderElectionMetrics leaderElectionMetrics; private final RaftServerMetricsImpl raftServerMetrics; - private final CountDownLatch closeFinishedLatch = new CountDownLatch(1); - // To avoid append entry before complete start() method - // For example, if thread1 start(), but before thread1 startAsFollower(), thread2 receive append entry - // request, and change state to RUNNING by lifeCycle.compareAndTransition(STARTING, RUNNING), - // then thread1 execute lifeCycle.transition(RUNNING) in startAsFollower(), - // So happens IllegalStateException: ILLEGAL TRANSITION: RUNNING -> RUNNING, - private final AtomicBoolean startComplete; + // Disallow appendEntries before start() complete; otherwise, it could fail with illegal lifeCycle transition + private final AtomicBoolean startComplete = new AtomicBoolean(false); + private final AtomicBoolean firstElectionSinceStartup = new AtomicBoolean(true); + private final CountDownLatch closeFinishedLatch = new CountDownLatch(1); private final TransferLeadership transferLeadership; private final SnapshotManagementRequestHandler snapshotRequestHandler; @@ -258,12 +257,8 @@ public long[] getFollowerMatchIndices() { private final ExecutorService serverExecutor; private final ExecutorService clientExecutor; - - private final AtomicBoolean firstElectionSinceStartup = new AtomicBoolean(true); private final ThreadGroup threadGroup; - private final NavigableIndices appendLogTermIndices; - RaftServerImpl(RaftGroup group, StateMachine stateMachine, RaftServerProxy proxy, RaftStorage.StartupOption option) throws IOException { final RaftPeerId id = proxy.getId(); @@ -292,9 +287,6 @@ public long[] getFollowerMatchIndices() { this.raftServerMetrics = RaftServerMetricsImpl.computeIfAbsentRaftServerMetrics( getMemberId(), this::getCommitIndex, retryCache::getStatistics); - this.startComplete = new AtomicBoolean(false); - this.threadGroup = new ThreadGroup(proxy.getThreadGroup(), getMemberId().toString()); - this.transferLeadership = new TransferLeadership(this, properties); this.snapshotRequestHandler = new SnapshotManagementRequestHandler(this); this.snapshotInstallationHandler = new SnapshotInstallationHandler(this, properties); @@ -309,6 +301,7 @@ public long[] getFollowerMatchIndices() { RaftServerConfigKeys.ThreadPool.clientCached(properties), RaftServerConfigKeys.ThreadPool.clientSize(properties), id + "-client"); + this.threadGroup = new ThreadGroup(proxy.getThreadGroup(), getMemberId().toString()); } private long getCommitIndex(RaftPeerId id) { @@ -1703,6 +1696,11 @@ leaderId, getMemberId(), currentTerm, followerCommit, inconsistencyReplyNextInde final long commitIndex = effectiveCommitIndex(proto.getLeaderCommit(), previous, entries.size()); final long matchIndex = isHeartbeat? RaftLog.INVALID_LOG_INDEX: entries.get(entries.size() - 1).getIndex(); return appendFuture.whenCompleteAsync((r, t) -> { + if (t != null) { + LOG.warn("{}: appendEntries* failed: {}", getMemberId(), toLogEntryTermIndexString(entries), t); + } else if (LOG.isDebugEnabled()) { + LOG.debug("{}: appendEntries* succeeded: {}", getMemberId(), toLogEntryTermIndexString(entries)); + } followerState.ifPresent(fs -> fs.updateLastRpcTime(FollowerState.UpdateType.APPEND_COMPLETE)); timer.stop(); }, getServerExecutor()).thenApply(v -> { @@ -1753,7 +1751,7 @@ private long checkInconsistentAppendEntries(TermIndex previous, List entries) { public static String toLogEntriesShortString(List entries, Function stateMachineToString) { - return entries == null ? null - : entries.isEmpty()? "" - : "size=" + entries.size() + ", first=" + toLogEntryString(entries.get(0), stateMachineToString); + if (entries == null) { + return null; + } + return toLogEntryTermIndexString(entries) + + (entries.isEmpty() ? "" : ", first=" + toLogEntryString(entries.get(0), stateMachineToString)); + } + + public static String toLogEntryTermIndexString(List entries) { + final int n = entries.size(); + return n == 0 ? toLogEntryTermIndexString(n, null, null) + : toLogEntryTermIndexString(n, TermIndex.valueOf(entries.get(0)), TermIndex.valueOf(entries.get(n - 1))); + } + + public static String toLogEntryTermIndexString(int n, TermIndex first, TermIndex last) { + return n == 0 ? "HEARTBEAT" + : n == 1 ? "entry=" + first + : n + " entries=" + first + "..." + last; } public static LogEntryProto toLogEntryProto(RaftConfiguration conf, Long term, long index) { diff --git a/ratis-server/src/main/java/org/apache/ratis/server/util/ServerStringUtils.java b/ratis-server/src/main/java/org/apache/ratis/server/util/ServerStringUtils.java index 3a5db62859..50b0918b83 100644 --- a/ratis-server/src/main/java/org/apache/ratis/server/util/ServerStringUtils.java +++ b/ratis-server/src/main/java/org/apache/ratis/server/util/ServerStringUtils.java @@ -21,18 +21,17 @@ import org.apache.ratis.proto.RaftProtos.AppendEntriesRequestProto; import org.apache.ratis.proto.RaftProtos.InstallSnapshotReplyProto; import org.apache.ratis.proto.RaftProtos.InstallSnapshotRequestProto; -import org.apache.ratis.proto.RaftProtos.LogEntryProto; import org.apache.ratis.proto.RaftProtos.RequestVoteReplyProto; import org.apache.ratis.proto.RaftProtos.StateMachineLogEntryProto; import org.apache.ratis.protocol.RaftGroupMemberId; import org.apache.ratis.server.protocol.TermIndex; -import org.apache.ratis.server.raftlog.LogProtoUtils; import org.apache.ratis.util.JavaUtils; import org.apache.ratis.util.ProtoUtils; -import java.util.List; import java.util.function.Function; +import static org.apache.ratis.server.raftlog.LogProtoUtils.toLogEntriesShortString; + /** * This class provides convenient utilities for converting Protocol Buffers messages to strings. * The output strings are for information purpose only. @@ -50,14 +49,12 @@ public static String toAppendEntriesRequestString(AppendEntriesRequestProto requ if (request == null) { return null; } - final List entries = request.getEntriesList(); return ProtoUtils.toString(request.getServerRequest()) + "-t" + request.getLeaderTerm() + ",previous=" + TermIndex.valueOf(request.getPreviousLog()) + ",leaderCommit=" + request.getLeaderCommit() + ",initializing? " + request.getInitializing() - + "," + (entries.isEmpty()? "HEARTBEAT" : "entries: " + - LogProtoUtils.toLogEntriesShortString(entries, stateMachineToString)); + + "," + toLogEntriesShortString(request.getEntriesList(), stateMachineToString); } public static String toAppendEntriesReplyString(AppendEntriesReplyProto reply) { @@ -87,7 +84,7 @@ public static String toInstallSnapshotRequestString(InstallSnapshotRequestProto s = "notify:" + TermIndex.valueOf(notification.getFirstAvailableTermIndex()); break; default: - throw new IllegalStateException("Unexpected body case in " + request); + throw new IllegalStateException("Unexpected InstallSnapshotRequestBodyCase in " + request); } return ProtoUtils.toString(request.getServerRequest()) + "-t" + request.getLeaderTerm() @@ -122,11 +119,7 @@ public static String toRequestVoteReplyString(RequestVoteReplyProto proto) { + "-last:" + TermIndex.valueOf(proto.getLastEntry()); } - /** - * Used to generate the necessary unified name in the submodules under - * {@link org.apache.ratis.server.impl.RaftServerImpl}, which consists - * of {@link org.apache.ratis.server.impl.ServerState#memberId} and the specific class. - */ + /** Generate the unified name for the given member and class. */ public static String generateUnifiedName(RaftGroupMemberId memberId, Class clazz) { return memberId + "-" + JavaUtils.getClassSimpleName(clazz); } From 62e9c3d09509e292a846160cd9a20bdcec9b408c Mon Sep 17 00:00:00 2001 From: "dependabot[bot]" <49699333+dependabot[bot]@users.noreply.github.com> Date: Tue, 5 May 2026 00:29:36 -0700 Subject: [PATCH 324/397] RATIS-2513. Bump bouncycastle.version from 1.83 to 1.84 (#1437) --- ratis-test/pom.xml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/ratis-test/pom.xml b/ratis-test/pom.xml index 050600319c..577262d84c 100644 --- a/ratis-test/pom.xml +++ b/ratis-test/pom.xml @@ -24,7 +24,7 @@ Apache Ratis Test - 1.83 + 1.84 From 41f9a7d6f02c98224cfefc878ba1c534bb1053b1 Mon Sep 17 00:00:00 2001 From: "dependabot[bot]" <49699333+dependabot[bot]@users.noreply.github.com> Date: Tue, 5 May 2026 10:06:07 +0200 Subject: [PATCH 325/397] RATIS-2515. Bump common-custom-user-data-maven-extension to 2.2.0 (#1442) --- .mvn/extensions.xml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/.mvn/extensions.xml b/.mvn/extensions.xml index 960b8cd9ef..597996803a 100644 --- a/.mvn/extensions.xml +++ b/.mvn/extensions.xml @@ -29,6 +29,6 @@ com.gradle common-custom-user-data-maven-extension - 2.1.0 + 2.2.0 From 6e171d8779397f7245feacd4a2a87d27f4ca3cbf Mon Sep 17 00:00:00 2001 From: "dependabot[bot]" <49699333+dependabot[bot]@users.noreply.github.com> Date: Tue, 5 May 2026 10:22:41 +0200 Subject: [PATCH 326/397] RATIS-2517. Bump actions/upload-artifact to 7.0.1 (#1430) --- .github/workflows/check.yaml | 8 ++++---- .github/workflows/ci.yaml | 2 +- .github/workflows/repeat-test.yaml | 2 +- .github/workflows/vulnerability-check.yaml | 2 +- 4 files changed, 7 insertions(+), 7 deletions(-) diff --git a/.github/workflows/check.yaml b/.github/workflows/check.yaml index 7d38342068..41f85a8f37 100644 --- a/.github/workflows/check.yaml +++ b/.github/workflows/check.yaml @@ -176,7 +176,7 @@ jobs: - name: Archive build results if: ${{ !cancelled() }} - uses: actions/upload-artifact@bbbca2ddaa5d8feaa63e36b76fdaad77386f024f # v7.0.0 + uses: actions/upload-artifact@043fb46d1a93c77aae656e7c1c64a875d1fc6a0a # v7.0.1 with: name: ${{ (inputs.split && format('{0}-{1}', inputs.script, inputs.split)) || inputs.script }} path: target/${{ inputs.script }} @@ -186,7 +186,7 @@ jobs: # to avoid the need for 3 more inputs. - name: Store binaries for tests if: ${{ inputs.script == 'build' && !cancelled() }} - uses: actions/upload-artifact@bbbca2ddaa5d8feaa63e36b76fdaad77386f024f # v7.0.0 + uses: actions/upload-artifact@043fb46d1a93c77aae656e7c1c64a875d1fc6a0a # v7.0.1 with: name: ratis-bin path: | @@ -195,7 +195,7 @@ jobs: - name: Store source tarball for compilation if: ${{ inputs.script == 'build' && !cancelled() }} - uses: actions/upload-artifact@bbbca2ddaa5d8feaa63e36b76fdaad77386f024f # v7.0.0 + uses: actions/upload-artifact@043fb46d1a93c77aae656e7c1c64a875d1fc6a0a # v7.0.1 with: name: ratis-src path: | @@ -204,7 +204,7 @@ jobs: - name: Store Maven repo for tests if: ${{ inputs.script == 'build' && !cancelled() }} - uses: actions/upload-artifact@bbbca2ddaa5d8feaa63e36b76fdaad77386f024f # v7.0.0 + uses: actions/upload-artifact@043fb46d1a93c77aae656e7c1c64a875d1fc6a0a # v7.0.1 with: name: maven-repo path: | diff --git a/.github/workflows/ci.yaml b/.github/workflows/ci.yaml index d8f9f8365d..0875603368 100644 --- a/.github/workflows/ci.yaml +++ b/.github/workflows/ci.yaml @@ -159,7 +159,7 @@ jobs: SONAR_TOKEN: ${{ secrets.SONARCLOUD_TOKEN }} GITHUB_TOKEN: ${{ secrets.GITHUB_TOKEN }} - name: Archive build results - uses: actions/upload-artifact@bbbca2ddaa5d8feaa63e36b76fdaad77386f024f # v7.0.0 + uses: actions/upload-artifact@043fb46d1a93c77aae656e7c1c64a875d1fc6a0a # v7.0.1 if: always() with: name: ${{ github.job }} diff --git a/.github/workflows/repeat-test.yaml b/.github/workflows/repeat-test.yaml index 26fef26f75..9664de8dc0 100644 --- a/.github/workflows/repeat-test.yaml +++ b/.github/workflows/repeat-test.yaml @@ -125,7 +125,7 @@ jobs: run: dev-support/checks/_summary.sh target/unit/summary.txt if: ${{ !cancelled() }} - name: Archive build results - uses: actions/upload-artifact@bbbca2ddaa5d8feaa63e36b76fdaad77386f024f # v7.0.0 + uses: actions/upload-artifact@043fb46d1a93c77aae656e7c1c64a875d1fc6a0a # v7.0.1 if: ${{ failure() }} with: name: result-${{ github.run_number }}-${{ github.run_id }}-split-${{ matrix.split }} diff --git a/.github/workflows/vulnerability-check.yaml b/.github/workflows/vulnerability-check.yaml index 336332a845..307ef97185 100644 --- a/.github/workflows/vulnerability-check.yaml +++ b/.github/workflows/vulnerability-check.yaml @@ -62,7 +62,7 @@ jobs: utc_time: ${{ github.run_started_at }} - name: Upload Artifact - uses: actions/upload-artifact@bbbca2ddaa5d8feaa63e36b76fdaad77386f024f # v7.0.0 + uses: actions/upload-artifact@043fb46d1a93c77aae656e7c1c64a875d1fc6a0a # v7.0.1 with: name: vulnerability-check-result-${{ env.REPORT_DATE }} path: target/dependency-check-report.html From 85b18d30521b1fc63e321f91ef6c8aaadd5ba64f Mon Sep 17 00:00:00 2001 From: "dependabot[bot]" <49699333+dependabot[bot]@users.noreply.github.com> Date: Tue, 5 May 2026 10:56:53 +0200 Subject: [PATCH 327/397] RATIS-2519. Bump actions/cache to 5.0.5 (#1435) --- .github/workflows/check.yaml | 4 ++-- .github/workflows/ci.yaml | 2 +- .github/workflows/repeat-test.yaml | 2 +- 3 files changed, 4 insertions(+), 4 deletions(-) diff --git a/.github/workflows/check.yaml b/.github/workflows/check.yaml index 41f85a8f37..a10c02f3d5 100644 --- a/.github/workflows/check.yaml +++ b/.github/workflows/check.yaml @@ -112,7 +112,7 @@ jobs: - name: Create cache for Maven dependencies if: ${{ inputs.script == 'build' }} - uses: actions/cache@668228422ae6a00e4ad889ee87cd7109ec5666a7 # v5.0.4 + uses: actions/cache@27d5ce7f107fe9357f9df03efb73ab90386fccae # v5.0.5 with: path: | ~/.m2/repository/*/*/* @@ -123,7 +123,7 @@ jobs: - name: Restore cache for Maven dependencies if: ${{ inputs.script != 'build' }} - uses: actions/cache/restore@668228422ae6a00e4ad889ee87cd7109ec5666a7 # v5.0.4 + uses: actions/cache/restore@27d5ce7f107fe9357f9df03efb73ab90386fccae # v5.0.5 with: path: | ~/.m2/repository/*/*/* diff --git a/.github/workflows/ci.yaml b/.github/workflows/ci.yaml index 0875603368..a00a07955f 100644 --- a/.github/workflows/ci.yaml +++ b/.github/workflows/ci.yaml @@ -129,7 +129,7 @@ jobs: fetch-depth: 0 persist-credentials: false - name: Cache for maven dependencies - uses: actions/cache/restore@668228422ae6a00e4ad889ee87cd7109ec5666a7 # v5.0.4 + uses: actions/cache/restore@27d5ce7f107fe9357f9df03efb73ab90386fccae # v5.0.5 with: path: | ~/.m2/repository diff --git a/.github/workflows/repeat-test.yaml b/.github/workflows/repeat-test.yaml index 9664de8dc0..4bfeebe7f1 100644 --- a/.github/workflows/repeat-test.yaml +++ b/.github/workflows/repeat-test.yaml @@ -103,7 +103,7 @@ jobs: persist-credentials: false ref: ${{ needs.prepare.outputs.ref }} - name: Cache for maven dependencies - uses: actions/cache@668228422ae6a00e4ad889ee87cd7109ec5666a7 # v5.0.4 + uses: actions/cache@27d5ce7f107fe9357f9df03efb73ab90386fccae # v5.0.5 with: path: | ~/.m2/repository From e1cdcd4d9fff680d01e77e6b1cfb6b9fa258a951 Mon Sep 17 00:00:00 2001 From: Duong Nguyen Date: Fri, 12 Jan 2024 19:00:53 -0800 Subject: [PATCH 328/397] RATIS-1925. Support Zero-Copy in GrpcClientProtocolService (#1007) (cherry picked from commit 84285d3eadca2f734157d2970fc2c5cce02331e1) --- .../ratis/protocol/RaftClientRequest.java | 10 +- .../ratis/util/ReferenceCountedObject.java | 21 + .../java/org/apache/ratis/grpc/GrpcUtil.java | 56 ++- .../ratis/grpc/metrics/ZeroCopyMetrics.java | 58 +++ .../server/GrpcClientProtocolService.java | 135 +++--- .../apache/ratis/grpc/server/GrpcService.java | 386 ++++++++++++++++++ .../grpc/util/ZeroCopyMessageMarshaller.java | 8 +- .../ratis/server/impl/LeaderStateImpl.java | 12 +- .../server/impl/MessageStreamRequests.java | 45 +- .../ratis/server/impl/RaftServerImpl.java | 7 +- .../impl/SimpleStateMachine4Testing.java | 2 +- .../grpc/util/GrpcZeroCopyTestServer.java | 4 +- 12 files changed, 635 insertions(+), 109 deletions(-) create mode 100644 ratis-grpc/src/main/java/org/apache/ratis/grpc/metrics/ZeroCopyMetrics.java create mode 100644 ratis-grpc/src/main/java/org/apache/ratis/grpc/server/GrpcService.java diff --git a/ratis-common/src/main/java/org/apache/ratis/protocol/RaftClientRequest.java b/ratis-common/src/main/java/org/apache/ratis/protocol/RaftClientRequest.java index b04402fe15..85ede62e8c 100644 --- a/ratis-common/src/main/java/org/apache/ratis/protocol/RaftClientRequest.java +++ b/ratis-common/src/main/java/org/apache/ratis/protocol/RaftClientRequest.java @@ -488,7 +488,13 @@ public SpanContextProto getSpanContext() { @Override public String toString() { - return super.toString() + ", seq=" + ProtoUtils.toString(slidingWindowEntry) + ", " - + type + ", " + getMessage(); + return toStringShort() + ", " + getMessage(); + } + + /** + * @return a short string which does not include {@link #message}. + */ + public String toStringShort() { + return super.toString() + ", seq=" + ProtoUtils.toString(slidingWindowEntry) + ", " + type; } } diff --git a/ratis-common/src/main/java/org/apache/ratis/util/ReferenceCountedObject.java b/ratis-common/src/main/java/org/apache/ratis/util/ReferenceCountedObject.java index 0dd378dc01..7214d34e14 100644 --- a/ratis-common/src/main/java/org/apache/ratis/util/ReferenceCountedObject.java +++ b/ratis-common/src/main/java/org/apache/ratis/util/ReferenceCountedObject.java @@ -19,6 +19,7 @@ import org.apache.ratis.util.function.UncheckedAutoCloseableSupplier; +import java.util.Collection; import java.util.Objects; import java.util.concurrent.atomic.AtomicBoolean; import java.util.concurrent.atomic.AtomicInteger; @@ -101,6 +102,26 @@ static ReferenceCountedObject wrap(V value) { return wrap(value, () -> {}, ignored -> {}); } + static ReferenceCountedObject delegateFrom(Collection> fromRefs, V value) { + return new ReferenceCountedObject() { + @Override + public V get() { + return value; + } + + @Override + public V retain() { + fromRefs.forEach(ReferenceCountedObject::retain); + return value; + } + + @Override + public boolean release() { + return fromRefs.stream().map(ReferenceCountedObject::release).allMatch(r -> r); + } + }; + } + /** * Wrap the given value as a {@link ReferenceCountedObject}. * diff --git a/ratis-grpc/src/main/java/org/apache/ratis/grpc/GrpcUtil.java b/ratis-grpc/src/main/java/org/apache/ratis/grpc/GrpcUtil.java index df076875bf..b4d9cd6a5e 100644 --- a/ratis-grpc/src/main/java/org/apache/ratis/grpc/GrpcUtil.java +++ b/ratis-grpc/src/main/java/org/apache/ratis/grpc/GrpcUtil.java @@ -24,8 +24,12 @@ import org.apache.ratis.security.TlsConf.CertificatesConf; import org.apache.ratis.security.TlsConf.PrivateKeyConf; import org.apache.ratis.security.TlsConf.KeyManagerConf; +import org.apache.ratis.thirdparty.com.google.protobuf.MessageLite; import org.apache.ratis.thirdparty.io.grpc.ManagedChannel; import org.apache.ratis.thirdparty.io.grpc.Metadata; +import org.apache.ratis.thirdparty.io.grpc.MethodDescriptor; +import org.apache.ratis.thirdparty.io.grpc.ServerCallHandler; +import org.apache.ratis.thirdparty.io.grpc.ServerServiceDefinition; import org.apache.ratis.thirdparty.io.grpc.Status; import org.apache.ratis.thirdparty.io.grpc.StatusRuntimeException; import org.apache.ratis.thirdparty.io.grpc.netty.GrpcSslContexts; @@ -306,37 +310,25 @@ static void setKeyManager(SslContextBuilder b, KeyManagerConf keyManagerConfig) } } - static SslContext buildSslContextForServer(GrpcTlsConfig tlsConf) { - if (tlsConf == null) { - return null; - } - SslContextBuilder b = initSslContextBuilderForServer(tlsConf.getKeyManager()); - if (tlsConf.getMtlsEnabled()) { - b.clientAuth(ClientAuth.REQUIRE); - setTrustManager(b, tlsConf.getTrustManager()); - } - b = GrpcSslContexts.configure(b, OPENSSL); - try { - return b.build(); - } catch (Exception e) { - throw new IllegalArgumentException("Failed to buildSslContextForServer from tlsConfig " + tlsConf, e); - } - } - - static SslContext buildSslContextForClient(GrpcTlsConfig tlsConf) { - if (tlsConf == null) { - return null; - } - - final SslContextBuilder b = GrpcSslContexts.forClient(); - setTrustManager(b, tlsConf.getTrustManager()); - if (tlsConf.getMtlsEnabled()) { - setKeyManager(b, tlsConf.getKeyManager()); - } - try { - return b.build(); - } catch (SSLException e) { - throw new IllegalArgumentException("Failed to buildSslContextForClient from tlsConfig " + tlsConf, e); - } + /** + * Used to add a method to Service definition with a custom request marshaller. + * + * @param orig original service definition. + * @param newServiceBuilder builder of the new service definition. + * @param origMethod the original method definition. + * @param customMarshaller custom marshaller to be set for the method. + * @param + * @param + */ + static void addMethodWithCustomMarshaller( + ServerServiceDefinition orig, ServerServiceDefinition.Builder newServiceBuilder, + MethodDescriptor origMethod, MethodDescriptor.PrototypeMarshaller customMarshaller) { + MethodDescriptor newMethod = origMethod.toBuilder() + .setRequestMarshaller(customMarshaller) + .build(); + @SuppressWarnings("unchecked") + ServerCallHandler serverCallHandler = + (ServerCallHandler) orig.getMethod(newMethod.getFullMethodName()).getServerCallHandler(); + newServiceBuilder.addMethod(newMethod, serverCallHandler); } } diff --git a/ratis-grpc/src/main/java/org/apache/ratis/grpc/metrics/ZeroCopyMetrics.java b/ratis-grpc/src/main/java/org/apache/ratis/grpc/metrics/ZeroCopyMetrics.java new file mode 100644 index 0000000000..20da4ee634 --- /dev/null +++ b/ratis-grpc/src/main/java/org/apache/ratis/grpc/metrics/ZeroCopyMetrics.java @@ -0,0 +1,58 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + *

+ * http://www.apache.org/licenses/LICENSE-2.0 + *

+ * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.ratis.grpc.metrics; + +import org.apache.ratis.metrics.LongCounter; +import org.apache.ratis.metrics.MetricRegistryInfo; +import org.apache.ratis.metrics.RatisMetricRegistry; +import org.apache.ratis.metrics.RatisMetrics; +import org.apache.ratis.thirdparty.com.google.protobuf.AbstractMessage; + +public class ZeroCopyMetrics extends RatisMetrics { + private static final String RATIS_GRPC_METRICS_APP_NAME = "ratis_grpc"; + private static final String RATIS_GRPC_METRICS_COMP_NAME = "zero_copy"; + private static final String RATIS_GRPC_METRICS_DESC = "Metrics for Ratis Grpc Zero copy"; + + private final LongCounter zeroCopyMessages = getRegistry().counter("num_zero_copy_messages"); + private final LongCounter nonZeroCopyMessages = getRegistry().counter("num_non_zero_copy_messages"); + private final LongCounter releasedMessages = getRegistry().counter("num_released_messages"); + + public ZeroCopyMetrics() { + super(createRegistry()); + } + + private static RatisMetricRegistry createRegistry() { + return create(new MetricRegistryInfo("", + RATIS_GRPC_METRICS_APP_NAME, + RATIS_GRPC_METRICS_COMP_NAME, RATIS_GRPC_METRICS_DESC)); + } + + + public void onZeroCopyMessage(AbstractMessage ignored) { + zeroCopyMessages.inc(); + } + + public void onNonZeroCopyMessage(AbstractMessage ignored) { + nonZeroCopyMessages.inc(); + } + + public void onReleasedMessage(AbstractMessage ignored) { + releasedMessages.inc(); + } + +} \ No newline at end of file diff --git a/ratis-grpc/src/main/java/org/apache/ratis/grpc/server/GrpcClientProtocolService.java b/ratis-grpc/src/main/java/org/apache/ratis/grpc/server/GrpcClientProtocolService.java index 1da3587e91..e8de4def04 100644 --- a/ratis-grpc/src/main/java/org/apache/ratis/grpc/server/GrpcClientProtocolService.java +++ b/ratis-grpc/src/main/java/org/apache/ratis/grpc/server/GrpcClientProtocolService.java @@ -19,10 +19,13 @@ import org.apache.ratis.client.impl.ClientProtoUtils; import org.apache.ratis.grpc.GrpcUtil; +import org.apache.ratis.grpc.metrics.ZeroCopyMetrics; +import org.apache.ratis.grpc.util.ZeroCopyMessageMarshaller; import org.apache.ratis.protocol.*; import org.apache.ratis.protocol.exceptions.AlreadyClosedException; import org.apache.ratis.protocol.exceptions.GroupMismatchException; import org.apache.ratis.protocol.exceptions.RaftException; +import org.apache.ratis.thirdparty.io.grpc.ServerServiceDefinition; import org.apache.ratis.thirdparty.io.grpc.stub.StreamObserver; import org.apache.ratis.proto.RaftProtos.RaftClientReplyProto; import org.apache.ratis.proto.RaftProtos.RaftClientRequestProto; @@ -30,16 +33,15 @@ import org.apache.ratis.util.CollectionUtils; import org.apache.ratis.util.JavaUtils; import org.apache.ratis.util.Preconditions; +import org.apache.ratis.util.ReferenceCountedObject; import org.apache.ratis.util.SlidingWindow; import org.slf4j.Logger; import org.slf4j.LoggerFactory; -import java.io.IOException; import java.util.HashMap; import java.util.Iterator; import java.util.Map; import java.util.concurrent.CompletableFuture; -import java.util.concurrent.CompletionException; import java.util.concurrent.ConcurrentHashMap; import java.util.concurrent.ExecutorService; import java.util.concurrent.atomic.AtomicBoolean; @@ -48,15 +50,21 @@ import java.util.function.Consumer; import java.util.function.Supplier; +import static org.apache.ratis.grpc.GrpcUtil.addMethodWithCustomMarshaller; +import static org.apache.ratis.proto.grpc.RaftClientProtocolServiceGrpc.getOrderedMethod; +import static org.apache.ratis.proto.grpc.RaftClientProtocolServiceGrpc.getUnorderedMethod; + class GrpcClientProtocolService extends RaftClientProtocolServiceImplBase { private static final Logger LOG = LoggerFactory.getLogger(GrpcClientProtocolService.class); private static class PendingOrderedRequest implements SlidingWindow.ServerSideRequest { + private final ReferenceCountedObject requestRef; private final RaftClientRequest request; private final AtomicReference reply = new AtomicReference<>(); - PendingOrderedRequest(RaftClientRequest request) { - this.request = request; + PendingOrderedRequest(ReferenceCountedObject requestRef) { + this.requestRef = requestRef; + this.request = requestRef != null ? requestRef.get() : null; } @Override @@ -76,15 +84,16 @@ public boolean hasReply() { @Override public void setReply(RaftClientReply r) { final boolean set = reply.compareAndSet(null, r); - Preconditions.assertTrue(set, () -> "Reply is already set: request=" + request + ", reply=" + reply); + Preconditions.assertTrue(set, () -> "Reply is already set: request=" + + request.toStringShort() + ", reply=" + reply); } RaftClientReply getReply() { return reply.get(); } - RaftClientRequest getRequest() { - return request; + ReferenceCountedObject getRequestRef() { + return requestRef; } @Override @@ -135,18 +144,31 @@ void closeAllExisting(RaftGroupId groupId) { private final ExecutorService executor; private final OrderedStreamObservers orderedStreamObservers = new OrderedStreamObservers(); + private final ZeroCopyMessageMarshaller zeroCopyRequestMarshaller; GrpcClientProtocolService(Supplier idSupplier, RaftClientAsynchronousProtocol protocol, - ExecutorService executor) { + ExecutorService executor, ZeroCopyMetrics zeroCopyMetrics) { this.idSupplier = idSupplier; this.protocol = protocol; this.executor = executor; + this.zeroCopyRequestMarshaller = new ZeroCopyMessageMarshaller<>(RaftClientRequestProto.getDefaultInstance(), + zeroCopyMetrics::onZeroCopyMessage, zeroCopyMetrics::onNonZeroCopyMessage, zeroCopyMetrics::onReleasedMessage); } RaftPeerId getId() { return idSupplier.get(); } + ServerServiceDefinition bindServiceWithZeroCopy() { + ServerServiceDefinition orig = super.bindService(); + ServerServiceDefinition.Builder builder = ServerServiceDefinition.builder(orig.getServiceDescriptor().getName()); + + addMethodWithCustomMarshaller(orig, builder, getOrderedMethod(), zeroCopyRequestMarshaller); + addMethodWithCustomMarshaller(orig, builder, getUnorderedMethod(), zeroCopyRequestMarshaller); + + return builder.build(); + } + @Override public StreamObserver ordered(StreamObserver responseObserver) { final OrderedRequestStreamObserver so = new OrderedRequestStreamObserver(responseObserver); @@ -220,31 +242,38 @@ boolean isClosed() { return isClosed.get(); } - CompletableFuture processClientRequest(RaftClientRequest request, Consumer replyHandler) { - try { - final String errMsg = LOG.isDebugEnabled() ? "processClientRequest for " + request : ""; - return protocol.submitClientRequestAsync(request - ).thenAcceptAsync(replyHandler, executor - ).exceptionally(exception -> { - // TODO: the exception may be from either raft or state machine. - // Currently we skip all the following responses when getting an - // exception from the state machine. - responseError(exception, () -> errMsg); - return null; - }); - } catch (IOException e) { - throw new CompletionException("Failed processClientRequest for " + request + " in " + name, e); - } + CompletableFuture processClientRequest(ReferenceCountedObject requestRef, + Consumer replyHandler) { + final String errMsg = LOG.isDebugEnabled() ? "processClientRequest for " + requestRef.get() : ""; + return protocol.submitClientRequestAsync(requestRef + ).thenAcceptAsync(replyHandler, executor + ).exceptionally(exception -> { + // TODO: the exception may be from either raft or state machine. + // Currently we skip all the following responses when getting an + // exception from the state machine. + responseError(exception, () -> errMsg); + return null; + }); } - abstract void processClientRequest(RaftClientRequest request); + abstract void processClientRequest(ReferenceCountedObject requestRef); @Override public void onNext(RaftClientRequestProto request) { + ReferenceCountedObject requestRef = null; try { final RaftClientRequest r = ClientProtoUtils.toRaftClientRequest(request); - processClientRequest(r); + requestRef = ReferenceCountedObject.wrap(r, () -> {}, released -> { + if (released) { + zeroCopyRequestMarshaller.release(request); + } + }); + + processClientRequest(requestRef); } catch (Exception e) { + if (requestRef == null) { + zeroCopyRequestMarshaller.release(request); + } responseError(e, () -> "onNext for " + ClientProtoUtils.toString(request) + " in " + name); } } @@ -278,15 +307,18 @@ private class UnorderedRequestStreamObserver extends RequestStreamObserver { } @Override - void processClientRequest(RaftClientRequest request) { - final CompletableFuture f = processClientRequest(request, reply -> { + void processClientRequest(ReferenceCountedObject requestRef) { + final RaftClientRequest request = requestRef.retain(); + final long callId = request.getCallId(); + + final CompletableFuture f = processClientRequest(requestRef, reply -> { if (!reply.isSuccess()) { - LOG.info("Failed " + request + ", reply=" + reply); + LOG.info("Failed {}, reply={}", request, reply); } final RaftClientReplyProto proto = ClientProtoUtils.toRaftClientReplyProto(reply); responseNext(proto); - }); - final long callId = request.getCallId(); + }).whenComplete((r, e) -> requestRef.release()); + put(callId, f); f.thenAccept(dummy -> remove(callId)); } @@ -329,32 +361,35 @@ RaftGroupId getGroupId() { void processClientRequest(PendingOrderedRequest pending) { final long seq = pending.getSeqNum(); - processClientRequest(pending.getRequest(), + processClientRequest(pending.getRequestRef(), reply -> slidingWindow.receiveReply(seq, reply, this::sendReply)); } @Override - void processClientRequest(RaftClientRequest r) { - if (isClosed()) { - final AlreadyClosedException exception = new AlreadyClosedException(getName() + ": the stream is closed"); - responseError(exception, () -> "processClientRequest (stream already closed) for " + r); - return; - } + void processClientRequest(ReferenceCountedObject requestRef) { + final RaftClientRequest request = requestRef.retain(); + try { + if (isClosed()) { + final AlreadyClosedException exception = new AlreadyClosedException(getName() + ": the stream is closed"); + responseError(exception, () -> "processClientRequest (stream already closed) for " + request); + } - final RaftGroupId requestGroupId = r.getRaftGroupId(); - // use the group id in the first request as the group id of this observer - final RaftGroupId updated = groupId.updateAndGet(g -> g != null ? g: requestGroupId); - final PendingOrderedRequest pending = new PendingOrderedRequest(r); - - if (!requestGroupId.equals(updated)) { - final GroupMismatchException exception = new GroupMismatchException(getId() - + ": The group (" + requestGroupId + ") of " + r.getClientId() - + " does not match the group (" + updated + ") of the " + JavaUtils.getClassSimpleName(getClass())); - responseError(exception, () -> "processClientRequest (Group mismatched) for " + r); - return; + final RaftGroupId requestGroupId = request.getRaftGroupId(); + // use the group id in the first request as the group id of this observer + final RaftGroupId updated = groupId.updateAndGet(g -> g != null ? g : requestGroupId); + final PendingOrderedRequest pending = new PendingOrderedRequest(requestRef); + + if (!requestGroupId.equals(updated)) { + final GroupMismatchException exception = new GroupMismatchException(getId() + + ": The group (" + requestGroupId + ") of " + request.getClientId() + + " does not match the group (" + updated + ") of the " + JavaUtils.getClassSimpleName(getClass())); + responseError(exception, () -> "processClientRequest (Group mismatched) for " + request); + return; + } + slidingWindow.receivedRequest(pending, this::processClientRequest); + } finally { + requestRef.release(); } - - slidingWindow.receivedRequest(pending, this::processClientRequest); } private void sendReply(PendingOrderedRequest ready) { diff --git a/ratis-grpc/src/main/java/org/apache/ratis/grpc/server/GrpcService.java b/ratis-grpc/src/main/java/org/apache/ratis/grpc/server/GrpcService.java new file mode 100644 index 0000000000..d89afd565e --- /dev/null +++ b/ratis-grpc/src/main/java/org/apache/ratis/grpc/server/GrpcService.java @@ -0,0 +1,386 @@ +/* + * 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.ratis.grpc.server; + +import org.apache.ratis.conf.RaftProperties; +import org.apache.ratis.grpc.GrpcConfigKeys; +import org.apache.ratis.grpc.GrpcTlsConfig; +import org.apache.ratis.grpc.GrpcUtil; +import org.apache.ratis.grpc.metrics.ZeroCopyMetrics; +import org.apache.ratis.grpc.metrics.intercept.server.MetricServerInterceptor; +import org.apache.ratis.protocol.RaftGroupId; +import org.apache.ratis.protocol.RaftPeerId; +import org.apache.ratis.rpc.SupportedRpcType; +import org.apache.ratis.server.RaftServer; +import org.apache.ratis.server.RaftServerConfigKeys; +import org.apache.ratis.server.RaftServerRpcWithProxy; +import org.apache.ratis.server.protocol.RaftServerAsynchronousProtocol; +import org.apache.ratis.thirdparty.io.grpc.ServerInterceptors; +import org.apache.ratis.thirdparty.io.grpc.netty.GrpcSslContexts; +import org.apache.ratis.thirdparty.io.grpc.netty.NettyServerBuilder; +import org.apache.ratis.thirdparty.io.grpc.Server; +import org.apache.ratis.thirdparty.io.grpc.stub.StreamObserver; +import org.apache.ratis.thirdparty.io.netty.channel.ChannelOption; +import org.apache.ratis.thirdparty.io.netty.handler.ssl.ClientAuth; +import org.apache.ratis.thirdparty.io.netty.handler.ssl.SslContextBuilder; + +import org.apache.ratis.proto.RaftProtos.*; +import org.apache.ratis.util.*; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.io.IOException; +import java.net.InetSocketAddress; +import java.util.HashMap; +import java.util.Map; +import java.util.concurrent.CompletableFuture; +import java.util.concurrent.ExecutorService; +import java.util.function.Supplier; + +import static org.apache.ratis.thirdparty.io.netty.handler.ssl.SslProvider.OPENSSL; + +/** A grpc implementation of {@link org.apache.ratis.server.RaftServerRpc}. */ +public final class GrpcService extends RaftServerRpcWithProxy> { + static final Logger LOG = LoggerFactory.getLogger(GrpcService.class); + public static final String GRPC_SEND_SERVER_REQUEST = + JavaUtils.getClassSimpleName(GrpcService.class) + ".sendRequest"; + + class AsyncService implements RaftServerAsynchronousProtocol { + + @Override + public CompletableFuture appendEntriesAsync(AppendEntriesRequestProto request) + throws IOException { + throw new UnsupportedOperationException("This method is not supported"); + } + + @Override + public CompletableFuture readIndexAsync(ReadIndexRequestProto request) throws IOException { + CodeInjectionForTesting.execute(GRPC_SEND_SERVER_REQUEST, getId(), null, request); + + final CompletableFuture f = new CompletableFuture<>(); + final StreamObserver s = new StreamObserver() { + @Override + public void onNext(ReadIndexReplyProto reply) { + f.complete(reply); + } + + @Override + public void onError(Throwable throwable) { + f.completeExceptionally(throwable); + } + + @Override + public void onCompleted() { + } + }; + + final RaftPeerId target = RaftPeerId.valueOf(request.getServerRequest().getReplyId()); + getProxies().getProxy(target).readIndex(request, s); + return f; + } + } + + public static final class Builder { + private RaftServer server; + private GrpcTlsConfig tlsConfig; + private GrpcTlsConfig adminTlsConfig; + private GrpcTlsConfig clientTlsConfig; + private GrpcTlsConfig serverTlsConfig; + + private Builder() {} + + public Builder setServer(RaftServer raftServer) { + this.server = raftServer; + return this; + } + + public GrpcService build() { + return new GrpcService(server, adminTlsConfig, clientTlsConfig, serverTlsConfig); + } + + public Builder setTlsConfig(GrpcTlsConfig tlsConfig) { + this.tlsConfig = tlsConfig; + return this; + } + + public Builder setAdminTlsConfig(GrpcTlsConfig config) { + this.adminTlsConfig = config; + return this; + } + + public Builder setClientTlsConfig(GrpcTlsConfig config) { + this.clientTlsConfig = config; + return this; + } + + public Builder setServerTlsConfig(GrpcTlsConfig config) { + this.serverTlsConfig = config; + return this; + } + + public GrpcTlsConfig getTlsConfig() { + return tlsConfig; + } + } + + public static Builder newBuilder() { + return new Builder(); + } + + private final Map servers = new HashMap<>(); + private final Supplier addressSupplier; + private final Supplier clientServerAddressSupplier; + private final Supplier adminServerAddressSupplier; + + private final AsyncService asyncService = new AsyncService(); + + private final ExecutorService executor; + private final GrpcClientProtocolService clientProtocolService; + + private final MetricServerInterceptor serverInterceptor; + private final ZeroCopyMetrics zeroCopyMetrics; + + public MetricServerInterceptor getServerInterceptor() { + return serverInterceptor; + } + + private GrpcService(RaftServer server, + GrpcTlsConfig adminTlsConfig, GrpcTlsConfig clientTlsConfig, GrpcTlsConfig serverTlsConfig) { + this(server, server::getId, + GrpcConfigKeys.Admin.host(server.getProperties()), + GrpcConfigKeys.Admin.port(server.getProperties()), + adminTlsConfig, + GrpcConfigKeys.Client.host(server.getProperties()), + GrpcConfigKeys.Client.port(server.getProperties()), + clientTlsConfig, + GrpcConfigKeys.Server.host(server.getProperties()), + GrpcConfigKeys.Server.port(server.getProperties()), + serverTlsConfig, + GrpcConfigKeys.messageSizeMax(server.getProperties(), LOG::info), + RaftServerConfigKeys.Log.Appender.bufferByteLimit(server.getProperties()), + GrpcConfigKeys.flowControlWindow(server.getProperties(), LOG::info), + RaftServerConfigKeys.Rpc.requestTimeout(server.getProperties()), + GrpcConfigKeys.Server.heartbeatChannel(server.getProperties())); + } + + @SuppressWarnings("checkstyle:ParameterNumber") // private constructor + private GrpcService(RaftServer raftServer, Supplier idSupplier, + String adminHost, int adminPort, GrpcTlsConfig adminTlsConfig, + String clientHost, int clientPort, GrpcTlsConfig clientTlsConfig, + String serverHost, int serverPort, GrpcTlsConfig serverTlsConfig, + SizeInBytes grpcMessageSizeMax, SizeInBytes appenderBufferSize, + SizeInBytes flowControlWindow,TimeDuration requestTimeoutDuration, + boolean useSeparateHBChannel) { + super(idSupplier, id -> new PeerProxyMap<>(id.toString(), + p -> new GrpcServerProtocolClient(p, flowControlWindow.getSizeInt(), + requestTimeoutDuration, serverTlsConfig, useSeparateHBChannel))); + if (appenderBufferSize.getSize() > grpcMessageSizeMax.getSize()) { + throw new IllegalArgumentException("Illegal configuration: " + + RaftServerConfigKeys.Log.Appender.BUFFER_BYTE_LIMIT_KEY + " = " + appenderBufferSize + + " > " + GrpcConfigKeys.MESSAGE_SIZE_MAX_KEY + " = " + grpcMessageSizeMax); + } + + final RaftProperties properties = raftServer.getProperties(); + this.executor = ConcurrentUtils.newThreadPoolWithMax( + GrpcConfigKeys.Server.asyncRequestThreadPoolCached(properties), + GrpcConfigKeys.Server.asyncRequestThreadPoolSize(properties), + getId() + "-request-"); + this.zeroCopyMetrics = new ZeroCopyMetrics(); + this.clientProtocolService = new GrpcClientProtocolService(idSupplier, raftServer, executor, zeroCopyMetrics); + + this.serverInterceptor = new MetricServerInterceptor( + idSupplier, + JavaUtils.getClassSimpleName(getClass()) + "_" + serverPort + ); + + final boolean separateAdminServer = adminPort != serverPort && adminPort > 0; + final boolean separateClientServer = clientPort != serverPort && clientPort > 0; + + final NettyServerBuilder serverBuilder = + startBuildingNettyServer(serverHost, serverPort, serverTlsConfig, grpcMessageSizeMax, flowControlWindow); + serverBuilder.addService(ServerInterceptors.intercept( + new GrpcServerProtocolService(idSupplier, raftServer), serverInterceptor)); + if (!separateAdminServer) { + addAdminService(raftServer, serverBuilder); + } + if (!separateClientServer) { + addClientService(serverBuilder); + } + + final Server server = serverBuilder.build(); + servers.put(GrpcServerProtocolService.class.getSimpleName(), server); + addressSupplier = newAddressSupplier(serverPort, server); + + if (separateAdminServer) { + final NettyServerBuilder builder = + startBuildingNettyServer(adminHost, adminPort, adminTlsConfig, grpcMessageSizeMax, flowControlWindow); + addAdminService(raftServer, builder); + final Server adminServer = builder.build(); + servers.put(GrpcAdminProtocolService.class.getName(), adminServer); + adminServerAddressSupplier = newAddressSupplier(adminPort, adminServer); + } else { + adminServerAddressSupplier = addressSupplier; + } + + if (separateClientServer) { + final NettyServerBuilder builder = + startBuildingNettyServer(clientHost, clientPort, clientTlsConfig, grpcMessageSizeMax, flowControlWindow); + addClientService(builder); + final Server clientServer = builder.build(); + servers.put(GrpcClientProtocolService.class.getName(), clientServer); + clientServerAddressSupplier = newAddressSupplier(clientPort, clientServer); + } else { + clientServerAddressSupplier = addressSupplier; + } + } + + private MemoizedSupplier newAddressSupplier(int port, Server server) { + return JavaUtils.memoize(() -> new InetSocketAddress(port != 0 ? port : server.getPort())); + } + + private void addClientService(NettyServerBuilder builder) { + builder.addService(ServerInterceptors.intercept(clientProtocolService.bindServiceWithZeroCopy(), + serverInterceptor)); + } + + private void addAdminService(RaftServer raftServer, NettyServerBuilder nettyServerBuilder) { + nettyServerBuilder.addService(ServerInterceptors.intercept( + new GrpcAdminProtocolService(raftServer), + serverInterceptor)); + } + + private static NettyServerBuilder startBuildingNettyServer(String hostname, int port, GrpcTlsConfig tlsConfig, + SizeInBytes grpcMessageSizeMax, SizeInBytes flowControlWindow) { + InetSocketAddress address = hostname == null || hostname.isEmpty() ? + new InetSocketAddress(port) : new InetSocketAddress(hostname, port); + NettyServerBuilder nettyServerBuilder = NettyServerBuilder.forAddress(address) + .withChildOption(ChannelOption.SO_REUSEADDR, true) + .maxInboundMessageSize(grpcMessageSizeMax.getSizeInt()) + .flowControlWindow(flowControlWindow.getSizeInt()); + + if (tlsConfig != null) { + SslContextBuilder sslContextBuilder = GrpcUtil.initSslContextBuilderForServer(tlsConfig.getKeyManager()); + if (tlsConfig.getMtlsEnabled()) { + sslContextBuilder.clientAuth(ClientAuth.REQUIRE); + GrpcUtil.setTrustManager(sslContextBuilder, tlsConfig.getTrustManager()); + } + sslContextBuilder = GrpcSslContexts.configure(sslContextBuilder, OPENSSL); + try { + nettyServerBuilder.sslContext(sslContextBuilder.build()); + } catch (Exception ex) { + throw new IllegalArgumentException("Failed to build SslContext, tlsConfig=" + tlsConfig, ex); + } + } + return nettyServerBuilder; + } + + @Override + public SupportedRpcType getRpcType() { + return SupportedRpcType.GRPC; + } + + @Override + public void startImpl() { + for (Server server : servers.values()) { + try { + server.start(); + } catch (IOException e) { + ExitUtils.terminate(1, "Failed to start Grpc server", e, LOG); + } + LOG.info("{}: {} started, listening on {}", + getId(), JavaUtils.getClassSimpleName(getClass()), server.getPort()); + } + } + + @Override + public void closeImpl() throws IOException { + for (Map.Entry server : servers.entrySet()) { + final String name = getId() + ": shutdown server " + server.getKey(); + LOG.info("{} now", name); + final Server s = server.getValue().shutdownNow(); + super.closeImpl(); + try { + s.awaitTermination(); + } catch (InterruptedException e) { + Thread.currentThread().interrupt(); + throw IOUtils.toInterruptedIOException(name + " failed", e); + } + LOG.info("{} successfully", name); + } + + serverInterceptor.close(); + ConcurrentUtils.shutdownAndWait(executor); + } + + @Override + public void notifyNotLeader(RaftGroupId groupId) { + clientProtocolService.closeAllOrderedRequestStreamObservers(groupId); + } + + @Override + public InetSocketAddress getInetSocketAddress() { + return addressSupplier.get(); + } + + @Override + public InetSocketAddress getClientServerAddress() { + return clientServerAddressSupplier.get(); + } + + @Override + public InetSocketAddress getAdminServerAddress() { + return adminServerAddressSupplier.get(); + } + + @Override + public RaftServerAsynchronousProtocol async() { + return asyncService; + } + + @Override + public AppendEntriesReplyProto appendEntries(AppendEntriesRequestProto request) { + throw new UnsupportedOperationException( + "Blocking " + JavaUtils.getCurrentStackTraceElement().getMethodName() + " call is not supported"); + } + + @Override + public InstallSnapshotReplyProto installSnapshot(InstallSnapshotRequestProto request) { + throw new UnsupportedOperationException( + "Blocking " + JavaUtils.getCurrentStackTraceElement().getMethodName() + " call is not supported"); + } + + @Override + public RequestVoteReplyProto requestVote(RequestVoteRequestProto request) + throws IOException { + CodeInjectionForTesting.execute(GRPC_SEND_SERVER_REQUEST, getId(), + null, request); + + final RaftPeerId target = RaftPeerId.valueOf(request.getServerRequest().getReplyId()); + return getProxies().getProxy(target).requestVote(request); + } + + @Override + public StartLeaderElectionReplyProto startLeaderElection(StartLeaderElectionRequestProto request) throws IOException { + CodeInjectionForTesting.execute(GRPC_SEND_SERVER_REQUEST, getId(), null, request); + + final RaftPeerId target = RaftPeerId.valueOf(request.getServerRequest().getReplyId()); + return getProxies().getProxy(target).startLeaderElection(request); + } + +} diff --git a/ratis-grpc/src/main/java/org/apache/ratis/grpc/util/ZeroCopyMessageMarshaller.java b/ratis-grpc/src/main/java/org/apache/ratis/grpc/util/ZeroCopyMessageMarshaller.java index 3fb3f067be..385ac592fe 100644 --- a/ratis-grpc/src/main/java/org/apache/ratis/grpc/util/ZeroCopyMessageMarshaller.java +++ b/ratis-grpc/src/main/java/org/apache/ratis/grpc/util/ZeroCopyMessageMarshaller.java @@ -62,12 +62,14 @@ public class ZeroCopyMessageMarshaller implements Prototy private final Consumer zeroCopyCount; private final Consumer nonZeroCopyCount; + private final Consumer releasedCount; public ZeroCopyMessageMarshaller(T defaultInstance) { - this(defaultInstance, m -> {}, m -> {}); + this(defaultInstance, m -> {}, m -> {}, m -> {}); } - public ZeroCopyMessageMarshaller(T defaultInstance, Consumer zeroCopyCount, Consumer nonZeroCopyCount) { + public ZeroCopyMessageMarshaller(T defaultInstance, Consumer zeroCopyCount, Consumer nonZeroCopyCount, + Consumer releasedCount) { this.name = JavaUtils.getClassSimpleName(defaultInstance.getClass()) + "-Marshaller"; @SuppressWarnings("unchecked") final Parser p = (Parser) defaultInstance.getParserForType(); @@ -76,6 +78,7 @@ public ZeroCopyMessageMarshaller(T defaultInstance, Consumer zeroCopyCount, C this.zeroCopyCount = zeroCopyCount; this.nonZeroCopyCount = nonZeroCopyCount; + this.releasedCount = releasedCount; } @Override @@ -124,6 +127,7 @@ public void release(T message) { } try { stream.close(); + releasedCount.accept(message); } catch (IOException e) { LOG.error(name + ": Failed to close stream.", e); } diff --git a/ratis-server/src/main/java/org/apache/ratis/server/impl/LeaderStateImpl.java b/ratis-server/src/main/java/org/apache/ratis/server/impl/LeaderStateImpl.java index 1c986ca638..b716781086 100644 --- a/ratis-server/src/main/java/org/apache/ratis/server/impl/LeaderStateImpl.java +++ b/ratis-server/src/main/java/org/apache/ratis/server/impl/LeaderStateImpl.java @@ -58,6 +58,7 @@ import org.apache.ratis.util.JavaUtils; import org.apache.ratis.util.MemoizedSupplier; import org.apache.ratis.util.Preconditions; +import org.apache.ratis.util.ReferenceCountedObject; import org.apache.ratis.util.TimeDuration; import org.apache.ratis.util.Timestamp; @@ -561,15 +562,16 @@ PendingRequest addPendingRequest(PendingRequests.Permit permit, RaftClientReques return pendingRequests.add(permit, request, entry); } - CompletableFuture streamAsync(RaftClientRequest request) { - return messageStreamRequests.streamAsync(request) + CompletableFuture streamAsync(ReferenceCountedObject requestRef) { + RaftClientRequest request = requestRef.get(); + return messageStreamRequests.streamAsync(requestRef) .thenApply(dummy -> server.newSuccessReply(request)) .exceptionally(e -> exception2RaftClientReply(request, e)); } - CompletableFuture streamEndOfRequestAsync(RaftClientRequest request) { - return messageStreamRequests.streamEndOfRequestAsync(request) - .thenApply(bytes -> RaftClientRequest.toWriteRequest(request, Message.valueOf(bytes))); + CompletableFuture> streamEndOfRequestAsync( + ReferenceCountedObject requestRef) { + return messageStreamRequests.streamEndOfRequestAsync(requestRef); } CompletableFuture addWatchRequest(RaftClientRequest request) { diff --git a/ratis-server/src/main/java/org/apache/ratis/server/impl/MessageStreamRequests.java b/ratis-server/src/main/java/org/apache/ratis/server/impl/MessageStreamRequests.java index ac81b348bb..c00c57b364 100644 --- a/ratis-server/src/main/java/org/apache/ratis/server/impl/MessageStreamRequests.java +++ b/ratis-server/src/main/java/org/apache/ratis/server/impl/MessageStreamRequests.java @@ -25,12 +25,15 @@ import org.apache.ratis.thirdparty.com.google.protobuf.ByteString; import org.apache.ratis.util.JavaUtils; import org.apache.ratis.util.Preconditions; +import org.apache.ratis.util.ReferenceCountedObject; import org.slf4j.Logger; import org.slf4j.LoggerFactory; +import java.util.HashMap; +import java.util.LinkedList; +import java.util.List; +import java.util.Map; import java.util.concurrent.CompletableFuture; -import java.util.concurrent.ConcurrentHashMap; -import java.util.concurrent.ConcurrentMap; class MessageStreamRequests { public static final Logger LOG = LoggerFactory.getLogger(MessageStreamRequests.class); @@ -39,12 +42,14 @@ private static class PendingStream { private final ClientInvocationId key; private long nextId = -1; private ByteString bytes = ByteString.EMPTY; + private final List> pendingRefs = new LinkedList<>(); PendingStream(ClientInvocationId key) { this.key = key; } - synchronized CompletableFuture append(long messageId, Message message) { + synchronized CompletableFuture append(long messageId, + ReferenceCountedObject requestRef) { if (nextId == -1) { nextId = messageId; } else if (messageId != nextId) { @@ -52,27 +57,38 @@ synchronized CompletableFuture append(long messageId, Message messag "Unexpected message id in " + key + ": messageId = " + messageId + " != nextId = " + nextId)); } nextId++; + final Message message = requestRef.retain().getMessage(); + pendingRefs.add(requestRef); bytes = bytes.concat(message.getContent()); return CompletableFuture.completedFuture(bytes); } - synchronized CompletableFuture getBytes(long messageId, Message message) { - return append(messageId, message); + synchronized CompletableFuture> getWriteRequest(long messageId, + ReferenceCountedObject requestRef) { + return append(messageId, requestRef) + .thenApply(appended -> RaftClientRequest.toWriteRequest(requestRef.get(), () -> appended)) + .thenApply(request -> ReferenceCountedObject.delegateFrom(pendingRefs, request)); + } + + synchronized void clear() { + pendingRefs.forEach(ReferenceCountedObject::release); + pendingRefs.clear(); } } static class StreamMap { - private final ConcurrentMap map = new ConcurrentHashMap<>(); + private final Map map = new HashMap<>(); - PendingStream computeIfAbsent(ClientInvocationId key) { + synchronized PendingStream computeIfAbsent(ClientInvocationId key) { return map.computeIfAbsent(key, PendingStream::new); } - PendingStream remove(ClientInvocationId key) { + synchronized PendingStream remove(ClientInvocationId key) { return map.remove(key); } - void clear() { + synchronized void clear() { + map.values().forEach(PendingStream::clear); map.clear(); } } @@ -84,15 +100,18 @@ void clear() { this.name = name + "-" + JavaUtils.getClassSimpleName(getClass()); } - CompletableFuture streamAsync(RaftClientRequest request) { + CompletableFuture streamAsync(ReferenceCountedObject requestRef) { + final RaftClientRequest request = requestRef.get(); final MessageStreamRequestTypeProto stream = request.getType().getMessageStream(); Preconditions.assertTrue(!stream.getEndOfRequest()); final ClientInvocationId key = ClientInvocationId.valueOf(request.getClientId(), stream.getStreamId()); final PendingStream pending = streams.computeIfAbsent(key); - return pending.append(stream.getMessageId(), request.getMessage()); + return pending.append(stream.getMessageId(), requestRef); } - CompletableFuture streamEndOfRequestAsync(RaftClientRequest request) { + CompletableFuture> streamEndOfRequestAsync( + ReferenceCountedObject requestRef) { + final RaftClientRequest request = requestRef.get(); final MessageStreamRequestTypeProto stream = request.getType().getMessageStream(); Preconditions.assertTrue(stream.getEndOfRequest()); final ClientInvocationId key = ClientInvocationId.valueOf(request.getClientId(), stream.getStreamId()); @@ -101,7 +120,7 @@ CompletableFuture streamEndOfRequestAsync(RaftClientRequest request) if (pending == null) { return JavaUtils.completeExceptionally(new StreamException(name + ": " + key + " not found")); } - return pending.getBytes(stream.getMessageId(), request.getMessage()); + return pending.getWriteRequest(stream.getMessageId(), requestRef); } void clear() { diff --git a/ratis-server/src/main/java/org/apache/ratis/server/impl/RaftServerImpl.java b/ratis-server/src/main/java/org/apache/ratis/server/impl/RaftServerImpl.java index 1c9cd3f658..32e223e009 100644 --- a/ratis-server/src/main/java/org/apache/ratis/server/impl/RaftServerImpl.java +++ b/ratis-server/src/main/java/org/apache/ratis/server/impl/RaftServerImpl.java @@ -1167,7 +1167,7 @@ private CompletableFuture messageStreamAsync(RaftClientRequest } if (request.getType().getMessageStream().getEndOfRequest()) { - final CompletableFuture f = streamEndOfRequestAsync(request); + final CompletableFuture> f = streamEndOfRequestAsync(requestRef); if (f.isCompletedExceptionally()) { return f.thenApply(r -> null); } @@ -1176,12 +1176,13 @@ private CompletableFuture messageStreamAsync(RaftClientRequest } return role.getLeaderState() - .map(ls -> ls.streamAsync(request)) + .map(ls -> ls.streamAsync(requestRef)) .orElseGet(() -> CompletableFuture.completedFuture( newExceptionReply(request, generateNotLeaderException()))); } - private CompletableFuture streamEndOfRequestAsync(RaftClientRequest request) { + private CompletableFuture> streamEndOfRequestAsync( + ReferenceCountedObject request) { return role.getLeaderState() .map(ls -> ls.streamEndOfRequestAsync(request)) .orElse(null); diff --git a/ratis-server/src/test/java/org/apache/ratis/statemachine/impl/SimpleStateMachine4Testing.java b/ratis-server/src/test/java/org/apache/ratis/statemachine/impl/SimpleStateMachine4Testing.java index 1ffbdbcb99..1e77c3d516 100644 --- a/ratis-server/src/test/java/org/apache/ratis/statemachine/impl/SimpleStateMachine4Testing.java +++ b/ratis-server/src/test/java/org/apache/ratis/statemachine/impl/SimpleStateMachine4Testing.java @@ -331,7 +331,7 @@ public CompletableFuture query(Message request) { final String string = request.getContent().toStringUtf8(); Exception exception; try { - LOG.info("query {}", string); + LOG.info("query {}, all available: {}", string, dataMap.keySet()); final LogEntryProto entry = dataMap.get(string); if (entry != null) { return CompletableFuture.completedFuture(Message.valueOf(entry.toByteString())); diff --git a/ratis-test/src/test/java/org/apache/ratis/grpc/util/GrpcZeroCopyTestServer.java b/ratis-test/src/test/java/org/apache/ratis/grpc/util/GrpcZeroCopyTestServer.java index ca2709270b..af7991a416 100644 --- a/ratis-test/src/test/java/org/apache/ratis/grpc/util/GrpcZeroCopyTestServer.java +++ b/ratis-test/src/test/java/org/apache/ratis/grpc/util/GrpcZeroCopyTestServer.java @@ -75,6 +75,7 @@ public synchronized String toString() { private final Count zeroCopyCount = new Count(); private final Count nonZeroCopyCount = new Count(); + private final Count releasedCount = new Count(); private final Server server; // Allow tests to disable release to validate leak detection. @@ -82,7 +83,8 @@ public synchronized String toString() { private final ZeroCopyMessageMarshaller marshaller = new ZeroCopyMessageMarshaller<>( BinaryRequest.getDefaultInstance(), zeroCopyCount::inc, - nonZeroCopyCount::inc); + nonZeroCopyCount::inc, + releasedCount::inc); GrpcZeroCopyTestServer(int port) { this(port, true); From 992155257d4609db0b63ff539edb93b48c2a3d5a Mon Sep 17 00:00:00 2001 From: Duong Nguyen Date: Thu, 18 Jan 2024 00:23:22 -0800 Subject: [PATCH 329/397] RATIS-1990. Refactor appendEntries processing to support reference count (#1011) (cherry picked from commit 5d4ec6933f538d0bf2c40483ad3d37173c82a1ca) --- .../server/GrpcServerProtocolService.java | 3 +- .../RaftServerAsynchronousProtocol.java | 31 ++++++++++++- .../server/raftlog/RaftLogSequentialOps.java | 21 ++++++++- .../ratis/server/impl/RaftServerImpl.java | 46 +++++++++---------- .../ratis/server/impl/RaftServerProxy.java | 15 ++++-- .../ratis/server/raftlog/RaftLogBase.java | 14 +++++- .../server/raftlog/memory/MemoryRaftLog.java | 35 ++++++++++---- .../raftlog/segmented/SegmentedRaftLog.java | 15 ++++-- .../raftlog/memory/MemoryRaftLogTest.java | 1 + .../raftlog/segmented/TestCacheEviction.java | 5 +- .../segmented/TestSegmentedRaftLog.java | 8 ++-- 11 files changed, 141 insertions(+), 53 deletions(-) diff --git a/ratis-grpc/src/main/java/org/apache/ratis/grpc/server/GrpcServerProtocolService.java b/ratis-grpc/src/main/java/org/apache/ratis/grpc/server/GrpcServerProtocolService.java index a13e74b89d..7c5ce9ae62 100644 --- a/ratis-grpc/src/main/java/org/apache/ratis/grpc/server/GrpcServerProtocolService.java +++ b/ratis-grpc/src/main/java/org/apache/ratis/grpc/server/GrpcServerProtocolService.java @@ -33,6 +33,7 @@ import org.apache.ratis.util.BatchLogger; import org.apache.ratis.util.MemoizedSupplier; import org.apache.ratis.util.ProtoUtils; +import org.apache.ratis.util.ReferenceCountedObject; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -245,7 +246,7 @@ public StreamObserver appendEntries( RaftServerProtocol.Op.APPEND_ENTRIES, responseObserver) { @Override CompletableFuture process(AppendEntriesRequestProto request) throws IOException { - return server.appendEntriesAsync(request); + return server.appendEntriesAsync(ReferenceCountedObject.wrap(request)); } @Override diff --git a/ratis-server-api/src/main/java/org/apache/ratis/server/protocol/RaftServerAsynchronousProtocol.java b/ratis-server-api/src/main/java/org/apache/ratis/server/protocol/RaftServerAsynchronousProtocol.java index 8a904069ba..1244e72545 100644 --- a/ratis-server-api/src/main/java/org/apache/ratis/server/protocol/RaftServerAsynchronousProtocol.java +++ b/ratis-server-api/src/main/java/org/apache/ratis/server/protocol/RaftServerAsynchronousProtocol.java @@ -22,14 +22,41 @@ import org.apache.ratis.proto.RaftProtos.ReadIndexReplyProto; import org.apache.ratis.proto.RaftProtos.AppendEntriesReplyProto; import org.apache.ratis.proto.RaftProtos.AppendEntriesRequestProto; +import org.apache.ratis.util.ReferenceCountedObject; import java.io.IOException; import java.util.concurrent.CompletableFuture; public interface RaftServerAsynchronousProtocol { - CompletableFuture appendEntriesAsync(AppendEntriesRequestProto request) - throws IOException; + /** + * It is recommended to override {@link #appendEntriesAsync(ReferenceCountedObject)} instead. + * Then, it does not have to override this method. + */ + default CompletableFuture appendEntriesAsync(AppendEntriesRequestProto request) + throws IOException { + throw new UnsupportedOperationException(); + } + + /** + * A referenced counted request is submitted from a client for processing. + * Implementations of this method should retain the request, process it and then release it. + * The request may be retained even after the future returned by this method has completed. + * + * @return a future of the reply + * @see ReferenceCountedObject + */ + default CompletableFuture appendEntriesAsync( + ReferenceCountedObject requestRef) throws IOException { + // Default implementation for backward compatibility. + try { + return appendEntriesAsync(requestRef.retain()) + .whenComplete((r, e) -> requestRef.release()); + } catch (Exception e) { + requestRef.release(); + throw e; + } + } CompletableFuture readIndexAsync(ReadIndexRequestProto request) throws IOException; diff --git a/ratis-server-api/src/main/java/org/apache/ratis/server/raftlog/RaftLogSequentialOps.java b/ratis-server-api/src/main/java/org/apache/ratis/server/raftlog/RaftLogSequentialOps.java index 5e8bd6d784..81d70393e4 100644 --- a/ratis-server-api/src/main/java/org/apache/ratis/server/raftlog/RaftLogSequentialOps.java +++ b/ratis-server-api/src/main/java/org/apache/ratis/server/raftlog/RaftLogSequentialOps.java @@ -22,8 +22,10 @@ import org.apache.ratis.server.RaftConfiguration; import org.apache.ratis.statemachine.TransactionContext; import org.apache.ratis.util.Preconditions; +import org.apache.ratis.util.ReferenceCountedObject; import org.apache.ratis.util.StringUtils; import org.apache.ratis.util.function.CheckedSupplier; +import org.apache.ratis.util.function.UncheckedAutoCloseableSupplier; import java.util.Arrays; import java.util.List; @@ -134,21 +136,36 @@ default CompletableFuture appendEntry(LogEntryProto entry, TransactionCont /** * The same as append(Arrays.asList(entries)). * - * @deprecated use {@link #append(List)} + * @deprecated use {@link #append(ReferenceCountedObject)}. */ @Deprecated default List> append(LogEntryProto... entries) { return append(Arrays.asList(entries)); } + /** + * @deprecated use {@link #append(ReferenceCountedObject)}. + */ + @Deprecated + default List> append(List entries) { + throw new UnsupportedOperationException(); + } + /** * Append asynchronously all the given log entries. * Used by the followers. * * If an existing entry conflicts with a new one (same index but different terms), * delete the existing entry and all entries that follow it (§5.3). + * + * A reference counter is also submitted. + * For each entry, implementations of this method should retain the counter, process it and then release. */ - List> append(List entries); + default List> append(ReferenceCountedObject> entriesRef) { + try(UncheckedAutoCloseableSupplier> entries = entriesRef.retainAndReleaseOnClose()) { + return append(entries.get()); + } + } /** * Truncate asynchronously the log entries till the given index (inclusively). diff --git a/ratis-server/src/main/java/org/apache/ratis/server/impl/RaftServerImpl.java b/ratis-server/src/main/java/org/apache/ratis/server/impl/RaftServerImpl.java index 32e223e009..5cd3ad3bc4 100644 --- a/ratis-server/src/main/java/org/apache/ratis/server/impl/RaftServerImpl.java +++ b/ratis-server/src/main/java/org/apache/ratis/server/impl/RaftServerImpl.java @@ -1533,35 +1533,30 @@ private RequestVoteReplyProto requestVote(Phase phase, public AppendEntriesReplyProto appendEntries(AppendEntriesRequestProto r) throws IOException { try { - return appendEntriesAsync(r).join(); + return appendEntriesAsync(ReferenceCountedObject.wrap(r)).join(); } catch (CompletionException e) { throw IOUtils.asIOException(JavaUtils.unwrapCompletionException(e)); } } @Override - public CompletableFuture appendEntriesAsync(AppendEntriesRequestProto r) - throws IOException { + public CompletableFuture appendEntriesAsync( + ReferenceCountedObject requestRef) throws IOException { + final AppendEntriesRequestProto r = requestRef.retain(); final RaftRpcRequestProto request = r.getServerRequest(); final TermIndex previous = r.hasPreviousLog()? TermIndex.valueOf(r.getPreviousLog()) : null; - try { - final RaftPeerId leaderId = RaftPeerId.valueOf(request.getRequestorId()); - final RaftGroupId leaderGroupId = ProtoUtils.toRaftGroupId(request.getRaftGroupId()); - - CodeInjectionForTesting.execute(APPEND_ENTRIES, getId(), leaderId, previous, r); - - assertLifeCycleState(LifeCycle.States.STARTING_OR_RUNNING); - if (!startComplete.get()) { - throw new ServerNotReadyException(getMemberId() + ": The server role is not yet initialized."); - } - assertGroup(getMemberId(), leaderId, leaderGroupId); - assertEntries(r, previous, state); + final RaftPeerId requestorId = RaftPeerId.valueOf(request.getRequestorId()); - return appendEntriesAsync(leaderId, request.getCallId(), previous, r); + try { + preAppendEntriesAsync(requestorId, ProtoUtils.toRaftGroupId(request.getRaftGroupId()), r.getLeaderTerm(), + previous, r.getLeaderCommit(), r.getInitializing(), entries); + return appendEntriesAsync(requestorId, r.getLeaderTerm(), previous, r.getLeaderCommit(), + request.getCallId(), r.getInitializing(), r.getCommitInfosList(), entries, requestRef) + .whenComplete((reply, e) -> requestRef.release()); } catch(Exception t) { - LOG.error("{}: Failed appendEntries* {}", getMemberId(), - toAppendEntriesRequestString(r, stateMachine::toStateMachineLogEntryString), t); - throw IOUtils.asIOException(t); + LOG.error("{}: Failed appendEntriesAsync {}", getMemberId(), r, t); + requestRef.release(); + throw t; } } @@ -1624,9 +1619,11 @@ ExecutorService getServerExecutor() { return serverExecutor; } - private CompletableFuture appendEntriesAsync(RaftPeerId leaderId, long callId, - TermIndex previous, AppendEntriesRequestProto proto) throws IOException { - final List entries = proto.getEntriesList(); + @SuppressWarnings("checkstyle:parameternumber") + private CompletableFuture appendEntriesAsync( + RaftPeerId leaderId, long leaderTerm, TermIndex previous, long leaderCommit, long callId, boolean initializing, + List commitInfos, List entries, + ReferenceCountedObject requestRef) throws IOException { final boolean isHeartbeat = entries.isEmpty(); logAppendEntries(isHeartbeat, () -> getMemberId() + ": appendEntries* " + toAppendEntriesRequestString(proto, stateMachine::toStateMachineLogEntryString)); @@ -1683,7 +1680,10 @@ leaderId, getMemberId(), currentTerm, followerCommit, inconsistencyReplyNextInde : appendLogTermIndices != null ? appendLogTermIndices.append(entries, this::appendLog) : JavaUtils.allOf(state.getLog().append(entries)); - proto.getCommitInfosList().forEach(commitInfoCache::update); + + final List> futures = entries.isEmpty() ? Collections.emptyList() + : state.getLog().append(requestRef.delegate(entries)); + commitInfos.forEach(commitInfoCache::update); CodeInjectionForTesting.execute(LOG_SYNC, getId(), null); if (!isHeartbeat) { diff --git a/ratis-server/src/main/java/org/apache/ratis/server/impl/RaftServerProxy.java b/ratis-server/src/main/java/org/apache/ratis/server/impl/RaftServerProxy.java index 8539fa99ec..4ab3ac7e27 100644 --- a/ratis-server/src/main/java/org/apache/ratis/server/impl/RaftServerProxy.java +++ b/ratis-server/src/main/java/org/apache/ratis/server/impl/RaftServerProxy.java @@ -648,11 +648,16 @@ public StartLeaderElectionReplyProto startLeaderElection(StartLeaderElectionRequ } @Override - public CompletableFuture appendEntriesAsync(AppendEntriesRequestProto request) { - final RaftGroupId groupId = ProtoUtils.toRaftGroupId(request.getServerRequest().getRaftGroupId()); - return getImplFuture(groupId) - .thenCompose(impl -> JavaUtils.callAsUnchecked( - () -> impl.appendEntriesAsync(request), CompletionException::new)); + public CompletableFuture appendEntriesAsync( + ReferenceCountedObject requestRef) { + AppendEntriesRequestProto request = requestRef.retain(); + try { + final RaftGroupId groupId = ProtoUtils.toRaftGroupId(request.getServerRequest().getRaftGroupId()); + return getImplFuture(groupId) + .thenCompose(impl -> impl.executeSubmitServerRequestAsync(() -> impl.appendEntriesAsync(requestRef))); + } finally { + requestRef.release(); + } } @Override diff --git a/ratis-server/src/main/java/org/apache/ratis/server/raftlog/RaftLogBase.java b/ratis-server/src/main/java/org/apache/ratis/server/raftlog/RaftLogBase.java index 48b410147c..956eee5dde 100644 --- a/ratis-server/src/main/java/org/apache/ratis/server/raftlog/RaftLogBase.java +++ b/ratis-server/src/main/java/org/apache/ratis/server/raftlog/RaftLogBase.java @@ -31,7 +31,9 @@ import org.apache.ratis.util.JavaUtils; import org.apache.ratis.util.OpenCloseState; import org.apache.ratis.util.Preconditions; +import org.apache.ratis.util.ReferenceCountedObject; import org.apache.ratis.util.TimeDuration; +import org.apache.ratis.util.function.UncheckedAutoCloseableSupplier; import java.io.IOException; import java.util.List; @@ -357,11 +359,19 @@ public final CompletableFuture appendEntry(LogEntryProto entry, Transactio protected abstract CompletableFuture appendEntryImpl(LogEntryProto entry, TransactionContext context); @Override - public final List> append(List entries) { + public final List> append(ReferenceCountedObject> entries) { return runner.runSequentially(() -> appendImpl(entries)); } - protected abstract List> appendImpl(List entries); + protected List> appendImpl(List entries) { + throw new UnsupportedOperationException(); + } + + protected List> appendImpl(ReferenceCountedObject> entriesRef) { + try(UncheckedAutoCloseableSupplier> entries = entriesRef.retainAndReleaseOnClose()) { + return appendImpl(entries.get()); + } + } @Override public String toString() { diff --git a/ratis-server/src/main/java/org/apache/ratis/server/raftlog/memory/MemoryRaftLog.java b/ratis-server/src/main/java/org/apache/ratis/server/raftlog/memory/MemoryRaftLog.java index ebb1e27d77..7435bb178d 100644 --- a/ratis-server/src/main/java/org/apache/ratis/server/raftlog/memory/MemoryRaftLog.java +++ b/ratis-server/src/main/java/org/apache/ratis/server/raftlog/memory/MemoryRaftLog.java @@ -28,6 +28,7 @@ import org.apache.ratis.statemachine.TransactionContext; import org.apache.ratis.util.AutoCloseableLock; import org.apache.ratis.util.Preconditions; +import org.apache.ratis.util.ReferenceCountedObject; import java.io.IOException; import java.util.ArrayList; @@ -35,6 +36,7 @@ import java.util.List; import java.util.concurrent.CompletableFuture; import java.util.concurrent.atomic.AtomicReference; +import java.util.function.Function; import java.util.function.LongSupplier; /** @@ -42,10 +44,10 @@ */ public class MemoryRaftLog extends RaftLogBase { static class EntryList { - private final List entries = new ArrayList<>(); + private final List> entries = new ArrayList<>(); LogEntryProto get(int i) { - return i >= 0 && i < entries.size() ? entries.get(i) : null; + return i >= 0 && i < entries.size() ? entries.get(i).get() : null; } TermIndex getTermIndex(int i) { @@ -62,18 +64,25 @@ int size() { void truncate(int index) { if (entries.size() > index) { - entries.subList(index, entries.size()).clear(); + clear(index, entries.size()); } } void purge(int index) { if (entries.size() > index) { - entries.subList(0, index).clear(); + clear(0, index); } } - void add(LogEntryProto entry) { - entries.add(entry); + void clear(int from, int to) { + List> subList = entries.subList(from, to); + subList.forEach(ReferenceCountedObject::release); + subList.clear(); + } + + void add(ReferenceCountedObject entryRef) { + entryRef.retain(); + entries.add(entryRef); } } @@ -170,7 +179,9 @@ protected CompletableFuture appendEntryImpl(LogEntryProto entry, Transacti checkLogState(); try(AutoCloseableLock writeLock = writeLock()) { validateLogEntry(entry); - entries.add(entry); + Function> wrap = context != null ? + context::wrap : ReferenceCountedObject::wrap; + entries.add(wrap.apply(entry)); } return CompletableFuture.completedFuture(entry.getIndex()); } @@ -181,12 +192,14 @@ public long getStartIndex() { } @Override - public List> appendImpl(List logEntryProtos) { + public List> appendImpl(ReferenceCountedObject> entriesRef) { checkLogState(); + final List logEntryProtos = entriesRef.retain(); if (logEntryProtos == null || logEntryProtos.isEmpty()) { + entriesRef.release(); return Collections.emptyList(); } - try(AutoCloseableLock writeLock = writeLock()) { + try (AutoCloseableLock writeLock = writeLock()) { // Before truncating the entries, we first need to check if some // entries are duplicated. If the leader sends entry 6, entry 7, then // entry 6 again, without this check the follower may truncate entry 7 @@ -214,10 +227,12 @@ public List> appendImpl(List logEntryProt } for (int i = index; i < logEntryProtos.size(); i++) { LogEntryProto logEntryProto = logEntryProtos.get(i); - this.entries.add(logEntryProto); + entries.add(entriesRef.delegate(logEntryProto)); futures.add(CompletableFuture.completedFuture(logEntryProto.getIndex())); } return futures; + } finally { + entriesRef.release(); } } diff --git a/ratis-server/src/main/java/org/apache/ratis/server/raftlog/segmented/SegmentedRaftLog.java b/ratis-server/src/main/java/org/apache/ratis/server/raftlog/segmented/SegmentedRaftLog.java index a6ea6e3caf..dddbd4c2d5 100644 --- a/ratis-server/src/main/java/org/apache/ratis/server/raftlog/segmented/SegmentedRaftLog.java +++ b/ratis-server/src/main/java/org/apache/ratis/server/raftlog/segmented/SegmentedRaftLog.java @@ -36,6 +36,7 @@ import org.apache.ratis.proto.RaftProtos.LogEntryProto; import org.apache.ratis.statemachine.StateMachine; import org.apache.ratis.statemachine.TransactionContext; +import org.apache.ratis.statemachine.impl.TransactionContextImpl; import org.apache.ratis.thirdparty.com.google.protobuf.ByteString; import org.apache.ratis.util.AutoCloseableLock; import org.apache.ratis.util.AwaitToRun; @@ -461,12 +462,14 @@ private boolean isSegmentFull(LogSegment segment, LogEntryProto entry) { } @Override - public List> appendImpl(List entries) { + protected List> appendImpl(ReferenceCountedObject> entriesRef) { checkLogState(); + final List entries = entriesRef.retain(); if (entries == null || entries.isEmpty()) { + entriesRef.release(); return Collections.emptyList(); } - try(AutoCloseableLock writeLock = writeLock()) { + try (AutoCloseableLock writeLock = writeLock()) { final TruncateIndices ti = cache.computeTruncateIndices(server::notifyTruncatedLogEntry, entries); final long truncateIndex = ti.getTruncateIndex(); final int index = ti.getArrayIndex(); @@ -481,9 +484,15 @@ public List> appendImpl(List entries) { } for (int i = index; i < entries.size(); i++) { final LogEntryProto entry = entries.get(i); - futures.add(appendEntry(entry, server.getTransactionContext(entry, true))); + TransactionContextImpl transactionContext = (TransactionContextImpl) server.getTransactionContext(entry, true); + if (transactionContext != null) { + transactionContext.setDelegatedRef(entriesRef); + } + futures.add(appendEntry(entry, transactionContext)); } return futures; + } finally { + entriesRef.release(); } } diff --git a/ratis-test/src/test/java/org/apache/ratis/server/raftlog/memory/MemoryRaftLogTest.java b/ratis-test/src/test/java/org/apache/ratis/server/raftlog/memory/MemoryRaftLogTest.java index 5a41f9ed9a..17c309f0bd 100644 --- a/ratis-test/src/test/java/org/apache/ratis/server/raftlog/memory/MemoryRaftLogTest.java +++ b/ratis-test/src/test/java/org/apache/ratis/server/raftlog/memory/MemoryRaftLogTest.java @@ -37,6 +37,7 @@ import org.junit.jupiter.api.Test; import org.slf4j.event.Level; +@SuppressWarnings({"deprecation"}) public class MemoryRaftLogTest extends BaseTest { static { Slf4jUtils.setLogLevel(MemoryRaftLog.LOG, Level.DEBUG); diff --git a/ratis-test/src/test/java/org/apache/ratis/server/raftlog/segmented/TestCacheEviction.java b/ratis-test/src/test/java/org/apache/ratis/server/raftlog/segmented/TestCacheEviction.java index 6ad429249b..ae8bf34323 100644 --- a/ratis-test/src/test/java/org/apache/ratis/server/raftlog/segmented/TestCacheEviction.java +++ b/ratis-test/src/test/java/org/apache/ratis/server/raftlog/segmented/TestCacheEviction.java @@ -38,6 +38,7 @@ import org.apache.ratis.statemachine.impl.SimpleStateMachine4Testing; import org.apache.ratis.statemachine.StateMachine; import org.apache.ratis.util.JavaUtils; +import org.apache.ratis.util.ReferenceCountedObject; import org.apache.ratis.util.SizeInBytes; import org.junit.jupiter.api.Assertions; import org.junit.jupiter.api.Test; @@ -174,7 +175,7 @@ public void testEvictionInSegmentedLog() throws Exception { raftLog.open(RaftLog.INVALID_LOG_INDEX, null); List slist = TestSegmentedRaftLog.prepareRanges(0, maxCachedNum, 7, 0); List entries = generateEntries(slist); - raftLog.append(entries).forEach(CompletableFuture::join); + raftLog.append(ReferenceCountedObject.wrap(entries)).forEach(CompletableFuture::join); // check the current cached segment number: the last segment is still open Assertions.assertEquals(maxCachedNum - 1, @@ -184,7 +185,7 @@ public void testEvictionInSegmentedLog() throws Exception { Mockito.when(info.getFollowerNextIndices()).thenReturn(new long[]{21, 40, 40}); slist = TestSegmentedRaftLog.prepareRanges(maxCachedNum, maxCachedNum + 2, 7, 7 * maxCachedNum); entries = generateEntries(slist); - raftLog.append(entries).forEach(CompletableFuture::join); + raftLog.append(ReferenceCountedObject.wrap(entries)).forEach(CompletableFuture::join); // check the cached segment number again. since the slowest follower is on // index 21, the eviction should happen and evict 3 segments diff --git a/ratis-test/src/test/java/org/apache/ratis/server/raftlog/segmented/TestSegmentedRaftLog.java b/ratis-test/src/test/java/org/apache/ratis/server/raftlog/segmented/TestSegmentedRaftLog.java index 181d1fa430..580de03e98 100644 --- a/ratis-test/src/test/java/org/apache/ratis/server/raftlog/segmented/TestSegmentedRaftLog.java +++ b/ratis-test/src/test/java/org/apache/ratis/server/raftlog/segmented/TestSegmentedRaftLog.java @@ -43,6 +43,7 @@ import org.apache.ratis.util.CodeInjectionForTesting; import org.apache.ratis.util.DataBlockingQueue; import org.apache.ratis.util.LifeCycle; +import org.apache.ratis.util.ReferenceCountedObject; import org.apache.ratis.util.Slf4jUtils; import org.apache.ratis.util.FileUtils; import org.apache.ratis.util.JavaUtils; @@ -677,7 +678,7 @@ public void testAppendEntriesWithInconsistency(Boolean useAsyncFlush, Boolean sm LOG.info("newEntries[0] = {}", newEntries.get(0)); final int last = newEntries.size() - 1; LOG.info("newEntries[{}] = {}", last, newEntries.get(last)); - raftLog.append(newEntries).forEach(CompletableFuture::join); + raftLog.append(ReferenceCountedObject.wrap(newEntries)).forEach(CompletableFuture::join); checkFailedEntries(entries, 650, retryCache); checkEntries(raftLog, entries, 0, 650); @@ -886,8 +887,9 @@ public void testAsyncFlushPerf1(Boolean useAsyncFlush, Boolean smSyncFlush) thro long start = System.nanoTime(); for (int i = 0; i < entries.size(); i += 5) { // call append API - futures.add(raftLog.append(Arrays.asList( - entries.get(i), entries.get(i + 1), entries.get(i + 2), entries.get(i + 3), entries.get(i + 4)))); + List entries1 = Arrays.asList( + entries.get(i), entries.get(i + 1), entries.get(i + 2), entries.get(i + 3), entries.get(i + 4)); + futures.add(raftLog.append(ReferenceCountedObject.wrap(entries1))); } for (List> futureList: futures) { futureList.forEach(CompletableFuture::join); From b34799f637bed633357c52adaa99afdd0d30061f Mon Sep 17 00:00:00 2001 From: Duong Nguyen Date: Thu, 18 Jan 2024 22:40:40 -0800 Subject: [PATCH 330/397] RATIS-2009. ReferenceCount should work for all LogEntry types. (#1021) (cherry picked from commit 82c31eaf39b24ffb31452ec58b3d96dcee9bab7e) --- .../server/GrpcClientProtocolService.java | 3 ++- .../server/raftlog/RaftLogSequentialOps.java | 16 +++++++++++--- .../ratis/server/raftlog/RaftLogBase.java | 11 ++++++---- .../server/raftlog/memory/MemoryRaftLog.java | 13 ++++++----- .../raftlog/segmented/SegmentedRaftLog.java | 22 +++++++++---------- .../java/org/apache/ratis/RaftTestUtil.java | 3 ++- 6 files changed, 42 insertions(+), 26 deletions(-) diff --git a/ratis-grpc/src/main/java/org/apache/ratis/grpc/server/GrpcClientProtocolService.java b/ratis-grpc/src/main/java/org/apache/ratis/grpc/server/GrpcClientProtocolService.java index e8de4def04..67e75d6063 100644 --- a/ratis-grpc/src/main/java/org/apache/ratis/grpc/server/GrpcClientProtocolService.java +++ b/ratis-grpc/src/main/java/org/apache/ratis/grpc/server/GrpcClientProtocolService.java @@ -64,7 +64,7 @@ private static class PendingOrderedRequest implements SlidingWindow.ServerSideRe PendingOrderedRequest(ReferenceCountedObject requestRef) { this.requestRef = requestRef; - this.request = requestRef != null ? requestRef.get() : null; + this.request = requestRef != null ? requestRef.retain() : null; } @Override @@ -363,6 +363,7 @@ void processClientRequest(PendingOrderedRequest pending) { final long seq = pending.getSeqNum(); processClientRequest(pending.getRequestRef(), reply -> slidingWindow.receiveReply(seq, reply, this::sendReply)); + pending.getRequestRef().release(); } @Override diff --git a/ratis-server-api/src/main/java/org/apache/ratis/server/raftlog/RaftLogSequentialOps.java b/ratis-server-api/src/main/java/org/apache/ratis/server/raftlog/RaftLogSequentialOps.java index 81d70393e4..5a25728830 100644 --- a/ratis-server-api/src/main/java/org/apache/ratis/server/raftlog/RaftLogSequentialOps.java +++ b/ratis-server-api/src/main/java/org/apache/ratis/server/raftlog/RaftLogSequentialOps.java @@ -126,11 +126,21 @@ OUTPUT runSequentially( CompletableFuture appendEntry(LogEntryProto entry); /** - * Append asynchronously an entry. - * Used by the leader. + * @deprecated use {@link #appendEntry(ReferenceCountedObject, TransactionContext)}}. */ + @Deprecated default CompletableFuture appendEntry(LogEntryProto entry, TransactionContext context) { - return appendEntry(entry); + throw new UnsupportedOperationException(); + } + + /** + * Append asynchronously an entry. + * Used for scenarios that there is a ReferenceCountedObject context for resource cleanup when the given entry + * is no longer used/referenced by this log. + */ + default CompletableFuture appendEntry(ReferenceCountedObject entryRef, + TransactionContext context) { + return appendEntry(entryRef.get(), context); } /** diff --git a/ratis-server/src/main/java/org/apache/ratis/server/raftlog/RaftLogBase.java b/ratis-server/src/main/java/org/apache/ratis/server/raftlog/RaftLogBase.java index 956eee5dde..c6c33e6714 100644 --- a/ratis-server/src/main/java/org/apache/ratis/server/raftlog/RaftLogBase.java +++ b/ratis-server/src/main/java/org/apache/ratis/server/raftlog/RaftLogBase.java @@ -195,7 +195,8 @@ private long appendImpl(long term, TransactionContext operation) throws StateMac throw new StateMachineException(memberId, new RaftLogIOException( "Log entry size " + entrySize + " exceeds the max buffer limit of " + maxBufferSize)); } - appendEntry(e, operation).whenComplete((returned, t) -> { + + appendEntry(operation.wrap(e), operation).whenComplete((returned, t) -> { if (t != null) { LOG.error(name + ": Failed to write log entry " + toLogEntryString(e), t); } else if (returned != nextIndex) { @@ -348,15 +349,17 @@ public final CompletableFuture purge(long suggestedIndex) { @Override public final CompletableFuture appendEntry(LogEntryProto entry) { - return appendEntry(entry, null); + return appendEntry(ReferenceCountedObject.wrap(entry), null); } @Override - public final CompletableFuture appendEntry(LogEntryProto entry, TransactionContext context) { + public final CompletableFuture appendEntry(ReferenceCountedObject entry, + TransactionContext context) { return runner.runSequentially(() -> appendEntryImpl(entry, context)); } - protected abstract CompletableFuture appendEntryImpl(LogEntryProto entry, TransactionContext context); + protected abstract CompletableFuture appendEntryImpl(ReferenceCountedObject entry, + TransactionContext context); @Override public final List> append(ReferenceCountedObject> entries) { diff --git a/ratis-server/src/main/java/org/apache/ratis/server/raftlog/memory/MemoryRaftLog.java b/ratis-server/src/main/java/org/apache/ratis/server/raftlog/memory/MemoryRaftLog.java index 7435bb178d..fc7973aab6 100644 --- a/ratis-server/src/main/java/org/apache/ratis/server/raftlog/memory/MemoryRaftLog.java +++ b/ratis-server/src/main/java/org/apache/ratis/server/raftlog/memory/MemoryRaftLog.java @@ -36,7 +36,6 @@ import java.util.List; import java.util.concurrent.CompletableFuture; import java.util.concurrent.atomic.AtomicReference; -import java.util.function.Function; import java.util.function.LongSupplier; /** @@ -175,13 +174,15 @@ public TermIndex getLastEntryTermIndex() { } @Override - protected CompletableFuture appendEntryImpl(LogEntryProto entry, TransactionContext context) { + protected CompletableFuture appendEntryImpl(ReferenceCountedObject entryRef, + TransactionContext context) { checkLogState(); - try(AutoCloseableLock writeLock = writeLock()) { + LogEntryProto entry = entryRef.retain(); + try (AutoCloseableLock writeLock = writeLock()) { validateLogEntry(entry); - Function> wrap = context != null ? - context::wrap : ReferenceCountedObject::wrap; - entries.add(wrap.apply(entry)); + entries.add(entryRef); + } finally { + entryRef.release(); } return CompletableFuture.completedFuture(entry.getIndex()); } diff --git a/ratis-server/src/main/java/org/apache/ratis/server/raftlog/segmented/SegmentedRaftLog.java b/ratis-server/src/main/java/org/apache/ratis/server/raftlog/segmented/SegmentedRaftLog.java index dddbd4c2d5..86c9314b44 100644 --- a/ratis-server/src/main/java/org/apache/ratis/server/raftlog/segmented/SegmentedRaftLog.java +++ b/ratis-server/src/main/java/org/apache/ratis/server/raftlog/segmented/SegmentedRaftLog.java @@ -396,8 +396,10 @@ protected CompletableFuture purgeImpl(long index) { } @Override - protected CompletableFuture appendEntryImpl(LogEntryProto entry, TransactionContext context) { + protected CompletableFuture appendEntryImpl(ReferenceCountedObject entryRef, + TransactionContext context) { checkLogState(); + LogEntryProto entry = entryRef.retain(); if (LOG.isTraceEnabled()) { LOG.trace("{}: appendEntry {}", getName(), LogProtoUtils.toLogEntryString(entry)); } @@ -431,21 +433,22 @@ protected CompletableFuture appendEntryImpl(LogEntryProto entry, Transacti // If the entry has state machine data, then the entry should be inserted // to statemachine first and then to the cache. Not following the order // will leave a spurious entry in the cache. - cache.verifyAppendEntryIndex(entry); - CompletableFuture writeFuture = - fileLogWorker.writeLogEntry(entry, context).getFuture(); + final Task write = fileLogWorker.writeLogEntry(entry, removedStateMachineData, context); if (stateMachineCachingEnabled) { // The stateMachineData will be cached inside the StateMachine itself. - cache.appendEntry(LogProtoUtils.removeStateMachineData(entry), - LogSegment.Op.WRITE_CACHE_WITH_STATE_MACHINE_CACHE); + cache.appendEntry(LogSegment.Op.WRITE_CACHE_WITH_STATE_MACHINE_CACHE, + entryRef.delegate(removedStateMachineData)); } else { - cache.appendEntry(entry, LogSegment.Op.WRITE_CACHE_WITHOUT_STATE_MACHINE_CACHE); + cache.appendEntry(LogSegment.Op.WRITE_CACHE_WITHOUT_STATE_MACHINE_CACHE, entryRef + ); } writeFuture.whenComplete((clientReply, exception) -> appendEntryTimerContext.stop()); return writeFuture; } catch (Exception e) { LOG.error("{}: Failed to append {}", getName(), toLogEntryString(entry), e); throw e; + } finally { + entryRef.release(); } } @@ -485,10 +488,7 @@ protected List> appendImpl(ReferenceCountedObject Date: Thu, 18 Jan 2024 23:11:52 -0800 Subject: [PATCH 331/397] RATIS-1934. Support Zero-Copy in GrpcServerProtocolService (#1014) (cherry picked from commit d459eb61c356185e085fbd948a1d068e97bea7d4) --- .../server/GrpcServerProtocolService.java | 72 ++++++++++++++++--- .../apache/ratis/grpc/server/GrpcService.java | 4 +- 2 files changed, 65 insertions(+), 11 deletions(-) diff --git a/ratis-grpc/src/main/java/org/apache/ratis/grpc/server/GrpcServerProtocolService.java b/ratis-grpc/src/main/java/org/apache/ratis/grpc/server/GrpcServerProtocolService.java index 7c5ce9ae62..2c2aa2bed7 100644 --- a/ratis-grpc/src/main/java/org/apache/ratis/grpc/server/GrpcServerProtocolService.java +++ b/ratis-grpc/src/main/java/org/apache/ratis/grpc/server/GrpcServerProtocolService.java @@ -20,11 +20,13 @@ import java.util.function.Consumer; import java.util.function.Function; import org.apache.ratis.grpc.GrpcUtil; +import org.apache.ratis.grpc.metrics.ZeroCopyMetrics; +import org.apache.ratis.grpc.util.ZeroCopyMessageMarshaller; import org.apache.ratis.protocol.RaftPeerId; import org.apache.ratis.server.RaftServer; import org.apache.ratis.server.protocol.RaftServerProtocol; import org.apache.ratis.server.util.ServerStringUtils; -import org.apache.ratis.thirdparty.com.google.protobuf.MessageOrBuilder; +import org.apache.ratis.thirdparty.io.grpc.ServerServiceDefinition; import org.apache.ratis.thirdparty.io.grpc.Status; import org.apache.ratis.thirdparty.io.grpc.StatusRuntimeException; import org.apache.ratis.thirdparty.io.grpc.stub.StreamObserver; @@ -44,6 +46,9 @@ import java.util.concurrent.atomic.AtomicReference; import java.util.function.Supplier; +import static org.apache.ratis.grpc.GrpcUtil.addMethodWithCustomMarshaller; +import static org.apache.ratis.proto.grpc.RaftServerProtocolServiceGrpc.getAppendEntriesMethod; + class GrpcServerProtocolService extends RaftServerProtocolServiceImplBase { public static final Logger LOG = LoggerFactory.getLogger(GrpcServerProtocolService.class); @@ -56,8 +61,9 @@ static class PendingServerRequest { private final REQUEST request; private final CompletableFuture future = new CompletableFuture<>(); - PendingServerRequest(REQUEST request) { - this.request = request; + PendingServerRequest(ReferenceCountedObject requestRef) { + this.request = requestRef.retain(); + this.future.whenComplete((r, e) -> requestRef.release()); } REQUEST getRequest() { @@ -98,7 +104,21 @@ private String getPreviousRequestString() { .orElse(null); } - abstract CompletableFuture process(REQUEST request) throws IOException; + CompletableFuture process(REQUEST request) throws IOException { + throw new UnsupportedOperationException("This method is not supported."); + } + + CompletableFuture process(ReferenceCountedObject requestRef) + throws IOException { + try { + return process(requestRef.retain()); + } finally { + requestRef.release(); + } + } + + void release(REQUEST req) { + } abstract long getCallId(REQUEST request); @@ -135,22 +155,29 @@ void composeRequest(CompletableFuture current) { @Override public void onNext(REQUEST request) { + ReferenceCountedObject requestRef = ReferenceCountedObject.wrap(request, () -> {}, released -> { + if (released) { + release(request); + } + }); + if (!replyInOrder(request)) { try { - composeRequest(process(request).thenApply(this::handleReply)); + composeRequest(process(requestRef).thenApply(this::handleReply)); } catch (Exception e) { handleError(e, request); + release(request); } return; } - final PendingServerRequest current = new PendingServerRequest<>(request); + final PendingServerRequest current = new PendingServerRequest<>(requestRef); final PendingServerRequest previous = previousOnNext.getAndSet(current); final CompletableFuture previousFuture = Optional.ofNullable(previous) .map(PendingServerRequest::getFuture) .orElse(CompletableFuture.completedFuture(null)); try { - final CompletableFuture f = process(request).exceptionally(e -> { + final CompletableFuture f = process(requestRef).exceptionally(e -> { // Handle cases, such as RaftServer is paused handleError(e, request); current.getFuture().completeExceptionally(e); @@ -195,16 +222,35 @@ public void onError(Throwable t) { private final Supplier idSupplier; private final RaftServer server; + private final ZeroCopyMessageMarshaller zeroCopyRequestMarshaller; - GrpcServerProtocolService(Supplier idSupplier, RaftServer server) { + GrpcServerProtocolService(Supplier idSupplier, RaftServer server, ZeroCopyMetrics zeroCopyMetrics) { this.idSupplier = idSupplier; this.server = server; + this.zeroCopyRequestMarshaller = new ZeroCopyMessageMarshaller<>(AppendEntriesRequestProto.getDefaultInstance(), + zeroCopyMetrics::onZeroCopyMessage, zeroCopyMetrics::onNonZeroCopyMessage, zeroCopyMetrics::onReleasedMessage); } RaftPeerId getId() { return idSupplier.get(); } + ServerServiceDefinition bindServiceWithZeroCopy() { + ServerServiceDefinition orig = super.bindService(); + ServerServiceDefinition.Builder builder = ServerServiceDefinition.builder(orig.getServiceDescriptor().getName()); + + // Add appendEntries with zero copy marshaller. + addMethodWithCustomMarshaller(orig, builder, getAppendEntriesMethod(), zeroCopyRequestMarshaller); + // Add remaining methods as is. + orig.getMethods().stream().filter( + x -> !x.getMethodDescriptor().getFullMethodName().equals(getAppendEntriesMethod().getFullMethodName()) + ).forEach( + builder::addMethod + ); + + return builder.build(); + } + @Override public void requestVote(RequestVoteRequestProto request, StreamObserver responseObserver) { @@ -245,8 +291,14 @@ public StreamObserver appendEntries( return new ServerRequestStreamObserver( RaftServerProtocol.Op.APPEND_ENTRIES, responseObserver) { @Override - CompletableFuture process(AppendEntriesRequestProto request) throws IOException { - return server.appendEntriesAsync(ReferenceCountedObject.wrap(request)); + CompletableFuture process(ReferenceCountedObject requestRef) + throws IOException { + return server.appendEntriesAsync(requestRef); + } + + @Override + void release(AppendEntriesRequestProto req) { + zeroCopyRequestMarshaller.release(req); } @Override diff --git a/ratis-grpc/src/main/java/org/apache/ratis/grpc/server/GrpcService.java b/ratis-grpc/src/main/java/org/apache/ratis/grpc/server/GrpcService.java index d89afd565e..d2dadcd90a 100644 --- a/ratis-grpc/src/main/java/org/apache/ratis/grpc/server/GrpcService.java +++ b/ratis-grpc/src/main/java/org/apache/ratis/grpc/server/GrpcService.java @@ -214,8 +214,10 @@ private GrpcService(RaftServer raftServer, Supplier idSupplier, final NettyServerBuilder serverBuilder = startBuildingNettyServer(serverHost, serverPort, serverTlsConfig, grpcMessageSizeMax, flowControlWindow); + GrpcServerProtocolService serverProtocolService = new GrpcServerProtocolService(idSupplier, raftServer, + zeroCopyMetrics); serverBuilder.addService(ServerInterceptors.intercept( - new GrpcServerProtocolService(idSupplier, raftServer), serverInterceptor)); + serverProtocolService.bindServiceWithZeroCopy(), serverInterceptor)); if (!separateAdminServer) { addAdminService(raftServer, serverBuilder); } From 71d832c4d03b5cc99f08807f397e004c53862b18 Mon Sep 17 00:00:00 2001 From: Duong Nguyen Date: Wed, 24 Jan 2024 22:54:32 -0800 Subject: [PATCH 332/397] RATIS-2007. Zero-copy buffers are not released (#1027) (cherry picked from commit e2aaddca9b850e978ee7a1634cadac76e81559c4) --- .../RaftClientAsynchronousProtocol.java | 19 ++++++++++ .../org/apache/ratis/util/SlidingWindow.java | 12 +++++-- .../server/GrpcClientProtocolService.java | 17 +++++++-- .../ratis/server/impl/RaftServerImpl.java | 36 +++++++++++++------ .../server/raftlog/segmented/LogSegment.java | 1 + 5 files changed, 69 insertions(+), 16 deletions(-) diff --git a/ratis-common/src/main/java/org/apache/ratis/protocol/RaftClientAsynchronousProtocol.java b/ratis-common/src/main/java/org/apache/ratis/protocol/RaftClientAsynchronousProtocol.java index 1a9f83c823..49408aa937 100644 --- a/ratis-common/src/main/java/org/apache/ratis/protocol/RaftClientAsynchronousProtocol.java +++ b/ratis-common/src/main/java/org/apache/ratis/protocol/RaftClientAsynchronousProtocol.java @@ -25,4 +25,23 @@ public interface RaftClientAsynchronousProtocol { CompletableFuture submitClientRequestAsync( RaftClientRequest request) throws IOException; + /** + * A referenced counted request is submitted from a client for processing. + * Implementations of this method should retain the request, process it and then release it. + * The request may be retained even after the future returned by this method has completed. + * + * @return a future of the reply + * @see ReferenceCountedObject + */ + default CompletableFuture submitClientRequestAsync( + ReferenceCountedObject requestRef) { + try { + // for backward compatibility + return submitClientRequestAsync(requestRef.retain()); + } catch (Exception e) { + return JavaUtils.completeExceptionally(e); + } finally { + requestRef.release(); + } + } } \ No newline at end of file diff --git a/ratis-common/src/main/java/org/apache/ratis/util/SlidingWindow.java b/ratis-common/src/main/java/org/apache/ratis/util/SlidingWindow.java index 7e37d81322..b27ca13b42 100644 --- a/ratis-common/src/main/java/org/apache/ratis/util/SlidingWindow.java +++ b/ratis-common/src/main/java/org/apache/ratis/util/SlidingWindow.java @@ -51,6 +51,9 @@ interface Request { boolean hasReply(); void fail(Throwable e); + + default void release() { + } } interface ClientSideRequest extends Request { @@ -166,8 +169,12 @@ void endOfRequests(long nextToProcess, REQUEST end, Consumer replyMetho putNewRequest(end); } - void clear() { + void clear(long nextToProcess) { LOG.debug("close {}", this); + final SortedMap tail = requests.tailMap(nextToProcess); + for (REQUEST r : tail.values()) { + r.release(); + } requests.clear(); } @@ -465,6 +472,7 @@ private void processRequestsFromHead(Consumer processingMethod) { return; } else if (r.getSeqNum() == nextToProcess) { processingMethod.accept(r); + r.release(); nextToProcess++; } } @@ -510,7 +518,7 @@ public synchronized boolean endOfRequests(Consumer replyMethod) { @Override public void close() { - requests.clear(); + requests.clear(nextToProcess); } } } \ No newline at end of file diff --git a/ratis-grpc/src/main/java/org/apache/ratis/grpc/server/GrpcClientProtocolService.java b/ratis-grpc/src/main/java/org/apache/ratis/grpc/server/GrpcClientProtocolService.java index 67e75d6063..0671a1841d 100644 --- a/ratis-grpc/src/main/java/org/apache/ratis/grpc/server/GrpcClientProtocolService.java +++ b/ratis-grpc/src/main/java/org/apache/ratis/grpc/server/GrpcClientProtocolService.java @@ -96,6 +96,13 @@ ReferenceCountedObject getRequestRef() { return requestRef; } + @Override + public void release() { + if (requestRef != null) { + requestRef.release(); + } + } + @Override public long getSeqNum() { return request != null? request.getSlidingWindowEntry().getSeqNum(): Long.MAX_VALUE; @@ -363,7 +370,6 @@ void processClientRequest(PendingOrderedRequest pending) { final long seq = pending.getSeqNum(); processClientRequest(pending.getRequestRef(), reply -> slidingWindow.receiveReply(seq, reply, this::sendReply)); - pending.getRequestRef().release(); } @Override @@ -378,7 +384,6 @@ void processClientRequest(ReferenceCountedObject requestRef) final RaftGroupId requestGroupId = request.getRaftGroupId(); // use the group id in the first request as the group id of this observer final RaftGroupId updated = groupId.updateAndGet(g -> g != null ? g : requestGroupId); - final PendingOrderedRequest pending = new PendingOrderedRequest(requestRef); if (!requestGroupId.equals(updated)) { final GroupMismatchException exception = new GroupMismatchException(getId() @@ -387,7 +392,13 @@ void processClientRequest(ReferenceCountedObject requestRef) responseError(exception, () -> "processClientRequest (Group mismatched) for " + request); return; } - slidingWindow.receivedRequest(pending, this::processClientRequest); + final PendingOrderedRequest pending = new PendingOrderedRequest(requestRef); + try { + slidingWindow.receivedRequest(pending, this::processClientRequest); + } catch (Exception e) { + pending.release(); + throw e; + } } finally { requestRef.release(); } diff --git a/ratis-server/src/main/java/org/apache/ratis/server/impl/RaftServerImpl.java b/ratis-server/src/main/java/org/apache/ratis/server/impl/RaftServerImpl.java index 5cd3ad3bc4..59bfe41650 100644 --- a/ratis-server/src/main/java/org/apache/ratis/server/impl/RaftServerImpl.java +++ b/ratis-server/src/main/java/org/apache/ratis/server/impl/RaftServerImpl.java @@ -982,14 +982,28 @@ private CompletableFuture submitClientRequestAsyncInternal( RaftClientRequest request) throws IOException { assertLifeCycleState(LifeCycle.States.RUNNING); LOG.debug("{}: receive client request({})", getMemberId(), request); - final Timekeeper timer = raftServerMetrics.getClientRequestTimer(request.getType()); - final Optional timerContext = Optional.ofNullable(timer).map(Timekeeper::time); - return replyFuture(request).whenComplete((clientReply, exception) -> { - timerContext.ifPresent(Timekeeper.Context::stop); - if (exception != null || clientReply.getException() != null) { - raftServerMetrics.incFailedRequestCount(request.getType()); - } - }); + + try { + assertLifeCycleState(LifeCycle.States.RUNNING); + } catch (ServerNotReadyException e) { + final RaftClientReply reply = newExceptionReply(request, e); + requestRef.release(); + return CompletableFuture.completedFuture(reply); + } + + try { + RaftClientRequest.Type type = request.getType(); + final Timekeeper timer = raftServerMetrics.getClientRequestTimer(type); + final Optional timerContext = Optional.ofNullable(timer).map(Timekeeper::time); + return replyFuture(requestRef).whenComplete((clientReply, exception) -> { + timerContext.ifPresent(Timekeeper.Context::stop); + if (exception != null || clientReply.getException() != null) { + raftServerMetrics.incFailedRequestCount(type); + } + }); + } finally { + requestRef.release(); + } } private CompletableFuture replyFuture(RaftClientRequest request) throws IOException { @@ -1551,12 +1565,12 @@ public CompletableFuture appendEntriesAsync( preAppendEntriesAsync(requestorId, ProtoUtils.toRaftGroupId(request.getRaftGroupId()), r.getLeaderTerm(), previous, r.getLeaderCommit(), r.getInitializing(), entries); return appendEntriesAsync(requestorId, r.getLeaderTerm(), previous, r.getLeaderCommit(), - request.getCallId(), r.getInitializing(), r.getCommitInfosList(), entries, requestRef) - .whenComplete((reply, e) -> requestRef.release()); + request.getCallId(), r.getInitializing(), r.getCommitInfosList(), entries, requestRef); } catch(Exception t) { LOG.error("{}: Failed appendEntriesAsync {}", getMemberId(), r, t); - requestRef.release(); throw t; + } finally { + requestRef.release(); } } diff --git a/ratis-server/src/main/java/org/apache/ratis/server/raftlog/segmented/LogSegment.java b/ratis-server/src/main/java/org/apache/ratis/server/raftlog/segmented/LogSegment.java index bb2bde7edb..860bd196d8 100644 --- a/ratis-server/src/main/java/org/apache/ratis/server/raftlog/segmented/LogSegment.java +++ b/ratis-server/src/main/java/org/apache/ratis/server/raftlog/segmented/LogSegment.java @@ -283,6 +283,7 @@ public LogEntryProto load(LogRecord key) throws IOException { if (ti.equals(key.getTermIndex())) { toReturn.set(entry); } + entryRef.release(); }); loadingTimes.incrementAndGet(); final LogEntryProto proto = toReturn.get(); From d6b9168656ca61f54eaa6a6b18d16f2e440eff9c Mon Sep 17 00:00:00 2001 From: Duong Nguyen Date: Mon, 29 Jan 2024 18:30:00 -0800 Subject: [PATCH 333/397] RATIS-2018. Zero-copy buffers are not released - 2nd chunk (#1032) (cherry picked from commit c4b10faa8a82085bf8ba801fd46e4b71f81297fb) --- .../ratis/util/ReferenceCountedObject.java | 8 ++- .../org/apache/ratis/util/SlidingWindow.java | 16 +++-- .../java/org/apache/ratis/grpc/GrpcUtil.java | 8 +-- .../server/GrpcClientProtocolService.java | 9 ++- .../server/GrpcServerProtocolService.java | 67 ++++++++++++------- .../RaftServerAsynchronousProtocol.java | 6 +- .../ratis/server/impl/RaftServerImpl.java | 8 ++- .../segmented/SegmentedRaftLogCache.java | 18 +++-- .../impl/SimpleStateMachine4Testing.java | 14 +++- 9 files changed, 100 insertions(+), 54 deletions(-) diff --git a/ratis-common/src/main/java/org/apache/ratis/util/ReferenceCountedObject.java b/ratis-common/src/main/java/org/apache/ratis/util/ReferenceCountedObject.java index 7214d34e14..5137e5912c 100644 --- a/ratis-common/src/main/java/org/apache/ratis/util/ReferenceCountedObject.java +++ b/ratis-common/src/main/java/org/apache/ratis/util/ReferenceCountedObject.java @@ -117,7 +117,13 @@ public V retain() { @Override public boolean release() { - return fromRefs.stream().map(ReferenceCountedObject::release).allMatch(r -> r); + boolean allReleased = true; + for (ReferenceCountedObject ref : fromRefs) { + if (!ref.release()) { + allReleased = false; + } + } + return allReleased; } }; } diff --git a/ratis-common/src/main/java/org/apache/ratis/util/SlidingWindow.java b/ratis-common/src/main/java/org/apache/ratis/util/SlidingWindow.java index b27ca13b42..a41b4b136a 100644 --- a/ratis-common/src/main/java/org/apache/ratis/util/SlidingWindow.java +++ b/ratis-common/src/main/java/org/apache/ratis/util/SlidingWindow.java @@ -163,6 +163,7 @@ void endOfRequests(long nextToProcess, REQUEST end, Consumer replyMetho + " will NEVER be processed; request = " + r); r.fail(e); replyMethod.accept(r); + r.release(); } tail.clear(); @@ -451,19 +452,26 @@ public synchronized String toString() { /** A request (or a retry) arrives (may be out-of-order except for the first request). */ public synchronized void receivedRequest(REQUEST request, Consumer processingMethod) { final long seqNum = request.getSeqNum(); + final boolean accepted; if (nextToProcess == -1 && (request.isFirstRequest() || seqNum == 0)) { nextToProcess = seqNum; requests.putNewRequest(request); LOG.debug("Received seq={} (first request), {}", seqNum, this); + accepted = true; + } else if (request.getSeqNum() < nextToProcess) { + LOG.debug("Received seq={} < nextToProcess {}, {}", seqNum, nextToProcess, this); + accepted = false; } else { final boolean isRetry = requests.putIfAbsent(request); LOG.debug("Received seq={}, isRetry? {}, {}", seqNum, isRetry, this); - if (isRetry) { - return; - } + accepted = !isRetry; } - processRequestsFromHead(processingMethod); + if (accepted) { + processRequestsFromHead(processingMethod); + } else { + request.release(); + } } private void processRequestsFromHead(Consumer processingMethod) { diff --git a/ratis-grpc/src/main/java/org/apache/ratis/grpc/GrpcUtil.java b/ratis-grpc/src/main/java/org/apache/ratis/grpc/GrpcUtil.java index b4d9cd6a5e..ff1a52c7a7 100644 --- a/ratis-grpc/src/main/java/org/apache/ratis/grpc/GrpcUtil.java +++ b/ratis-grpc/src/main/java/org/apache/ratis/grpc/GrpcUtil.java @@ -69,14 +69,8 @@ public interface GrpcUtil { Metadata.Key.of("heartbeat", Metadata.ASCII_STRING_MARSHALLER); static StatusRuntimeException wrapException(Throwable t) { - return wrapException(t, -1); - } - - static StatusRuntimeException wrapException(Throwable t, long callId) { t = JavaUtils.unwrapCompletionException(t); - Metadata trailers = new StatusRuntimeExceptionMetadataBuilder(t) - .addCallId(callId) - .build(); + Metadata trailers = new StatusRuntimeExceptionMetadataBuilder(t).build(); return wrapException(t, trailers); } diff --git a/ratis-grpc/src/main/java/org/apache/ratis/grpc/server/GrpcClientProtocolService.java b/ratis-grpc/src/main/java/org/apache/ratis/grpc/server/GrpcClientProtocolService.java index 0671a1841d..97b0118909 100644 --- a/ratis-grpc/src/main/java/org/apache/ratis/grpc/server/GrpcClientProtocolService.java +++ b/ratis-grpc/src/main/java/org/apache/ratis/grpc/server/GrpcClientProtocolService.java @@ -29,6 +29,7 @@ import org.apache.ratis.thirdparty.io.grpc.stub.StreamObserver; import org.apache.ratis.proto.RaftProtos.RaftClientReplyProto; import org.apache.ratis.proto.RaftProtos.RaftClientRequestProto; +import org.apache.ratis.proto.RaftProtos.SlidingWindowEntry; import org.apache.ratis.proto.grpc.RaftClientProtocolServiceGrpc.RaftClientProtocolServiceImplBase; import org.apache.ratis.util.CollectionUtils; import org.apache.ratis.util.JavaUtils; @@ -317,14 +318,16 @@ private class UnorderedRequestStreamObserver extends RequestStreamObserver { void processClientRequest(ReferenceCountedObject requestRef) { final RaftClientRequest request = requestRef.retain(); final long callId = request.getCallId(); - + final SlidingWindowEntry slidingWindowEntry = request.getSlidingWindowEntry(); final CompletableFuture f = processClientRequest(requestRef, reply -> { if (!reply.isSuccess()) { - LOG.info("Failed {}, reply={}", request, reply); + LOG.info("Failed request cid={}, {}, reply={}", callId, slidingWindowEntry, reply); } final RaftClientReplyProto proto = ClientProtoUtils.toRaftClientReplyProto(reply); responseNext(proto); - }).whenComplete((r, e) -> requestRef.release()); + }); + + requestRef.release(); put(callId, f); f.thenAccept(dummy -> remove(callId)); diff --git a/ratis-grpc/src/main/java/org/apache/ratis/grpc/server/GrpcServerProtocolService.java b/ratis-grpc/src/main/java/org/apache/ratis/grpc/server/GrpcServerProtocolService.java index 2c2aa2bed7..867230572c 100644 --- a/ratis-grpc/src/main/java/org/apache/ratis/grpc/server/GrpcServerProtocolService.java +++ b/ratis-grpc/src/main/java/org/apache/ratis/grpc/server/GrpcServerProtocolService.java @@ -28,7 +28,6 @@ import org.apache.ratis.server.util.ServerStringUtils; import org.apache.ratis.thirdparty.io.grpc.ServerServiceDefinition; import org.apache.ratis.thirdparty.io.grpc.Status; -import org.apache.ratis.thirdparty.io.grpc.StatusRuntimeException; import org.apache.ratis.thirdparty.io.grpc.stub.StreamObserver; import org.apache.ratis.proto.RaftProtos.*; import org.apache.ratis.proto.grpc.RaftServerProtocolServiceGrpc.RaftServerProtocolServiceImplBase; @@ -58,21 +57,28 @@ private enum BatchLogKey implements BatchLogger.Key { } static class PendingServerRequest { - private final REQUEST request; + private final AtomicReference> requestRef; private final CompletableFuture future = new CompletableFuture<>(); PendingServerRequest(ReferenceCountedObject requestRef) { - this.request = requestRef.retain(); - this.future.whenComplete((r, e) -> requestRef.release()); + requestRef.retain(); + this.requestRef = new AtomicReference<>(requestRef); } REQUEST getRequest() { - return request; + return Optional.ofNullable(requestRef.get()) + .map(ReferenceCountedObject::get) + .orElse(null); } CompletableFuture getFuture() { return future; } + + void release() { + Optional.ofNullable(requestRef.getAndSet(null)) + .ifPresent(ReferenceCountedObject::release); + } } abstract class ServerRequestStreamObserver @@ -122,20 +128,21 @@ void release(REQUEST req) { abstract long getCallId(REQUEST request); + boolean isHeartbeat(REQUEST request) { + return false; + } + abstract String requestToString(REQUEST request); abstract String replyToString(REPLY reply); abstract boolean replyInOrder(REQUEST request); - StatusRuntimeException wrapException(Throwable e, REQUEST request) { - return GrpcUtil.wrapException(e, getCallId(request)); - } - - private void handleError(Throwable e, REQUEST request) { - GrpcUtil.warn(LOG, () -> getId() + ": Failed " + op + " request " + requestToString(request), e); + private synchronized void handleError(Throwable e, long callId, boolean isHeartbeat) { + GrpcUtil.warn(LOG, () -> getId() + ": Failed " + op + " request cid=" + callId + ", isHeartbeat? " + + isHeartbeat, e); if (isClosed.compareAndSet(false, true)) { - responseObserver.onError(wrapException(e, request)); + responseObserver.onError(GrpcUtil.wrapException(e, callId, isHeartbeat)); } } @@ -165,21 +172,22 @@ public void onNext(REQUEST request) { try { composeRequest(process(requestRef).thenApply(this::handleReply)); } catch (Exception e) { - handleError(e, request); + handleError(e, getCallId(request), isHeartbeat(request)); release(request); } return; } final PendingServerRequest current = new PendingServerRequest<>(requestRef); - final PendingServerRequest previous = previousOnNext.getAndSet(current); - final CompletableFuture previousFuture = Optional.ofNullable(previous) - .map(PendingServerRequest::getFuture) + final long callId = getCallId(current.getRequest()); + final boolean isHeartbeat = isHeartbeat(current.getRequest()); + final Optional> previous = Optional.ofNullable(previousOnNext.getAndSet(current)); + final CompletableFuture previousFuture = previous.map(PendingServerRequest::getFuture) .orElse(CompletableFuture.completedFuture(null)); try { final CompletableFuture f = process(requestRef).exceptionally(e -> { // Handle cases, such as RaftServer is paused - handleError(e, request); + handleError(e, callId, isHeartbeat); current.getFuture().completeExceptionally(e); return null; }).thenCombine(previousFuture, (reply, v) -> { @@ -189,8 +197,14 @@ public void onNext(REQUEST request) { }); composeRequest(f); } catch (Exception e) { - handleError(e, request); + handleError(e, callId, isHeartbeat); current.getFuture().completeExceptionally(e); + } finally { + previous.ifPresent(PendingServerRequest::release); + if (isClosed.get()) { + // Some requests may come after onCompleted or onError, ensure they're released. + releaseLast(); + } } } @@ -206,8 +220,10 @@ public void onCompleted() { getId(), op, ProtoUtils.shortDebugString(reply), suffix)); responseObserver.onCompleted(); }); + releaseLast(); } } + @Override public void onError(Throwable t) { GrpcUtil.warn(LOG, () -> getId() + ": "+ op + " onError, lastRequest: " + getPreviousRequestString(), t); @@ -216,8 +232,13 @@ public void onError(Throwable t) { if (status != null && status.getCode() != Status.Code.CANCELLED) { responseObserver.onCompleted(); } + releaseLast(); } } + + private void releaseLast() { + Optional.ofNullable(previousOnNext.get()).ifPresent(PendingServerRequest::release); + } } private final Supplier idSupplier; @@ -306,6 +327,11 @@ long getCallId(AppendEntriesRequestProto request) { return request.getServerRequest().getCallId(); } + @Override + boolean isHeartbeat(AppendEntriesRequestProto request) { + return request.getEntriesCount() == 0; + } + @Override String requestToString(AppendEntriesRequestProto request) { return ServerStringUtils.toAppendEntriesRequestString(request, null); @@ -320,11 +346,6 @@ String replyToString(AppendEntriesReplyProto reply) { boolean replyInOrder(AppendEntriesRequestProto request) { return request.getEntriesCount() != 0; } - - @Override - StatusRuntimeException wrapException(Throwable e, AppendEntriesRequestProto request) { - return GrpcUtil.wrapException(e, getCallId(request), request.getEntriesCount() == 0); - } }; } diff --git a/ratis-server-api/src/main/java/org/apache/ratis/server/protocol/RaftServerAsynchronousProtocol.java b/ratis-server-api/src/main/java/org/apache/ratis/server/protocol/RaftServerAsynchronousProtocol.java index 1244e72545..035e0a815f 100644 --- a/ratis-server-api/src/main/java/org/apache/ratis/server/protocol/RaftServerAsynchronousProtocol.java +++ b/ratis-server-api/src/main/java/org/apache/ratis/server/protocol/RaftServerAsynchronousProtocol.java @@ -50,11 +50,9 @@ default CompletableFuture appendEntriesAsync( ReferenceCountedObject requestRef) throws IOException { // Default implementation for backward compatibility. try { - return appendEntriesAsync(requestRef.retain()) - .whenComplete((r, e) -> requestRef.release()); - } catch (Exception e) { + return appendEntriesAsync(requestRef.retain()); + } finally { requestRef.release(); - throw e; } } diff --git a/ratis-server/src/main/java/org/apache/ratis/server/impl/RaftServerImpl.java b/ratis-server/src/main/java/org/apache/ratis/server/impl/RaftServerImpl.java index 59bfe41650..4962f32a1d 100644 --- a/ratis-server/src/main/java/org/apache/ratis/server/impl/RaftServerImpl.java +++ b/ratis-server/src/main/java/org/apache/ratis/server/impl/RaftServerImpl.java @@ -1186,7 +1186,13 @@ private CompletableFuture messageStreamAsync(RaftClientRequest return f.thenApply(r -> null); } // the message stream has ended and the request become a WRITE request - return replyFuture(f.join()); + ReferenceCountedObject joinedRequest = f.join(); + try { + return replyFuture(joinedRequest); + } finally { + // Released pending streaming requests. + joinedRequest.release(); + } } return role.getLeaderState() diff --git a/ratis-server/src/main/java/org/apache/ratis/server/raftlog/segmented/SegmentedRaftLogCache.java b/ratis-server/src/main/java/org/apache/ratis/server/raftlog/segmented/SegmentedRaftLogCache.java index 714943c49c..86fb483eee 100644 --- a/ratis-server/src/main/java/org/apache/ratis/server/raftlog/segmented/SegmentedRaftLogCache.java +++ b/ratis-server/src/main/java/org/apache/ratis/server/raftlog/segmented/SegmentedRaftLogCache.java @@ -49,6 +49,7 @@ import java.util.Optional; import java.util.concurrent.CopyOnWriteArrayList; import java.util.function.Consumer; +import java.util.stream.Collectors; /** * In-memory RaftLog Cache. Currently we provide a simple implementation that @@ -364,16 +365,10 @@ TruncationSegments truncate(long index, LogSegment openSegment, Runnable clearOp TruncationSegments purge(long index) { try (AutoCloseableLock writeLock = writeLock()) { int segmentIndex = binarySearch(index); - List list = new ArrayList<>(); - if (segmentIndex == -1) { - // nothing to purge - return null; - } + List list = new LinkedList<>(); if (segmentIndex == -segments.size() - 1) { - for (LogSegment ls : segments) { - list.add(SegmentFileInfo.newClosedSegmentFileInfo(ls)); - } + list.addAll(segments); segments.clear(); sizeInBytes = 0; } else if (segmentIndex >= 0) { @@ -386,13 +381,16 @@ TruncationSegments purge(long index) { for (int i = 0; i <= startIndex; i++) { LogSegment segment = segments.remove(0); // must remove the first segment to avoid gaps. sizeInBytes -= segment.getTotalFileSize(); - list.add(SegmentFileInfo.newClosedSegmentFileInfo(segment)); + list.add(segment); } } else { throw new IllegalStateException("Unexpected gap in segments: binarySearch(" + index + ") returns " + segmentIndex + ", segments=" + segments); } - return list.isEmpty() ? null : new TruncationSegments("purge(" + index + ")", null, list); + list.forEach(LogSegment::evictCache); + List toDelete = list.stream().map(SegmentFileInfo::newClosedSegmentFileInfo) + .collect(Collectors.toList()); + return list.isEmpty() ? null : new TruncationSegments(null, toDelete); } } diff --git a/ratis-server/src/test/java/org/apache/ratis/statemachine/impl/SimpleStateMachine4Testing.java b/ratis-server/src/test/java/org/apache/ratis/statemachine/impl/SimpleStateMachine4Testing.java index 1e77c3d516..45e14dd189 100644 --- a/ratis-server/src/test/java/org/apache/ratis/statemachine/impl/SimpleStateMachine4Testing.java +++ b/ratis-server/src/test/java/org/apache/ratis/statemachine/impl/SimpleStateMachine4Testing.java @@ -42,6 +42,7 @@ import org.apache.ratis.statemachine.StateMachine; import org.apache.ratis.statemachine.TransactionContext; import org.apache.ratis.thirdparty.com.google.protobuf.ByteString; +import org.apache.ratis.thirdparty.com.google.protobuf.InvalidProtocolBufferException; import org.apache.ratis.util.Daemon; import org.apache.ratis.util.JavaUtils; import org.apache.ratis.util.LifeCycle; @@ -250,13 +251,24 @@ public CompletableFuture applyTransaction(TransactionContext trx) { LogEntryProto entry = Objects.requireNonNull(trx.getLogEntry()); LOG.info("applyTransaction for log index {}", entry.getIndex()); - put(entry); + // TODO: Logs kept in StateMachine's cache may be corrupted. Copy for now to have the test pass. + // Use ReferenceCount per RATIS-1997. + LogEntryProto copied = copy(entry); + put(copied); updateLastAppliedTermIndex(entry.getTerm(), entry.getIndex()); final SimpleMessage m = new SimpleMessage(entry.getIndex() + " OK"); return collecting.collect(Collecting.Type.APPLY_TRANSACTION, m); } + private LogEntryProto copy(LogEntryProto log) { + try { + return LogEntryProto.parseFrom(log.toByteString()); + } catch (InvalidProtocolBufferException e) { + throw new IllegalStateException("Error copying log entry", e); + } + } + @Override public long takeSnapshot() { final TermIndex termIndex = getLastAppliedTermIndex(); From 9df8fda6ff1360f864a052dadb60cdeef19d4b35 Mon Sep 17 00:00:00 2001 From: Duong Nguyen Date: Wed, 31 Jan 2024 09:23:29 -0800 Subject: [PATCH 334/397] RATIS-1978. Add tests assertions to verify all zero-copy messages are released properly (#1023) (cherry picked from commit c46da2213327ea89e9d91b351aee654a50c70da9) --- .../org/apache/ratis/util/LeakDetector.java | 139 +++++++++++ .../util/ReferenceCountedLeakDetector.java | 222 ++++++++++++++++++ .../ratis/util/ReferenceCountedObject.java | 54 +---- .../ratis/grpc/metrics/ZeroCopyMetrics.java | 15 ++ .../apache/ratis/grpc/server/GrpcService.java | 6 + .../ratis/grpc/MiniRaftClusterWithGrpc.java | 32 ++- .../org/apache/ratis/LogAppenderTests.java | 1 + .../apache/ratis/RaftExceptionBaseTest.java | 6 +- .../java/org/apache/ratis/RaftTestUtil.java | 14 ++ .../server/impl/LeaderElectionTests.java | 3 +- .../ratis/server/impl/MiniRaftCluster.java | 9 + 11 files changed, 452 insertions(+), 49 deletions(-) create mode 100644 ratis-common/src/main/java/org/apache/ratis/util/LeakDetector.java create mode 100644 ratis-common/src/main/java/org/apache/ratis/util/ReferenceCountedLeakDetector.java diff --git a/ratis-common/src/main/java/org/apache/ratis/util/LeakDetector.java b/ratis-common/src/main/java/org/apache/ratis/util/LeakDetector.java new file mode 100644 index 0000000000..d801868163 --- /dev/null +++ b/ratis-common/src/main/java/org/apache/ratis/util/LeakDetector.java @@ -0,0 +1,139 @@ +/* + * 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.ratis.util; + +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.lang.ref.ReferenceQueue; +import java.lang.ref.WeakReference; +import java.util.Collections; +import java.util.Set; +import java.util.concurrent.ConcurrentHashMap; +import java.util.concurrent.atomic.AtomicLong; + +/** + * Simple general resource leak detector using {@link ReferenceQueue} and {@link java.lang.ref.WeakReference} to + * observe resource object life-cycle and assert proper resource closure before they are GCed. + * + *

+ * Example usage: + * + *

 {@code
+ * class MyResource implements AutoClosable {
+ *   static final LeakDetector LEAK_DETECTOR = new LeakDetector("MyResource");
+ *
+ *   private final UncheckedAutoCloseable leakTracker = LEAK_DETECTOR.track(this, () -> {
+ *      // report leaks, don't refer to the original object (MyResource) here.
+ *      System.out.println("MyResource is not closed before being discarded.");
+ *   });
+ *
+ *   @Override
+ *   public void close() {
+ *     // proper resources cleanup...
+ *     // inform tracker that this object is closed properly.
+ *     leakTracker.close();
+ *   }
+ * }
+ *
+ * }
+ */ +public class LeakDetector { + private static final Logger LOG = LoggerFactory.getLogger(LeakDetector.class); + private static final AtomicLong COUNTER = new AtomicLong(); + + private final ReferenceQueue queue = new ReferenceQueue<>(); + private final Set allLeaks = Collections.newSetFromMap(new ConcurrentHashMap<>()); + private final String name; + + public LeakDetector(String name) { + this.name = name + COUNTER.getAndIncrement(); + } + + LeakDetector start() { + Thread t = new Thread(this::run); + t.setName(LeakDetector.class.getSimpleName() + "-" + name); + t.setDaemon(true); + LOG.info("Starting leak detector thread {}.", name); + t.start(); + return this; + } + + private void run() { + while (true) { + try { + LeakTracker tracker = (LeakTracker) queue.remove(); + // Original resource already been GCed, if tracker is not closed yet, + // report a leak. + if (allLeaks.remove(tracker)) { + tracker.reportLeak(); + } + } catch (InterruptedException e) { + LOG.warn("Thread interrupted, exiting.", e); + break; + } + } + + LOG.warn("Exiting leak detector {}.", name); + } + + public UncheckedAutoCloseable track(Object leakable, Runnable reportLeak) { + // A rate filter can be put here to only track a subset of all objects, e.g. 5%, 10%, + // if we have proofs that leak tracking impacts performance, or a single LeakDetector + // thread can't keep up with the pace of object allocation. + // For now, it looks effective enough and let keep it simple. + LeakTracker tracker = new LeakTracker(leakable, queue, allLeaks, reportLeak); + allLeaks.add(tracker); + return tracker; + } + + public void assertNoLeaks() { + Preconditions.assertTrue(allLeaks.isEmpty(), this::allLeaksString); + } + + String allLeaksString() { + if (allLeaks.isEmpty()) { + return "allLeaks = "; + } + allLeaks.forEach(LeakTracker::reportLeak); + return "allLeaks.size = " + allLeaks.size(); + } + + private static final class LeakTracker extends WeakReference implements UncheckedAutoCloseable { + private final Set allLeaks; + private final Runnable leakReporter; + LeakTracker(Object referent, ReferenceQueue referenceQueue, + Set allLeaks, Runnable leakReporter) { + super(referent, referenceQueue); + this.allLeaks = allLeaks; + this.leakReporter = leakReporter; + } + + /** + * Called by the tracked resource when closing. + */ + @Override + public void close() { + allLeaks.remove(this); + } + + void reportLeak() { + leakReporter.run(); + } + } +} diff --git a/ratis-common/src/main/java/org/apache/ratis/util/ReferenceCountedLeakDetector.java b/ratis-common/src/main/java/org/apache/ratis/util/ReferenceCountedLeakDetector.java new file mode 100644 index 0000000000..32abe805f1 --- /dev/null +++ b/ratis-common/src/main/java/org/apache/ratis/util/ReferenceCountedLeakDetector.java @@ -0,0 +1,222 @@ +/* + * 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.ratis.util; + +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.util.LinkedList; +import java.util.List; +import java.util.concurrent.atomic.AtomicInteger; +import java.util.concurrent.atomic.AtomicReference; +import java.util.function.Consumer; +import java.util.function.Supplier; + +/** + * A utility to detect leaks from @{@link ReferenceCountedObject}. + */ +public final class ReferenceCountedLeakDetector { + private static final Logger LOG = LoggerFactory.getLogger(ReferenceCountedLeakDetector.class); + // Leak detection is turned off by default. + + private static final AtomicReference FACTORY = new AtomicReference<>(Mode.NONE); + private static final Supplier SUPPLIER + = MemoizedSupplier.valueOf(() -> new LeakDetector(FACTORY.get().name()).start()); + + static Factory getFactory() { + return FACTORY.get(); + } + + public static LeakDetector getLeakDetector() { + return SUPPLIER.get(); + } + + private ReferenceCountedLeakDetector() { + } + + static synchronized void enable(boolean advanced) { + FACTORY.set(advanced ? Mode.ADVANCED : Mode.SIMPLE); + } + + interface Factory { + ReferenceCountedObject create(V value, Runnable retainMethod, Consumer releaseMethod); + } + + private enum Mode implements Factory { + /** Leak detector is not enable in production to avoid performance impacts. */ + NONE { + @Override + public ReferenceCountedObject create(V value, Runnable retainMethod, Consumer releaseMethod) { + return new Impl<>(value, retainMethod, releaseMethod); + } + }, + /** Leak detector is enabled to detect leaks. This is intended to use in every tests. */ + SIMPLE { + @Override + public ReferenceCountedObject create(V value, Runnable retainMethod, Consumer releaseMethod) { + return new SimpleTracing<>(value, retainMethod, releaseMethod, getLeakDetector()); + } + }, + /** + * Leak detector is enabled to detect leaks and report object creation stacktrace as well as every retain and + * release stacktraces. This has severe impact in performance and only used to debug specific test cases. + */ + ADVANCED { + @Override + public ReferenceCountedObject create(V value, Runnable retainMethod, Consumer releaseMethod) { + return new AdvancedTracing<>(value, retainMethod, releaseMethod, getLeakDetector()); + } + } + } + + private static class Impl implements ReferenceCountedObject { + private final AtomicInteger count; + private final V value; + private final Runnable retainMethod; + private final Consumer releaseMethod; + + Impl(V value, Runnable retainMethod, Consumer releaseMethod) { + this.value = value; + this.retainMethod = retainMethod; + this.releaseMethod = releaseMethod; + count = new AtomicInteger(); + } + + @Override + public V get() { + final int previous = count.get(); + if (previous < 0) { + throw new IllegalStateException("Failed to get: object has already been completely released."); + } else if (previous == 0) { + throw new IllegalStateException("Failed to get: object has not yet been retained."); + } + return value; + } + + @Override + public V retain() { + // n < 0: exception + // n >= 0: n++ + if (count.getAndUpdate(n -> n < 0? n : n + 1) < 0) { + throw new IllegalStateException("Failed to retain: object has already been completely released."); + } + + retainMethod.run(); + return value; + } + + @Override + public boolean release() { + // n <= 0: exception + // n > 1: n-- + // n == 1: n = -1 + final int previous = count.getAndUpdate(n -> n <= 1? -1: n - 1); + if (previous < 0) { + throw new IllegalStateException("Failed to release: object has already been completely released."); + } else if (previous == 0) { + throw new IllegalStateException("Failed to release: object has not yet been retained."); + } + final boolean completedReleased = previous == 1; + releaseMethod.accept(completedReleased); + return completedReleased; + } + } + + private static class SimpleTracing extends Impl { + private final UncheckedAutoCloseable leakTracker; + + SimpleTracing(T value, Runnable retainMethod, Consumer releaseMethod, LeakDetector leakDetector) { + super(value, retainMethod, releaseMethod); + final Class clazz = value.getClass(); + this.leakTracker = leakDetector.track(this, + () -> LOG.warn("LEAK: A {} is not released properly", clazz.getName())); + } + + @Override + public boolean release() { + boolean released = super.release(); + if (released) { + leakTracker.close(); + } + return released; + } + } + + private static class AdvancedTracing extends Impl { + private final UncheckedAutoCloseable leakTracker; + private final List retainsTraces; + private final List releaseTraces; + + AdvancedTracing(T value, Runnable retainMethod, Consumer releaseMethod, LeakDetector leakDetector) { + super(value, retainMethod, releaseMethod); + + StackTraceElement[] createStrace = Thread.currentThread().getStackTrace(); + final Class clazz = value.getClass(); + final List localRetainsTraces = new LinkedList<>(); + final List localReleaseTraces = new LinkedList<>(); + + this.leakTracker = leakDetector.track(this, () -> + LOG.warn("LEAK: A {} is not released properly.\nCreation trace:\n{}\n" + + "Retain traces({}):\n{}\nRelease traces({}):\n{}", + clazz.getName(), formatStackTrace(createStrace, 3), + localRetainsTraces.size(), formatStackTraces(localRetainsTraces, 2), + localReleaseTraces.size(), formatStackTraces(localReleaseTraces, 2))); + + this.retainsTraces = localRetainsTraces; + this.releaseTraces = localReleaseTraces; + } + + @Override + public T retain() { + T retain = super.retain(); + retainsTraces.add(Thread.currentThread().getStackTrace()); + return retain; + } + + @Override + public boolean release() { + boolean released = super.release(); + if (released) { + leakTracker.close(); + } + releaseTraces.add(Thread.currentThread().getStackTrace()); + return released; + } + } + + private static String formatStackTrace(StackTraceElement[] stackTrace, int startIdx) { + final StringBuilder sb = new StringBuilder(); + for (int line = startIdx; line < stackTrace.length; line++) { + sb.append(stackTrace[line]).append("\n"); + } + return sb.toString(); + } + + private static String formatStackTraces(List stackTraces, int startIdx) { + final StringBuilder sb = new StringBuilder(); + stackTraces.forEach(stackTrace -> { + if (sb.length() > 0) { + sb.append("\n"); + } + for (int line = startIdx; line < stackTrace.length; line++) { + sb.append(stackTrace[line]).append("\n"); + } + }); + return sb.toString(); + } +} diff --git a/ratis-common/src/main/java/org/apache/ratis/util/ReferenceCountedObject.java b/ratis-common/src/main/java/org/apache/ratis/util/ReferenceCountedObject.java index 5137e5912c..c3851a712d 100644 --- a/ratis-common/src/main/java/org/apache/ratis/util/ReferenceCountedObject.java +++ b/ratis-common/src/main/java/org/apache/ratis/util/ReferenceCountedObject.java @@ -22,7 +22,6 @@ import java.util.Collection; import java.util.Objects; import java.util.concurrent.atomic.AtomicBoolean; -import java.util.concurrent.atomic.AtomicInteger; import java.util.function.Consumer; /** @@ -45,6 +44,7 @@ * @param The object type. */ public interface ReferenceCountedObject { + /** @return the object. */ T get(); @@ -143,52 +143,20 @@ static ReferenceCountedObject wrap(V value, Runnable retainMethod, Consum Objects.requireNonNull(retainMethod, "retainMethod == null"); Objects.requireNonNull(releaseMethod, "releaseMethod == null"); - return new ReferenceCountedObject() { - private final AtomicInteger count = new AtomicInteger(); - - @Override - public V get() { - final int previous = count.get(); - if (previous < 0) { - throw new IllegalStateException("Failed to get: object has already been completely released."); - } else if (previous == 0) { - throw new IllegalStateException("Failed to get: object has not yet been retained."); - } - return value; - } - - @Override - public V retain() { - // n < 0: exception - // n >= 0: n++ - if (count.getAndUpdate(n -> n < 0? n : n + 1) < 0) { - throw new IllegalStateException("Failed to retain: object has already been completely released."); - } - - retainMethod.run(); - return value; - } - - @Override - public boolean release() { - // n <= 0: exception - // n > 1: n-- - // n == 1: n = -1 - final int previous = count.getAndUpdate(n -> n <= 1? -1: n - 1); - if (previous < 0) { - throw new IllegalStateException("Failed to release: object has already been completely released."); - } else if (previous == 0) { - throw new IllegalStateException("Failed to release: object has not yet been retained."); - } - final boolean completedReleased = previous == 1; - releaseMethod.accept(completedReleased); - return completedReleased; - } - }; + return ReferenceCountedLeakDetector.getFactory().create(value, retainMethod, releaseMethod); } /** The same as wrap(value, retainMethod, ignored -> releaseMethod.run()). */ static ReferenceCountedObject wrap(V value, Runnable retainMethod, Runnable releaseMethod) { return wrap(value, retainMethod, ignored -> releaseMethod.run()); } + + static void enableLeakDetection() { + ReferenceCountedLeakDetector.enable(false); + } + + static void enableAdvancedLeakDetection() { + ReferenceCountedLeakDetector.enable(true); + } + } diff --git a/ratis-grpc/src/main/java/org/apache/ratis/grpc/metrics/ZeroCopyMetrics.java b/ratis-grpc/src/main/java/org/apache/ratis/grpc/metrics/ZeroCopyMetrics.java index 20da4ee634..fec2135a98 100644 --- a/ratis-grpc/src/main/java/org/apache/ratis/grpc/metrics/ZeroCopyMetrics.java +++ b/ratis-grpc/src/main/java/org/apache/ratis/grpc/metrics/ZeroCopyMetrics.java @@ -21,6 +21,7 @@ import org.apache.ratis.metrics.MetricRegistryInfo; import org.apache.ratis.metrics.RatisMetricRegistry; import org.apache.ratis.metrics.RatisMetrics; +import org.apache.ratis.thirdparty.com.google.common.annotations.VisibleForTesting; import org.apache.ratis.thirdparty.com.google.protobuf.AbstractMessage; public class ZeroCopyMetrics extends RatisMetrics { @@ -55,4 +56,18 @@ public void onReleasedMessage(AbstractMessage ignored) { releasedMessages.inc(); } + @VisibleForTesting + public long zeroCopyMessages() { + return zeroCopyMessages.getCount(); + } + + @VisibleForTesting + public long nonZeroCopyMessages() { + return nonZeroCopyMessages.getCount(); + } + + @VisibleForTesting + public long releasedMessages() { + return releasedMessages.getCount(); + } } \ No newline at end of file diff --git a/ratis-grpc/src/main/java/org/apache/ratis/grpc/server/GrpcService.java b/ratis-grpc/src/main/java/org/apache/ratis/grpc/server/GrpcService.java index d2dadcd90a..fa93586040 100644 --- a/ratis-grpc/src/main/java/org/apache/ratis/grpc/server/GrpcService.java +++ b/ratis-grpc/src/main/java/org/apache/ratis/grpc/server/GrpcService.java @@ -30,6 +30,7 @@ import org.apache.ratis.server.RaftServerConfigKeys; import org.apache.ratis.server.RaftServerRpcWithProxy; import org.apache.ratis.server.protocol.RaftServerAsynchronousProtocol; +import org.apache.ratis.thirdparty.com.google.common.annotations.VisibleForTesting; import org.apache.ratis.thirdparty.io.grpc.ServerInterceptors; import org.apache.ratis.thirdparty.io.grpc.netty.GrpcSslContexts; import org.apache.ratis.thirdparty.io.grpc.netty.NettyServerBuilder; @@ -328,6 +329,7 @@ public void closeImpl() throws IOException { serverInterceptor.close(); ConcurrentUtils.shutdownAndWait(executor); + zeroCopyMetrics.unregister(); } @Override @@ -385,4 +387,8 @@ public StartLeaderElectionReplyProto startLeaderElection(StartLeaderElectionRequ return getProxies().getProxy(target).startLeaderElection(request); } + @VisibleForTesting + public ZeroCopyMetrics getZeroCopyMetrics() { + return zeroCopyMetrics; + } } diff --git a/ratis-grpc/src/test/java/org/apache/ratis/grpc/MiniRaftClusterWithGrpc.java b/ratis-grpc/src/test/java/org/apache/ratis/grpc/MiniRaftClusterWithGrpc.java index 0e4eb55544..20ac883589 100644 --- a/ratis-grpc/src/test/java/org/apache/ratis/grpc/MiniRaftClusterWithGrpc.java +++ b/ratis-grpc/src/test/java/org/apache/ratis/grpc/MiniRaftClusterWithGrpc.java @@ -21,14 +21,19 @@ import org.apache.ratis.RaftTestUtil; import org.apache.ratis.conf.Parameters; import org.apache.ratis.conf.RaftProperties; -import org.apache.ratis.grpc.server.GrpcServicesImpl; +import org.apache.ratis.grpc.metrics.ZeroCopyMetrics; +import org.apache.ratis.grpc.server.GrpcService; import org.apache.ratis.protocol.RaftGroup; import org.apache.ratis.protocol.RaftPeer; import org.apache.ratis.protocol.RaftPeerId; import org.apache.ratis.rpc.SupportedRpcType; +import org.apache.ratis.server.RaftServer; import org.apache.ratis.server.impl.DelayLocalExecutionInjection; import org.apache.ratis.server.impl.MiniRaftCluster; +import org.apache.ratis.server.impl.RaftServerTestUtil; import org.apache.ratis.util.NetUtils; +import org.apache.ratis.util.ReferenceCountedObject; +import org.junit.Assert; import java.util.Optional; @@ -45,6 +50,10 @@ public MiniRaftClusterWithGrpc newCluster(String[] ids, String[] listenerIds, Ra } }; + static { + ReferenceCountedObject.enableLeakDetection(); + } + public interface FactoryGet extends Factory.Get { @Override default Factory getFactory() { @@ -59,7 +68,8 @@ public MiniRaftClusterWithGrpc(String[] ids, RaftProperties properties, Paramete this(ids, new String[0], properties, parameters); } - public MiniRaftClusterWithGrpc(String[] ids, String[] listenerIds, RaftProperties properties, Parameters parameters) { + protected MiniRaftClusterWithGrpc(String[] ids, String[] listenerIds, RaftProperties properties, + Parameters parameters) { super(ids, listenerIds, properties, parameters); } @@ -79,4 +89,22 @@ protected void blockQueueAndSetDelay(String leaderId, int delayMs) RaftTestUtil.blockQueueAndSetDelay(getServers(), SEND_SERVER_REQUEST_INJECTION, leaderId, delayMs, getTimeoutMax()); } + + @Override + public void shutdown() { + super.shutdown(); + assertZeroCopyMetrics(); + } + + public void assertZeroCopyMetrics() { + getServers().forEach(server -> server.getGroupIds().forEach(id -> { + LOG.info("Checking {}-{}", server.getId(), id); + RaftServer.Division division = RaftServerTestUtil.getDivision(server, id); + GrpcService service = (GrpcService) RaftServerTestUtil.getServerRpc(division); + ZeroCopyMetrics zeroCopyMetrics = service.getZeroCopyMetrics(); + Assert.assertEquals(0, zeroCopyMetrics.nonZeroCopyMessages()); + Assert.assertEquals("Zero copy messages are not released, please check logs to find leaks. ", + zeroCopyMetrics.zeroCopyMessages(), zeroCopyMetrics.releasedMessages()); + })); + } } diff --git a/ratis-server/src/test/java/org/apache/ratis/LogAppenderTests.java b/ratis-server/src/test/java/org/apache/ratis/LogAppenderTests.java index 8a8731daf4..dc9cd8c0db 100644 --- a/ratis-server/src/test/java/org/apache/ratis/LogAppenderTests.java +++ b/ratis-server/src/test/java/org/apache/ratis/LogAppenderTests.java @@ -170,6 +170,7 @@ public void testFollowerHeartbeatMetric() throws IOException, InterruptedExcepti assertTrue(t.getTimer().getMeanRate() > 0.0d); assertTrue(t.getTimer().getCount() > 0L); } + cluster.shutdown(); } cluster.shutdown(); } diff --git a/ratis-server/src/test/java/org/apache/ratis/RaftExceptionBaseTest.java b/ratis-server/src/test/java/org/apache/ratis/RaftExceptionBaseTest.java index b06d6e904a..65e8063251 100644 --- a/ratis-server/src/test/java/org/apache/ratis/RaftExceptionBaseTest.java +++ b/ratis-server/src/test/java/org/apache/ratis/RaftExceptionBaseTest.java @@ -138,7 +138,7 @@ void runTestNotLeaderExceptionWithReconf(CLUSTER cluster) throws Exception { @Test public void testGroupMismatchException() throws Exception { - runWithSameCluster(NUM_PEERS, this::runTestGroupMismatchException); + runWithNewCluster(NUM_PEERS, this::runTestGroupMismatchException); } void runTestGroupMismatchException(CLUSTER cluster) throws Exception { @@ -171,7 +171,7 @@ void runTestGroupMismatchException(CLUSTER cluster) throws Exception { @Test public void testStaleReadException() throws Exception { - runWithSameCluster(NUM_PEERS, this::runTestStaleReadException); + runWithNewCluster(NUM_PEERS, this::runTestStaleReadException); } void runTestStaleReadException(CLUSTER cluster) throws Exception { @@ -186,7 +186,7 @@ void runTestStaleReadException(CLUSTER cluster) throws Exception { @Test public void testLogAppenderBufferCapacity() throws Exception { - runWithSameCluster(NUM_PEERS, this::runTestLogAppenderBufferCapacity); + runWithNewCluster(NUM_PEERS, this::runTestLogAppenderBufferCapacity); } void runTestLogAppenderBufferCapacity(CLUSTER cluster) throws Exception { diff --git a/ratis-server/src/test/java/org/apache/ratis/RaftTestUtil.java b/ratis-server/src/test/java/org/apache/ratis/RaftTestUtil.java index 98c2113eb5..b1a68b8816 100644 --- a/ratis-server/src/test/java/org/apache/ratis/RaftTestUtil.java +++ b/ratis-server/src/test/java/org/apache/ratis/RaftTestUtil.java @@ -606,4 +606,18 @@ static void gc() throws InterruptedException { Thread.sleep(100); } } + + static void gc() throws InterruptedException { + // use WeakReference to detect gc + Object obj = new Object(); + final WeakReference weakRef = new WeakReference<>(obj); + obj = null; + + // loop until gc has completed. + for (int i = 0; weakRef.get() != null; i++) { + LOG.info("gc {}", i); + System.gc(); + Thread.sleep(100); + } + } } diff --git a/ratis-server/src/test/java/org/apache/ratis/server/impl/LeaderElectionTests.java b/ratis-server/src/test/java/org/apache/ratis/server/impl/LeaderElectionTests.java index 3a47d127c5..13b23dcecb 100644 --- a/ratis-server/src/test/java/org/apache/ratis/server/impl/LeaderElectionTests.java +++ b/ratis-server/src/test/java/org/apache/ratis/server/impl/LeaderElectionTests.java @@ -427,7 +427,7 @@ public void testLateServerStart() throws Exception { .orElseThrow(() -> new IllegalStateException("No leader yet")), 10, ONE_SECOND, "getLeaderId", LOG); LOG.info(cluster.printServers()); - assertEquals(leader.getId(), lastServerLeaderId); + Assert.assertEquals(leader.getId(), lastServerLeaderId); cluster.shutdown(); } @@ -592,6 +592,7 @@ public void testLeaderElectionMetrics() throws IOException, InterruptedException Long leaderElectionLatency = (Long) ratisMetricRegistry.getGauges((s, metric) -> s.contains(LAST_LEADER_ELECTION_ELAPSED_TIME)).values().iterator().next().getValue(); assertTrue(leaderElectionLatency > 0L); + cluster.shutdown(); } @Test diff --git a/ratis-server/src/test/java/org/apache/ratis/server/impl/MiniRaftCluster.java b/ratis-server/src/test/java/org/apache/ratis/server/impl/MiniRaftCluster.java index 825ae8f55b..b47d13e814 100644 --- a/ratis-server/src/test/java/org/apache/ratis/server/impl/MiniRaftCluster.java +++ b/ratis-server/src/test/java/org/apache/ratis/server/impl/MiniRaftCluster.java @@ -49,6 +49,7 @@ import org.apache.ratis.util.JavaUtils; import org.apache.ratis.util.NetUtils; import org.apache.ratis.util.Preconditions; +import org.apache.ratis.util.ReferenceCountedLeakDetector; import org.apache.ratis.util.ReflectionUtils; import org.apache.ratis.util.TimeDuration; import org.apache.ratis.util.function.CheckedConsumer; @@ -822,6 +823,14 @@ public void shutdown() { Optional.ofNullable(timer.get()).ifPresent(Timer::cancel); ExitUtils.assertNotTerminated(); LOG.info("{} shutdown completed", JavaUtils.getClassSimpleName(getClass())); + + // GC to ensure leak detection work. + try { + RaftTestUtil.gc(); + } catch (InterruptedException e) { + LOG.info("gc interrupted."); + } + ReferenceCountedLeakDetector.getLeakDetector().assertNoLeaks(); } /** From e63011e1aba086b6385bf315fdfb9755cd456c1a Mon Sep 17 00:00:00 2001 From: Nandakumar Vadivelu Date: Thu, 1 Feb 2024 22:27:19 +0530 Subject: [PATCH 335/397] RATIS-1976. Migrate ratis-test tests to Junit 5 - Part 1 (#1038) (cherry picked from commit e44de26623a6b65d9d66eb782df70ffd98fab364) --- .../ratis/InstallSnapshotFromLeaderTests.java | 97 +++--------- .../server/impl/LeaderElectionTests.java | 140 +++++++++--------- ratis-test/pom.xml | 8 +- .../DataStreamAsyncClusterTests.java | 1 - .../ratis/datastream/DataStreamTestUtils.java | 9 -- .../apache/ratis/grpc/TestRaftWithGrpc.java | 24 ++- .../grpc/server/TestGrpcMessageMetrics.java | 6 +- .../ratis/grpc/util/TestGrpcZeroCopy.java | 18 +-- 8 files changed, 109 insertions(+), 194 deletions(-) diff --git a/ratis-server/src/test/java/org/apache/ratis/InstallSnapshotFromLeaderTests.java b/ratis-server/src/test/java/org/apache/ratis/InstallSnapshotFromLeaderTests.java index e7683a3991..f873e5dfdc 100644 --- a/ratis-server/src/test/java/org/apache/ratis/InstallSnapshotFromLeaderTests.java +++ b/ratis-server/src/test/java/org/apache/ratis/InstallSnapshotFromLeaderTests.java @@ -102,15 +102,18 @@ private void testMultiFileInstallSnapshot(CLUSTER cluster) throws Exception { RaftTestUtil.waitForLeader(cluster); final RaftPeerId leaderId = cluster.getLeader().getId(); - try (final RaftClient client = cluster.createClient(leaderId)) { - for (; i < SNAPSHOT_TRIGGER_THRESHOLD * 2 - 1; i++) { - RaftClientReply - reply = client.io().send(new RaftTestUtil.SimpleMessage("m" + i)); - Assertions.assertTrue(reply.isSuccess()); + try (final RaftClient client = cluster.createClient(leaderId)) { + for (; i < SNAPSHOT_TRIGGER_THRESHOLD * 2 - 1; i++) { + RaftClientReply + reply = client.io().send(new RaftTestUtil.SimpleMessage("m" + i)); + Assertions.assertTrue(reply.isSuccess()); + } + + client.getSnapshotManagementApi(leaderId).create(3000); } - client.getSnapshotManagementApi(leaderId).create(3000); - } + final SnapshotInfo snapshot = cluster.getLeader().getStateMachine().getLatestSnapshot(); + Assertions.assertEquals(3, snapshot.getFiles().size()); final SnapshotInfo snapshot = cluster.getLeader().getStateMachine().getLatestSnapshot(); Assertions.assertEquals(3, snapshot.getFiles().size()); @@ -120,75 +123,17 @@ private void testMultiFileInstallSnapshot(CLUSTER cluster) throws Exception { // trigger setConfiguration cluster.setConfiguration(change.getPeersInNewConf()); - RaftServerTestUtil.waitAndCheckNewConf(cluster, change.getPeersInNewConf(), 0, null); - - // Check the installed snapshot file number on each Follower matches with the - // leader snapshot. - JavaUtils.attempt(() -> { - for (RaftServer.Division follower : cluster.getFollowers()) { - final SnapshotInfo info = follower.getStateMachine().getLatestSnapshot(); - Assertions.assertNotNull(info); - Assertions.assertEquals(3, info.getFiles().size()); - } - }, 10, ONE_SECOND, "check snapshot", LOG); - } - - private void testInstallSnapshotDuringLeaderSwitch(CLUSTER cluster) throws Exception { - RaftTestUtil.waitForLeader(cluster); - final RaftPeerId leaderId = cluster.getLeader().getId(); - - // perform operations and force all peers to take snapshot - try (final RaftClient client = cluster.createClient(leaderId)) { - for (int i = 0; i < SNAPSHOT_TRIGGER_THRESHOLD * 2; i++) { - final RaftClientReply - reply = client.io().send(new RaftTestUtil.SimpleMessage("m" + i)); - Assertions.assertTrue(reply.isSuccess()); - } - - for (final RaftPeer peer: cluster.getPeers()) { - final RaftClientReply snapshotReply = client.getSnapshotManagementApi(leaderId).create(3000); - Assertions.assertTrue(snapshotReply.isSuccess()); - } - } - final SnapshotInfo snapshot = cluster.getLeader().getStateMachine().getLatestSnapshot(); - Assertions.assertNotNull(snapshot); - - // isolate two followers (majority) in old configuration - final List oldFollowers = cluster.getFollowers(); - for (RaftServer.Division f: oldFollowers) { - RaftTestUtil.isolate(cluster, f.getId()); - } - - // add two more peers and install snapshot from leaders - final PeerChanges change = cluster.addNewPeers(2, true); - try (final RaftClient client = cluster.createClient(leaderId, RetryPolicies.noRetry())) { - final RaftException e = Assertions.assertThrows(RaftException.class, - () -> client.admin().setConfiguration(change.getPeersInNewConf())); - Assertions.assertTrue( e instanceof RaftRetryFailureException - || e instanceof ReconfigurationTimeoutException, - () -> "Unexpected exception: " + e); - } - - final SnapshotInfo snapshotInfo = cluster.getDivision(change.getAddedPeers().get(0).getId()) - .getStateMachine().getLatestSnapshot(); - Assertions.assertNotNull(snapshotInfo); - - // recover the old followers and isolate the leader to force leader switch - RaftTestUtil.isolate(cluster, leaderId); - for (RaftServer.Division f: oldFollowers) { - RaftTestUtil.deIsolate(cluster, f.getId()); - } - RaftTestUtil.waitForLeader(cluster); - - try (final RaftClient client = cluster.createClient(cluster.getLeader().getId())) { - // successfully setConfiguration during leader switch - final RaftClientReply setConf = client.admin().setConfiguration(change.getPeersInNewConf()); - Assertions.assertTrue(setConf.isSuccess()); - - RaftTestUtil.deIsolate(cluster, leaderId); - final RaftClientReply - reply = client.io().send(new RaftTestUtil.SimpleMessage("final")); - Assertions.assertTrue(reply.isSuccess()); + // Check the installed snapshot file number on each Follower matches with the + // leader snapshot. + JavaUtils.attempt(() -> { + for (RaftServer.Division follower : cluster.getFollowers()) { + final SnapshotInfo info = follower.getStateMachine().getLatestSnapshot(); + Assertions.assertNotNull(info); + Assertions.assertEquals(3, info.getFiles().size()); + } + }, 10, ONE_SECOND, "check snapshot", LOG); + } finally { + cluster.shutdown(); } } diff --git a/ratis-server/src/test/java/org/apache/ratis/server/impl/LeaderElectionTests.java b/ratis-server/src/test/java/org/apache/ratis/server/impl/LeaderElectionTests.java index 13b23dcecb..8f2da0bba3 100644 --- a/ratis-server/src/test/java/org/apache/ratis/server/impl/LeaderElectionTests.java +++ b/ratis-server/src/test/java/org/apache/ratis/server/impl/LeaderElectionTests.java @@ -213,18 +213,18 @@ public void testLeaderNotCountListenerForMajority() throws Exception { void runTestLeaderNotCountListenerForMajority(CLUSTER cluster) throws Exception { final RaftServer.Division leader = waitForLeader(cluster); - assertEquals(2, ((RaftConfigurationImpl)cluster.getLeader().getRaftConf()).getMajorityCount()); + Assertions.assertEquals(2, ((RaftConfigurationImpl)cluster.getLeader().getRaftConf()).getMajorityCount()); try (RaftClient client = cluster.createClient(leader.getId())) { client.io().send(new RaftTestUtil.SimpleMessage("message")); List listeners = cluster.getListeners() .stream().map(RaftServer.Division::getPeer).collect(Collectors.toList()); - assertEquals(2, listeners.size()); + Assertions.assertEquals(2, listeners.size()); RaftClientReply reply = client.admin().setConfiguration(cluster.getPeers()); - assertTrue(reply.isSuccess()); + Assertions.assertTrue(reply.isSuccess()); Collection peer = leader.getRaftConf().getAllPeers(RaftProtos.RaftPeerRole.LISTENER); - assertEquals(0, peer.size()); + Assertions.assertEquals(0, peer.size()); } - assertEquals(3, ((RaftConfigurationImpl)cluster.getLeader().getRaftConf()).getMajorityCount()); + Assertions.assertEquals(3, ((RaftConfigurationImpl)cluster.getLeader().getRaftConf()).getMajorityCount()); } @Test @@ -242,7 +242,7 @@ void runTestListenerNotStartLeaderElection(CLUSTER cluster) throws Exception { RaftTestUtil.isolate(cluster, listenerId); maxTimeout.sleep(); maxTimeout.sleep(); - assertEquals(RaftProtos.RaftPeerRole.LISTENER, listener.getInfo().getCurrentRole()); + Assertions.assertEquals(RaftProtos.RaftPeerRole.LISTENER, listener.getInfo().getCurrentRole()); } finally { RaftTestUtil.deIsolate(cluster, listener.getId()); } @@ -258,18 +258,18 @@ public void testTransferLeader() throws Exception { client.io().send(new RaftTestUtil.SimpleMessage("message")); List followers = cluster.getFollowers(); - assertEquals(2, followers.size()); + Assertions.assertEquals(2, followers.size()); RaftServer.Division newLeader = followers.get(0); RaftClientReply reply = client.admin().transferLeadership(newLeader.getId(), 20000); - assertTrue(reply.isSuccess()); + Assertions.assertTrue(reply.isSuccess()); final RaftServer.Division currLeader = waitForLeader(cluster); - assertEquals(newLeader.getId(), currLeader.getId()); + Assertions.assertEquals(newLeader.getId(), currLeader.getId()); reply = client.io().send(new RaftTestUtil.SimpleMessage("message")); - assertEquals(newLeader.getId().toString(), reply.getReplierId()); - assertTrue(reply.isSuccess()); + Assertions.assertEquals(newLeader.getId().toString(), reply.getReplierId()); + Assertions.assertTrue(reply.isSuccess()); } cluster.shutdown(); @@ -286,24 +286,24 @@ public void testYieldLeaderToHigherPriority() throws Exception { client.io().send(new RaftTestUtil.SimpleMessage("message")); List followers = cluster.getFollowers(); - assertEquals(2, followers.size()); + Assertions.assertEquals(2, followers.size()); RaftServer.Division newLeader = followers.get(0); List peers = cluster.getPeers(); List peersWithNewPriority = getPeersWithPriority(peers, newLeader.getPeer()); RaftClientReply reply = client.admin().setConfiguration(peersWithNewPriority.toArray(new RaftPeer[0])); - assertTrue(reply.isSuccess()); + Assertions.assertTrue(reply.isSuccess()); // Wait the old leader to step down. // TODO: make it more deterministic. TimeDuration.valueOf(1, TimeUnit.SECONDS).sleep(); final RaftServer.Division currLeader = waitForLeader(cluster); - assertEquals(newLeader.getId(), currLeader.getId()); + Assertions.assertEquals(newLeader.getId(), currLeader.getId()); reply = client.io().send(new RaftTestUtil.SimpleMessage("message")); - assertEquals(newLeader.getId().toString(), reply.getReplierId()); - assertTrue(reply.isSuccess()); + Assertions.assertEquals(newLeader.getId().toString(), reply.getReplierId()); + Assertions.assertTrue(reply.isSuccess()); } cluster.shutdown(); @@ -318,7 +318,7 @@ public void testTransferLeaderTimeout() throws Exception { final RaftServer.Division leader = waitForLeader(cluster); try (RaftClient client = cluster.createClient(leader.getId())) { List followers = cluster.getFollowers(); - assertEquals(followers.size(), 2); + Assertions.assertEquals(followers.size(), 2); RaftServer.Division newLeader = followers.get(0); // isolate new leader, so that transfer leadership will timeout @@ -334,9 +334,9 @@ public void testTransferLeaderTimeout() throws Exception { client.admin().transferLeadership(newLeader.getId(), timeoutMs); } catch (TransferLeadershipException e) { long cost = System.currentTimeMillis() - start; - assertTrue(cost > timeoutMs); - assertTrue(e.getMessage().contains("Failed to transfer leadership to")); - assertTrue(e.getMessage().contains(TransferLeadership.Result.Type.TIMED_OUT.toString())); + Assertions.assertTrue(cost > timeoutMs); + Assertions.assertTrue(e.getMessage().contains("Failed to transfer leadership to")); + Assertions.assertTrue(e.getMessage().contains(TransferLeadership.Result.Type.TIMED_OUT.toString())); } return true; @@ -350,17 +350,17 @@ public void testTransferLeaderTimeout() throws Exception { try { client.io().send(new RaftTestUtil.SimpleMessage("message")); } catch (LeaderSteppingDownException e) { - assertTrue(e.getMessage().contains("is stepping down")); + Assertions.assertTrue(e.getMessage().contains("is stepping down")); } return null; }, 5, TimeDuration.ONE_SECOND, "check leader steppingDown", RaftServer.LOG); - assertTrue(transferTimeoutFuture.get()); + Assertions.assertTrue(transferTimeoutFuture.get()); // after transfer timeout, leader should accept request RaftClientReply reply = client.io().send(new RaftTestUtil.SimpleMessage("message")); - assertEquals(leader.getId().toString(), reply.getReplierId()); - assertTrue(reply.isSuccess()); + Assertions.assertEquals(leader.getId().toString(), reply.getReplierId()); + Assertions.assertTrue(reply.isSuccess()); RaftTestUtil.deIsolate(cluster, newLeader.getId()); } @@ -398,7 +398,7 @@ static void enforceLeader(MiniRaftCluster cluster, final String newLeader, Logge log.info(cluster.printServers()); final RaftServer.Division leader = cluster.getLeader(); - assertEquals(newLeader, leader.getId().toString()); + Assertions.assertEquals(newLeader, leader.getId().toString()); } @Test @@ -427,7 +427,7 @@ public void testLateServerStart() throws Exception { .orElseThrow(() -> new IllegalStateException("No leader yet")), 10, ONE_SECOND, "getLeaderId", LOG); LOG.info(cluster.printServers()); - Assert.assertEquals(leader.getId(), lastServerLeaderId); + Assertions.assertEquals(leader.getId(), lastServerLeaderId); cluster.shutdown(); } @@ -442,7 +442,7 @@ protected void testDisconnectLeader() throws Exception { RaftTestUtil.isolate(cluster, leader.getId()); RaftClientReply reply = client.io().send(new RaftTestUtil.SimpleMessage("message")); Assertions.assertNotEquals(reply.getReplierId(), leader.getId().toString()); - assertTrue(reply.isSuccess()); + Assertions.assertTrue(reply.isSuccess()); } finally { RaftTestUtil.deIsolate(cluster, leader.getId()); } @@ -459,15 +459,15 @@ public void testAddListener() throws Exception { try (RaftClient client = cluster.createClient(leader.getId())) { client.io().send(new RaftTestUtil.SimpleMessage("message")); List servers = cluster.getPeers(); - assertEquals(servers.size(), 3); - final PeerChanges changes = cluster.addNewPeers(1, true); - final List added = changes.getAddedPeers(); - final RaftClientReply reply = client.admin().setConfiguration(servers, added); - assertTrue(reply.isSuccess()); + Assertions.assertEquals(servers.size(), 3); + MiniRaftCluster.PeerChanges changes = cluster.addNewPeers(1, + true, false, RaftProtos.RaftPeerRole.LISTENER); + RaftClientReply reply = client.admin().setConfiguration(servers, Arrays.asList(changes.newPeers)); + Assertions.assertTrue(reply.isSuccess()); Collection listener = leader.getRaftConf().getAllPeers(RaftProtos.RaftPeerRole.LISTENER); - assertEquals(1, listener.size()); - assertEquals(added.get(0).getId(), listener.iterator().next().getId()); + Assertions.assertEquals(1, listener.size()); + Assertions.assertEquals(changes.newPeers[0].getId(), new ArrayList<>(listener).get(0).getId()); } cluster.shutdown(); } @@ -481,18 +481,18 @@ public void testAddFollowerWhenExistsListener() throws Exception { try (RaftClient client = cluster.createClient(leader.getId())) { client.io().send(new RaftTestUtil.SimpleMessage("message")); List servers = cluster.getPeers(); - assertEquals(4, servers.size()); + Assertions.assertEquals(4, servers.size()); List listener = new ArrayList<>( leader.getRaftConf().getAllPeers(RaftProtos.RaftPeerRole.LISTENER)); - assertEquals(1, listener.size()); - final PeerChanges changes = cluster.addNewPeers(1, true); - final List newPeers = new ArrayList<>(changes.getAddedPeers()); + Assertions.assertEquals(1, listener.size()); + MiniRaftCluster.PeerChanges changes = cluster.addNewPeers(1, true, false); + ArrayList newPeers = new ArrayList<>(Arrays.asList(changes.newPeers)); newPeers.addAll(leader.getRaftConf().getAllPeers(RaftProtos.RaftPeerRole.FOLLOWER)); RaftClientReply reply = client.admin().setConfiguration(newPeers, listener); - assertTrue(reply.isSuccess()); - assertEquals(4, + Assertions.assertTrue(reply.isSuccess()); + Assertions.assertEquals(4, leader.getRaftConf().getAllPeers(RaftProtos.RaftPeerRole.FOLLOWER).size()); - assertEquals(1, + Assertions.assertEquals(1, leader.getRaftConf().getAllPeers(RaftProtos.RaftPeerRole.LISTENER).size()); } cluster.shutdown(); @@ -506,13 +506,13 @@ public void testRemoveListener() throws Exception { final RaftServer.Division leader = waitForLeader(cluster); try (RaftClient client = cluster.createClient(leader.getId())) { client.io().send(new RaftTestUtil.SimpleMessage("message")); - assertEquals(1, cluster.getListeners().size()); + Assertions.assertEquals(1, cluster.getListeners().size()); List servers = cluster.getFollowers().stream().map(RaftServer.Division::getPeer).collect( Collectors.toList()); servers.add(leader.getPeer()); RaftClientReply reply = client.admin().setConfiguration(servers); - assertTrue(reply.isSuccess()); - assertEquals(0, leader.getRaftConf().getAllPeers(RaftProtos.RaftPeerRole.LISTENER).size()); + Assertions.assertTrue(reply.isSuccess()); + Assertions.assertEquals(0, leader.getRaftConf().getAllPeers(RaftProtos.RaftPeerRole.LISTENER).size()); } cluster.shutdown(); } @@ -527,15 +527,15 @@ public void testChangeFollowerToListener() throws Exception { client.io().send(new RaftTestUtil.SimpleMessage("message")); List followers = cluster.getFollowers().stream().map( RaftServer.Division::getPeer).collect(Collectors.toList()); - assertEquals(2, followers.size()); + Assertions.assertEquals(2, followers.size()); List listeners = new ArrayList<>(); listeners.add(followers.get(1)); followers.remove(1); RaftClientReply reply = client.admin().setConfiguration(followers, listeners); - assertTrue(reply.isSuccess()); + Assertions.assertTrue(reply.isSuccess()); Collection peer = leader.getRaftConf().getAllPeers(RaftProtos.RaftPeerRole.LISTENER); - assertEquals(1, peer.size()); - assertEquals(listeners.get(0).getId(), new ArrayList<>(peer).get(0).getId()); + Assertions.assertEquals(1, peer.size()); + Assertions.assertEquals(listeners.get(0).getId(), new ArrayList<>(peer).get(0).getId()); } cluster.shutdown(); } @@ -550,17 +550,11 @@ public void testChangeListenerToFollower() throws Exception { client.io().send(new RaftTestUtil.SimpleMessage("message")); List listeners = cluster.getListeners() .stream().map(RaftServer.Division::getPeer).collect(Collectors.toList()); - assertEquals(listeners.size(), 1); + Assertions.assertEquals(listeners.size(), 1); RaftClientReply reply = client.admin().setConfiguration(cluster.getPeers()); - assertTrue(reply.isSuccess()); + Assertions.assertTrue(reply.isSuccess()); Collection peer = leader.getRaftConf().getAllPeers(RaftProtos.RaftPeerRole.LISTENER); - assertEquals(0, peer.size()); - - RaftServerTestUtil.waitAndCheckNewConf(cluster, cluster.getPeers(), 0, null); - - listeners = cluster.getListeners() - .stream().map(RaftServer.Division::getPeer).collect(Collectors.toList()); - assertEquals(0, listeners.size()); + Assertions.assertEquals(0, peer.size()); } cluster.shutdown(); } @@ -641,7 +635,7 @@ public void testPreVote() { RaftTestUtil.isolate(cluster, follower.getId()); // send message so that the isolated follower's log lag the others RaftClientReply reply = client.io().send(new RaftTestUtil.SimpleMessage("message")); - assertTrue(reply.isSuccess()); + Assertions.assertTrue(reply.isSuccess()); final long savedTerm = leader.getInfo().getCurrentTerm(); LOG.info("Wait follower {} timeout and trigger pre-vote", follower.getId()); @@ -656,7 +650,7 @@ public void testPreVote() { assertEquals(savedTerm, leader.getInfo().getCurrentTerm()); reply = client.io().send(new RaftTestUtil.SimpleMessage("message")); - assertTrue(reply.isSuccess()); + Assertions.assertTrue(reply.isSuccess()); } cluster.shutdown(); @@ -691,23 +685,23 @@ void runTestPauseResumeLeaderElection(CLUSTER cluster) throws IOException, Inter final RaftServer.Division leader = RaftTestUtil.waitForLeader(cluster); final RaftPeerId leaderId = leader.getId(); final List followers = cluster.getFollowers(); - assertTrue(followers.size() >= 1); + Assertions.assertTrue(followers.size() >= 1); final RaftServerImpl f1 = (RaftServerImpl)followers.get(0); try (final RaftClient client = cluster.createClient()) { pauseLeaderReply = client.getLeaderElectionManagementApi(f1.getId()).pause(); - assertTrue(pauseLeaderReply.isSuccess()); + Assertions.assertTrue(pauseLeaderReply.isSuccess()); client.io().send(new RaftTestUtil.SimpleMessage("message")); RaftServer.Division newLeader = followers.get(0); List peers = cluster.getPeers(); List peersWithNewPriority = getPeersWithPriority(peers, newLeader.getPeer()); RaftClientReply reply = client.admin().setConfiguration(peersWithNewPriority.toArray(new RaftPeer[0])); - assertTrue(reply.isSuccess()); - JavaUtils.attempt(() -> assertEquals(leaderId, leader.getId()), + Assertions.assertTrue(reply.isSuccess()); + JavaUtils.attempt(() -> Assertions.assertEquals(leaderId, leader.getId()), 20, HUNDRED_MILLIS, "check leader id", LOG); final RaftClientReply resumeLeaderReply = client.getLeaderElectionManagementApi(f1.getId()).resume(); - assertTrue(resumeLeaderReply.isSuccess()); - JavaUtils.attempt(() -> assertEquals(f1.getId(), cluster.getLeader().getId()), + Assertions.assertTrue(resumeLeaderReply.isSuccess()); + JavaUtils.attempt(() -> Assertions.assertEquals(f1.getId(), cluster.getLeader().getId()), 20, HUNDRED_MILLIS, "check new leader", LOG); } } @@ -733,15 +727,15 @@ void runTestLeaderLease(CLUSTER cluster, long leaseTimeoutMs) throws Exception { try (final RaftClient client = cluster.createClient(leader.getId())) { client.io().send(new RaftTestUtil.SimpleMessage("message")); - assertTrue(leader.getInfo().isLeader()); - assertTrue(leader.getInfo().isLeaderReady()); + Assertions.assertTrue(leader.getInfo().isLeader()); + Assertions.assertTrue(leader.getInfo().isLeaderReady()); RaftServerTestUtil.assertLeaderLease(leader, true); RaftTestUtil.isolate(cluster, leader.getId()); Thread.sleep(leaseTimeoutMs); - assertTrue(leader.getInfo().isLeader()); - assertTrue(leader.getInfo().isLeaderReady()); + Assertions.assertTrue(leader.getInfo().isLeader()); + Assertions.assertTrue(leader.getInfo().isLeaderReady()); RaftServerTestUtil.assertLeaderLease(leader, false); } finally { RaftTestUtil.deIsolate(cluster, leader.getId()); @@ -762,8 +756,8 @@ void runTestLeaderLeaseDuringReconfiguration(CLUSTER cluster, long leaseTimeoutM try (final RaftClient client = cluster.createClient(leader.getId())) { client.io().send(new RaftTestUtil.SimpleMessage("message")); - assertTrue(leader.getInfo().isLeader()); - assertTrue(leader.getInfo().isLeaderReady()); + Assertions.assertTrue(leader.getInfo().isLeader()); + Assertions.assertTrue(leader.getInfo().isLeaderReady()); RaftServerTestUtil.assertLeaderLease(leader, true); final List followers = cluster.getFollowers(); @@ -784,8 +778,8 @@ void runTestLeaderLeaseDuringReconfiguration(CLUSTER cluster, long leaseTimeoutM Thread.sleep(leaseTimeoutMs); - assertTrue(leader.getInfo().isLeader()); - assertTrue(leader.getInfo().isLeaderReady()); + Assertions.assertTrue(leader.getInfo().isLeader()); + Assertions.assertTrue(leader.getInfo().isLeaderReady()); RaftServerTestUtil.assertLeaderLease(leader, false); } finally { diff --git a/ratis-test/pom.xml b/ratis-test/pom.xml index 577262d84c..8780a862ca 100644 --- a/ratis-test/pom.xml +++ b/ratis-test/pom.xml @@ -138,17 +138,17 @@ org.junit.jupiter - junit-jupiter-api + junit-jupiter-engine test org.junit.jupiter - junit-jupiter-params + junit-jupiter-api test - org.junit.platform - junit-platform-launcher + org.junit.jupiter + junit-jupiter-params test diff --git a/ratis-test/src/test/java/org/apache/ratis/datastream/DataStreamAsyncClusterTests.java b/ratis-test/src/test/java/org/apache/ratis/datastream/DataStreamAsyncClusterTests.java index eb25a369e6..6fd5b47213 100644 --- a/ratis-test/src/test/java/org/apache/ratis/datastream/DataStreamAsyncClusterTests.java +++ b/ratis-test/src/test/java/org/apache/ratis/datastream/DataStreamAsyncClusterTests.java @@ -36,7 +36,6 @@ import org.apache.ratis.util.function.CheckedBiFunction; import org.junit.jupiter.api.Assertions; import org.junit.jupiter.api.Test; -import org.junit.jupiter.api.Timeout; import org.slf4j.event.Level; import java.io.IOException; diff --git a/ratis-test/src/test/java/org/apache/ratis/datastream/DataStreamTestUtils.java b/ratis-test/src/test/java/org/apache/ratis/datastream/DataStreamTestUtils.java index 989b6cd2b2..f038dc5a45 100644 --- a/ratis-test/src/test/java/org/apache/ratis/datastream/DataStreamTestUtils.java +++ b/ratis-test/src/test/java/org/apache/ratis/datastream/DataStreamTestUtils.java @@ -419,13 +419,4 @@ static void assertLogEntry(RaftServer.Division division, SingleDataStream stream final LogEntryProto entryFromLog = searchLogEntry(ClientInvocationId.valueOf(request), division.getRaftLog()); Assertions.assertEquals(entryFromStream, entryFromLog); } - - ResourceLeakDetector.LeakListener LEAK_LISTENER = (resourceType, records) -> { - throw new IllegalStateException("Leak detected for resource type: " + resourceType + records); - }; - - static void enableResourceLeakDetector() { - ResourceLeakDetector.setLevel(Level.PARANOID); - ByteBufUtil.setLeakListener(DataStreamTestUtils.LEAK_LISTENER); - } } diff --git a/ratis-test/src/test/java/org/apache/ratis/grpc/TestRaftWithGrpc.java b/ratis-test/src/test/java/org/apache/ratis/grpc/TestRaftWithGrpc.java index 42211cefc1..3bd3b3f754 100644 --- a/ratis-test/src/test/java/org/apache/ratis/grpc/TestRaftWithGrpc.java +++ b/ratis-test/src/test/java/org/apache/ratis/grpc/TestRaftWithGrpc.java @@ -32,10 +32,8 @@ import org.apache.ratis.util.JavaUtils; import org.apache.ratis.util.TimeDuration; import org.junit.jupiter.api.Assertions; -import org.junit.jupiter.api.Disabled; -import org.junit.jupiter.api.Timeout; import org.junit.jupiter.params.ParameterizedTest; -import org.junit.jupiter.params.provider.ValueSource; +import org.junit.jupiter.params.provider.MethodSource; import java.util.concurrent.CompletableFuture; import java.util.concurrent.TimeUnit; @@ -51,32 +49,28 @@ public class TestRaftWithGrpc SimpleStateMachine4Testing.class, StateMachine.class); } - @Disabled - @Override - public void testWithLoad() { - // skip testWithLoad() from parent, called from parameterized testWithLoad(boolean) + public static Collection data() { + return Arrays.asList((new Boolean[][] {{Boolean.FALSE}, {Boolean.TRUE}})); } - @Flaky("RATIS-2253") - @Timeout(300) @ParameterizedTest - @ValueSource(booleans = {true, false}) - public void testWithLoad(boolean separateHeartbeat) throws Exception { + @MethodSource("data") + public void testWithLoad(Boolean separateHeartbeat) throws Exception { GrpcConfigKeys.Server.setHeartbeatChannel(getProperties(), separateHeartbeat); super.testWithLoad(); BlockRequestHandlingInjection.getInstance().unblockAll(); } @ParameterizedTest - @ValueSource(booleans = {true, false}) - public void testRequestTimeout(boolean separateHeartbeat) throws Exception { + @MethodSource("data") + public void testRequestTimeout(Boolean separateHeartbeat) throws Exception { GrpcConfigKeys.Server.setHeartbeatChannel(getProperties(), separateHeartbeat); runWithNewCluster(NUM_SERVERS, cluster -> testRequestTimeout(false, cluster, LOG)); } @ParameterizedTest - @ValueSource(booleans = {true, false}) - public void testUpdateViaHeartbeat(boolean separateHeartbeat) throws Exception { + @MethodSource("data") + public void testUpdateViaHeartbeat(Boolean separateHeartbeat) throws Exception { GrpcConfigKeys.Server.setHeartbeatChannel(getProperties(), separateHeartbeat); runWithNewCluster(NUM_SERVERS, this::runTestUpdateViaHeartbeat); } diff --git a/ratis-test/src/test/java/org/apache/ratis/grpc/server/TestGrpcMessageMetrics.java b/ratis-test/src/test/java/org/apache/ratis/grpc/server/TestGrpcMessageMetrics.java index a8cd6138ec..39b745a561 100644 --- a/ratis-test/src/test/java/org/apache/ratis/grpc/server/TestGrpcMessageMetrics.java +++ b/ratis-test/src/test/java/org/apache/ratis/grpc/server/TestGrpcMessageMetrics.java @@ -70,10 +70,6 @@ static void assertMessageCount(RaftServer.Division server) { final GrpcServicesImpl services = (GrpcServicesImpl) RaftServerTestUtil.getServerRpc(server); final RatisMetricRegistry registry = services.getMessageMetrics().getRegistry(); String counter_prefix = serverId + "_" + "ratis.grpc.RaftServerProtocolService"; - final String metricPrefix = counter_prefix + "_" + "requestVote" + "_OK"; - final long before = registry.counter(metricPrefix + "_completed_total").getCount(); - services.getMessageMetrics().rpcCompleted(metricPrefix); - final long after = registry.counter(metricPrefix + "_completed_total").getCount(); - Assertions.assertEquals(before + 1, after); + Assertions.assertTrue(registry.counter(counter_prefix + "_" + "requestVote" + "_OK_completed_total").getCount() > 0); } } diff --git a/ratis-test/src/test/java/org/apache/ratis/grpc/util/TestGrpcZeroCopy.java b/ratis-test/src/test/java/org/apache/ratis/grpc/util/TestGrpcZeroCopy.java index dde44e5799..f2427edd94 100644 --- a/ratis-test/src/test/java/org/apache/ratis/grpc/util/TestGrpcZeroCopy.java +++ b/ratis-test/src/test/java/org/apache/ratis/grpc/util/TestGrpcZeroCopy.java @@ -26,13 +26,9 @@ import org.apache.ratis.thirdparty.io.netty.buffer.PooledByteBufAllocator; import org.apache.ratis.util.NetUtils; import org.apache.ratis.util.TraditionalBinaryPrefix; +import org.junit.jupiter.api.Assertions; import org.junit.jupiter.api.Test; -import static org.junit.jupiter.api.Assertions.assertEquals; -import static org.junit.jupiter.api.Assertions.assertThrows; -import static org.junit.jupiter.api.Assertions.assertTrue; -import static org.junit.jupiter.api.Assumptions.assumeTrue; - import java.util.ArrayList; import java.util.List; import java.util.Random; @@ -64,8 +60,8 @@ static void verify(long seed, ByteString b) { RANDOM.nextBytes(ARRAY); final ByteString expected = UnsafeByteOperations.unsafeWrap(ARRAY, 0, remaining); final ByteString computed = b.substring(offset, offset + remaining); - assertEquals(expected.size(), computed.size()); - assertEquals(expected, computed); + Assertions.assertEquals(expected.size(), computed.size()); + Assertions.assertEquals(expected, computed); offset += remaining; } } @@ -103,7 +99,7 @@ public static boolean isReady() { /** Test a zero-copy marshaller is available from the versions of gRPC and Protobuf. */ @Test public void testReadiness() { - assertTrue(isReady()); + Assertions.assertTrue(isReady()); } @@ -150,7 +146,7 @@ void sendMessages(int n, GrpcZeroCopyTestClient client, GrpcZeroCopyTestServer s for (int i = 0; i < futures.size(); i++) { final String expected = GrpcZeroCopyTestServer.toReply(i, messages.get(i)); final String reply = futures.get(i).get(); - assertEquals(expected, reply, "expected = " + expected + " != reply = " + reply); + Assertions.assertEquals(expected, reply, "expected = " + expected + " != reply = " + reply); server.assertCounts(numElements, numBytes); } } @@ -175,8 +171,8 @@ void sendBinaries(int n, GrpcZeroCopyTestClient client, GrpcZeroCopyTestServer s } final ByteString reply = future.get(); - assertEquals(4, reply.size()); - assertEquals(size, reply.asReadOnlyByteBuffer().getInt()); + Assertions.assertEquals(4, reply.size()); + Assertions.assertEquals(size, reply.asReadOnlyByteBuffer().getInt()); numElements++; numBytes += size; From b7f425eee39a0ba1af8cccc7c4ef31aab9a69f3e Mon Sep 17 00:00:00 2001 From: Duong Nguyen Date: Thu, 1 Feb 2024 15:39:48 -0800 Subject: [PATCH 336/397] RATIS-1997. Refactor StateMachine interface to use ReferenceCountedObject (#1036) (cherry picked from commit 775b286c4540057e5f81419806d9e7737e5f568e) --- .../filestore/FileStoreStateMachine.java | 10 +++++-- .../ratis/statemachine/StateMachine.java | 27 ++++++++++++++++++ .../raftlog/segmented/SegmentedRaftLog.java | 2 +- .../segmented/SegmentedRaftLogWorker.java | 28 +++++++++++-------- .../impl/SimpleStateMachine4Testing.java | 4 ++- .../segmented/TestSegmentedRaftLog.java | 3 +- 6 files changed, 56 insertions(+), 18 deletions(-) diff --git a/ratis-examples/src/main/java/org/apache/ratis/examples/filestore/FileStoreStateMachine.java b/ratis-examples/src/main/java/org/apache/ratis/examples/filestore/FileStoreStateMachine.java index 5f258ee3b7..941b8b9123 100644 --- a/ratis-examples/src/main/java/org/apache/ratis/examples/filestore/FileStoreStateMachine.java +++ b/ratis-examples/src/main/java/org/apache/ratis/examples/filestore/FileStoreStateMachine.java @@ -40,6 +40,8 @@ import org.apache.ratis.thirdparty.com.google.protobuf.ByteString; import org.apache.ratis.thirdparty.com.google.protobuf.InvalidProtocolBufferException; import org.apache.ratis.util.FileUtils; +import org.apache.ratis.util.JavaUtils; +import org.apache.ratis.util.ReferenceCountedObject; import java.io.IOException; import java.nio.file.Path; @@ -121,7 +123,8 @@ public TransactionContext startTransaction(LogEntryProto entry, RaftProtos.RaftP } @Override - public CompletableFuture write(LogEntryProto entry, TransactionContext context) { + public CompletableFuture write(ReferenceCountedObject entryRef, TransactionContext context) { + LogEntryProto entry = entryRef.retain(); final FileStoreRequestProto proto = getProto(context, entry); if (proto.getRequestCase() != FileStoreRequestProto.RequestCase.WRITEHEADER) { return null; @@ -130,9 +133,10 @@ public CompletableFuture write(LogEntryProto entry, TransactionContext final WriteRequestHeaderProto h = proto.getWriteHeader(); final CompletableFuture f = files.write(entry.getIndex(), h.getPath().toStringUtf8(), h.getClose(), h.getSync(), h.getOffset(), - entry.getStateMachineLogEntry().getStateMachineEntry().getStateMachineData()); + entry.getStateMachineLogEntry().getStateMachineEntry().getStateMachineData() + ).whenComplete((r, e) -> entryRef.release()); // sync only if closing the file - return h.getClose()? f: null; + return h.getClose() ? f: null; } static FileStoreRequestProto getProto(TransactionContext context, LogEntryProto entry) { diff --git a/ratis-server-api/src/main/java/org/apache/ratis/statemachine/StateMachine.java b/ratis-server-api/src/main/java/org/apache/ratis/statemachine/StateMachine.java index 98d4537847..176811eb18 100644 --- a/ratis-server-api/src/main/java/org/apache/ratis/statemachine/StateMachine.java +++ b/ratis-server-api/src/main/java/org/apache/ratis/statemachine/StateMachine.java @@ -92,7 +92,9 @@ default CompletableFuture read(LogEntryProto entry, TransactionConte * Write asynchronously the state machine data in the given log entry to this state machine. * * @return a future for the write task + * @deprecated Applications should implement {@link #write(ReferenceCountedObject, TransactionContext)} instead. */ + @Deprecated default CompletableFuture write(LogEntryProto entry) { return CompletableFuture.completedFuture(null); } @@ -101,11 +103,36 @@ default CompletableFuture write(LogEntryProto entry) { * Write asynchronously the state machine data in the given log entry to this state machine. * * @return a future for the write task + * @deprecated Applications should implement {@link #write(ReferenceCountedObject, TransactionContext)} instead. */ + @Deprecated default CompletableFuture write(LogEntryProto entry, TransactionContext context) { return write(entry); } + /** + * Write asynchronously the state machine data in the given log entry to this state machine. + * + * @param entryRef Reference to a log entry. + * Implementations of this method may call {@link ReferenceCountedObject#get()} + * to access the log entry before this method returns. + * If the log entry is needed after this method returns, + * e.g. for asynchronous computation or caching, + * the implementation must invoke {@link ReferenceCountedObject#retain()} + * and {@link ReferenceCountedObject#release()}. + * @return a future for the write task + */ + default CompletableFuture write(ReferenceCountedObject entryRef, TransactionContext context) { + final LogEntryProto entry = entryRef.get(); + try { + final LogEntryProto copy = LogEntryProto.parseFrom(entry.toByteString()); + return write(copy, context); + } catch (InvalidProtocolBufferException e) { + return JavaUtils.completeExceptionally(new IllegalStateException( + "Failed to copy log entry " + TermIndex.valueOf(entry), e)); + } + } + /** * Create asynchronously a {@link DataStream} to stream state machine data. * The state machine may use the first message (i.e. request.getMessage()) as the header to create the stream. diff --git a/ratis-server/src/main/java/org/apache/ratis/server/raftlog/segmented/SegmentedRaftLog.java b/ratis-server/src/main/java/org/apache/ratis/server/raftlog/segmented/SegmentedRaftLog.java index 86c9314b44..8879024771 100644 --- a/ratis-server/src/main/java/org/apache/ratis/server/raftlog/segmented/SegmentedRaftLog.java +++ b/ratis-server/src/main/java/org/apache/ratis/server/raftlog/segmented/SegmentedRaftLog.java @@ -433,7 +433,7 @@ protected CompletableFuture appendEntryImpl(ReferenceCountedObject entry, + LogEntryProto removedStateMachineData, TransactionContext context) { + return addIOTask(new WriteLog(entry, removedStateMachineData, context)); } Task truncate(TruncationSegments ts, long index) { @@ -498,25 +499,28 @@ private class WriteLog extends Task { private final CompletableFuture stateMachineFuture; private final CompletableFuture combined; - WriteLog(LogEntryProto entry, TransactionContext context) { - this.entry = LogProtoUtils.removeStateMachineData(entry); - if (this.entry == entry) { - final StateMachineLogEntryProto proto = entry.hasStateMachineLogEntry()? entry.getStateMachineLogEntry(): null; + WriteLog(ReferenceCountedObject entryRef, LogEntryProto removedStateMachineData, + TransactionContext context) { + LogEntryProto origEntry = entryRef.get(); + this.entry = removedStateMachineData; + if (this.entry == origEntry) { + final StateMachineLogEntryProto proto = origEntry.hasStateMachineLogEntry() ? + origEntry.getStateMachineLogEntry(): null; if (stateMachine != null && proto != null && proto.getType() == StateMachineLogEntryProto.Type.DATASTREAM) { final ClientInvocationId invocationId = ClientInvocationId.valueOf(proto); final CompletableFuture removed = server.getDataStreamMap().remove(invocationId); - this.stateMachineFuture = removed == null? stateMachine.data().link(null, entry) - : removed.thenApply(stream -> stateMachine.data().link(stream, entry)); + this.stateMachineFuture = removed == null? stateMachine.data().link(null, origEntry) + : removed.thenApply(stream -> stateMachine.data().link(stream, origEntry)); } else { this.stateMachineFuture = null; } } else { try { - // this.entry != entry iff the entry has state machine data - this.stateMachineFuture = stateMachine.data().write(entry, context); + // this.entry != origEntry if it has state machine data + this.stateMachineFuture = stateMachine.data().write(entryRef, context); } catch (Exception e) { - LOG.error(name + ": writeStateMachineData failed for index " + entry.getIndex() - + ", entry=" + LogProtoUtils.toLogEntryString(entry, stateMachine::toStateMachineLogEntryString), e); + LOG.error(name + ": writeStateMachineData failed for index " + origEntry.getIndex() + + ", entry=" + LogProtoUtils.toLogEntryString(origEntry, stateMachine::toStateMachineLogEntryString), e); throw e; } } diff --git a/ratis-server/src/test/java/org/apache/ratis/statemachine/impl/SimpleStateMachine4Testing.java b/ratis-server/src/test/java/org/apache/ratis/statemachine/impl/SimpleStateMachine4Testing.java index 45e14dd189..9ef6e33048 100644 --- a/ratis-server/src/test/java/org/apache/ratis/statemachine/impl/SimpleStateMachine4Testing.java +++ b/ratis-server/src/test/java/org/apache/ratis/statemachine/impl/SimpleStateMachine4Testing.java @@ -48,6 +48,7 @@ import org.apache.ratis.util.LifeCycle; import org.apache.ratis.util.MD5FileUtil; import org.apache.ratis.util.Preconditions; +import org.apache.ratis.util.ReferenceCountedObject; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -370,7 +371,8 @@ public TransactionContext startTransaction(RaftClientRequest request) { } @Override - public CompletableFuture write(LogEntryProto entry) { + public CompletableFuture write(ReferenceCountedObject entry, TransactionContext context) { + Preconditions.assertTrue(entry.get() != null); return blocking.getFuture(Blocking.Type.WRITE_STATE_MACHINE_DATA); } diff --git a/ratis-test/src/test/java/org/apache/ratis/server/raftlog/segmented/TestSegmentedRaftLog.java b/ratis-test/src/test/java/org/apache/ratis/server/raftlog/segmented/TestSegmentedRaftLog.java index 580de03e98..d268377173 100644 --- a/ratis-test/src/test/java/org/apache/ratis/server/raftlog/segmented/TestSegmentedRaftLog.java +++ b/ratis-test/src/test/java/org/apache/ratis/server/raftlog/segmented/TestSegmentedRaftLog.java @@ -37,6 +37,7 @@ import org.apache.ratis.server.raftlog.RaftLog; import org.apache.ratis.server.storage.RaftStorage; import org.apache.ratis.server.storage.RaftStorageTestUtils; +import org.apache.ratis.statemachine.TransactionContext; import org.apache.ratis.statemachine.impl.SimpleStateMachine4Testing; import org.apache.ratis.statemachine.StateMachine; import org.apache.ratis.statemachine.impl.BaseStateMachine; @@ -811,7 +812,7 @@ public void testServerShutdownOnTimeoutIOException(Boolean useAsyncFlush, Boolea final LogEntryProto entry = prepareLogEntry(0, 0, null, true); final StateMachine sm = new BaseStateMachine() { @Override - public CompletableFuture write(LogEntryProto entry) { + public CompletableFuture write(ReferenceCountedObject entry, TransactionContext context) { getLifeCycle().transition(LifeCycle.State.STARTING); getLifeCycle().transition(LifeCycle.State.RUNNING); From 73adbd85f751a2a5cc828d0d43fae3f18a948974 Mon Sep 17 00:00:00 2001 From: Tsz-Wo Nicholas Sze Date: Tue, 6 Feb 2024 08:22:52 -0800 Subject: [PATCH 337/397] RATIS-2024. Refactor appendEntries code. (#1040) (cherry picked from commit 1d99921ef4bf676029cf231faf2ca79eff59f44b) --- .../ratis/server/impl/RaftServerImpl.java | 87 ++++++------------- .../impl/SnapshotInstallationHandler.java | 25 +++--- .../ratis/server/util/ServerStringUtils.java | 8 +- 3 files changed, 41 insertions(+), 79 deletions(-) diff --git a/ratis-server/src/main/java/org/apache/ratis/server/impl/RaftServerImpl.java b/ratis-server/src/main/java/org/apache/ratis/server/impl/RaftServerImpl.java index 4962f32a1d..927a6a83c6 100644 --- a/ratis-server/src/main/java/org/apache/ratis/server/impl/RaftServerImpl.java +++ b/ratis-server/src/main/java/org/apache/ratis/server/impl/RaftServerImpl.java @@ -117,43 +117,10 @@ import org.apache.ratis.util.TimeDuration; import org.apache.ratis.util.function.CheckedSupplier; -import java.io.File; -import java.io.IOException; -import java.nio.file.NoSuchFileException; -import java.util.ArrayList; -import java.util.Collection; -import java.util.Comparator; -import java.util.List; -import java.util.Map; -import java.util.Objects; -import java.util.Optional; -import java.util.Set; -import java.util.concurrent.CompletableFuture; -import java.util.concurrent.CompletionException; -import java.util.concurrent.CountDownLatch; -import java.util.concurrent.ExecutionException; -import java.util.concurrent.ExecutorService; -import java.util.concurrent.ThreadLocalRandom; -import java.util.concurrent.TimeUnit; -import java.util.concurrent.atomic.AtomicBoolean; -import java.util.function.Function; -import java.util.function.Supplier; -import java.util.stream.Collectors; -import java.util.stream.Stream; - -import static org.apache.ratis.server.impl.LeaderElection.Result.NOT_IN_CONF; -import static org.apache.ratis.server.impl.ServerImplUtils.assertEntries; -import static org.apache.ratis.server.impl.ServerImplUtils.assertGroup; import static org.apache.ratis.server.impl.ServerImplUtils.effectiveCommitIndex; import static org.apache.ratis.server.impl.ServerProtoUtils.toAppendEntriesReplyProto; -import static org.apache.ratis.server.impl.ServerProtoUtils.toReadIndexReplyProto; -import static org.apache.ratis.server.impl.ServerProtoUtils.toReadIndexRequestProto; -import static org.apache.ratis.server.impl.ServerProtoUtils.toRequestVoteReplyProto; -import static org.apache.ratis.server.impl.ServerProtoUtils.toStartLeaderElectionReplyProto; -import static org.apache.ratis.server.raftlog.LogProtoUtils.toLogEntryTermIndexString; import static org.apache.ratis.server.util.ServerStringUtils.toAppendEntriesReplyString; import static org.apache.ratis.server.util.ServerStringUtils.toAppendEntriesRequestString; -import static org.apache.ratis.server.util.ServerStringUtils.toRequestVoteReplyString; class RaftServerImpl implements RaftServer.Division, RaftServerProtocol, RaftServerAsynchronousProtocol, @@ -1565,16 +1532,23 @@ public CompletableFuture appendEntriesAsync( final AppendEntriesRequestProto r = requestRef.retain(); final RaftRpcRequestProto request = r.getServerRequest(); final TermIndex previous = r.hasPreviousLog()? TermIndex.valueOf(r.getPreviousLog()) : null; - final RaftPeerId requestorId = RaftPeerId.valueOf(request.getRequestorId()); - try { - preAppendEntriesAsync(requestorId, ProtoUtils.toRaftGroupId(request.getRaftGroupId()), r.getLeaderTerm(), - previous, r.getLeaderCommit(), r.getInitializing(), entries); - return appendEntriesAsync(requestorId, r.getLeaderTerm(), previous, r.getLeaderCommit(), - request.getCallId(), r.getInitializing(), r.getCommitInfosList(), entries, requestRef); + final RaftPeerId leaderId = RaftPeerId.valueOf(request.getRequestorId()); + final RaftGroupId leaderGroupId = ProtoUtils.toRaftGroupId(request.getRaftGroupId()); + + CodeInjectionForTesting.execute(APPEND_ENTRIES, getId(), leaderId, previous, r); + + assertLifeCycleState(LifeCycle.States.STARTING_OR_RUNNING); + if (!startComplete.get()) { + throw new ServerNotReadyException(getMemberId() + ": The server role is not yet initialized."); + } + assertGroup(leaderId, leaderGroupId); + validateEntries(r.getLeaderTerm(), previous, r.getEntriesList()); + + return appendEntriesAsync(leaderId, request.getCallId(), previous, requestRef); } catch(Exception t) { - LOG.error("{}: Failed appendEntriesAsync {}", getMemberId(), r, t); - throw t; + LOG.error("{}: Failed appendEntries* {}", getMemberId(), toAppendEntriesRequestString(r), t); + throw IOUtils.asIOException(t); } finally { requestRef.release(); } @@ -1639,14 +1613,13 @@ ExecutorService getServerExecutor() { return serverExecutor; } - @SuppressWarnings("checkstyle:parameternumber") - private CompletableFuture appendEntriesAsync( - RaftPeerId leaderId, long leaderTerm, TermIndex previous, long leaderCommit, long callId, boolean initializing, - List commitInfos, List entries, - ReferenceCountedObject requestRef) throws IOException { + private CompletableFuture appendEntriesAsync(RaftPeerId leaderId, long callId, + TermIndex previous, ReferenceCountedObject requestRef) throws IOException { + final AppendEntriesRequestProto proto = requestRef.get(); + final List entries = proto.getEntriesList(); final boolean isHeartbeat = entries.isEmpty(); logAppendEntries(isHeartbeat, () -> getMemberId() + ": appendEntries* " - + toAppendEntriesRequestString(proto, stateMachine::toStateMachineLogEntryString)); + + toAppendEntriesRequestString(proto)); final long leaderTerm = proto.getLeaderTerm(); final long currentTerm; @@ -1690,7 +1663,7 @@ leaderId, getMemberId(), currentTerm, followerCommit, inconsistencyReplyNextInde AppendResult.INCONSISTENCY, callId, RaftLog.INVALID_LOG_INDEX, isHeartbeat); LOG.info("{}: appendEntries* reply {}", getMemberId(), toAppendEntriesReplyString(reply)); followerState.ifPresent(fs -> fs.updateLastRpcTime(FollowerState.UpdateType.APPEND_COMPLETE)); - return future.thenApply(dummy -> reply); + return CompletableFuture.completedFuture(reply); } state.updateConfiguration(entries); @@ -1703,7 +1676,7 @@ leaderId, getMemberId(), currentTerm, followerCommit, inconsistencyReplyNextInde final List> futures = entries.isEmpty() ? Collections.emptyList() : state.getLog().append(requestRef.delegate(entries)); - commitInfos.forEach(commitInfoCache::update); + proto.getCommitInfosList().forEach(commitInfoCache::update); CodeInjectionForTesting.execute(LOG_SYNC, getId(), null); if (!isHeartbeat) { @@ -1716,12 +1689,7 @@ leaderId, getMemberId(), currentTerm, followerCommit, inconsistencyReplyNextInde final long commitIndex = effectiveCommitIndex(proto.getLeaderCommit(), previous, entries.size()); final long matchIndex = isHeartbeat? RaftLog.INVALID_LOG_INDEX: entries.get(entries.size() - 1).getIndex(); - return appendFuture.whenCompleteAsync((r, t) -> { - if (t != null) { - LOG.warn("{}: appendEntries* failed: {}", getMemberId(), toLogEntryTermIndexString(entries), t); - } else if (LOG.isDebugEnabled()) { - LOG.debug("{}: appendEntries* succeeded: {}", getMemberId(), toLogEntryTermIndexString(entries)); - } + return JavaUtils.allOf(futures).whenCompleteAsync((r, t) -> { followerState.ifPresent(fs -> fs.updateLastRpcTime(FollowerState.UpdateType.APPEND_COMPLETE)); timer.stop(); }, getServerExecutor()).thenApply(v -> { @@ -1739,11 +1707,6 @@ leaderId, getMemberId(), currentTerm, followerCommit, inconsistencyReplyNextInde }); } - private CompletableFuture appendLog(List entries) { - return CompletableFuture.completedFuture(null) - .thenComposeAsync(dummy -> JavaUtils.allOf(state.getLog().append(entries)), serverExecutor); - } - private long checkInconsistentAppendEntries(TermIndex previous, List entries) { // Check if a snapshot installation through state machine is in progress. final long installSnapshot = snapshotInstallationHandler.getInProgressInstallSnapshotIndex(); @@ -1829,7 +1792,7 @@ public StartLeaderElectionReplyProto startLeaderElection(StartLeaderElectionRequ if (!request.hasLeaderLastEntry()) { // It should have a leaderLastEntry since there is a placeHolder entry. LOG.warn("{}: leaderLastEntry is missing in {}", getMemberId(), request); - return toStartLeaderElectionReplyProto(leaderId, getMemberId(), false); + return ServerProtoUtils.toStartLeaderElectionReplyProto(leaderId, getMemberId(), false); } final TermIndex leaderLastEntry = TermIndex.valueOf(request.getLeaderLastEntry()); @@ -1843,7 +1806,7 @@ public StartLeaderElectionReplyProto startLeaderElection(StartLeaderElectionRequ assertLifeCycleState(LifeCycle.States.STARTING_OR_RUNNING); final boolean recognized = state.recognizeLeader("startLeaderElection", leaderId, leaderLastEntry.getTerm()); if (!recognized) { - return toStartLeaderElectionReplyProto(leaderId, getMemberId(), false); + return ServerProtoUtils.toStartLeaderElectionReplyProto(leaderId, getMemberId(), false); } if (!getInfo().isFollower()) { diff --git a/ratis-server/src/main/java/org/apache/ratis/server/impl/SnapshotInstallationHandler.java b/ratis-server/src/main/java/org/apache/ratis/server/impl/SnapshotInstallationHandler.java index 46b6aaf87f..f8ba31a395 100644 --- a/ratis-server/src/main/java/org/apache/ratis/server/impl/SnapshotInstallationHandler.java +++ b/ratis-server/src/main/java/org/apache/ratis/server/impl/SnapshotInstallationHandler.java @@ -31,8 +31,7 @@ import org.apache.ratis.protocol.RaftGroupMemberId; import org.apache.ratis.protocol.RaftPeerId; import org.apache.ratis.server.RaftServerConfigKeys; -import org.apache.ratis.server.impl.FollowerState.UpdateType; -import org.apache.ratis.server.protocol.RaftServerProtocol.Op; +import org.apache.ratis.server.protocol.RaftServerProtocol; import org.apache.ratis.server.protocol.TermIndex; import org.apache.ratis.server.raftlog.LogProtoUtils; import org.apache.ratis.server.util.ServerStringUtils; @@ -173,11 +172,11 @@ private CompletableFuture checkAndInstallSnapshot(Ins final long lastIncludedIndex = lastIncluded.getIndex(); final CompletableFuture future; synchronized (server) { - final boolean recognized = state.recognizeLeader(Op.INSTALL_SNAPSHOT, leaderId, leaderTerm); + final boolean recognized = state.recognizeLeader(RaftServerProtocol.Op.INSTALL_SNAPSHOT, leaderId, leaderTerm); currentTerm = state.getCurrentTerm(); if (!recognized) { - return CompletableFuture.completedFuture(toInstallSnapshotReplyProto(leaderId, getMemberId(), - currentTerm, snapshotChunkRequest.getRequestIndex(), InstallSnapshotResult.NOT_LEADER)); + return toInstallSnapshotReplyProto(leaderId, getMemberId(), + currentTerm, snapshotChunkRequest.getRequestIndex(), InstallSnapshotResult.NOT_LEADER); } future = server.changeToFollowerAndPersistMetadata(leaderTerm, true, Op.INSTALL_SNAPSHOT); state.setLeader(leaderId, Op.INSTALL_SNAPSHOT); @@ -235,7 +234,7 @@ private CompletableFuture checkAndInstallSnapshot(Ins if (snapshotChunkRequest.getDone()) { LOG.info("{}: successfully install the entire snapshot-{}", getMemberId(), lastIncludedIndex); } - final InstallSnapshotReplyProto reply = toInstallSnapshotReplyProto(leaderId, getMemberId(), + return toInstallSnapshotReplyProto(leaderId, getMemberId(), currentTerm, snapshotChunkRequest.getRequestIndex(), InstallSnapshotResult.SUCCESS); return future.thenApply(dummy -> reply); } @@ -249,11 +248,11 @@ private CompletableFuture notifyStateMachineToInstall final long firstAvailableLogIndex = firstAvailableLogTermIndex.getIndex(); final CompletableFuture future; synchronized (server) { - final boolean recognized = state.recognizeLeader(UpdateType.INSTALL_SNAPSHOT_NOTIFICATION, leaderId, leaderTerm); + final boolean recognized = state.recognizeLeader("notifyInstallSnapshot", leaderId, leaderTerm); currentTerm = state.getCurrentTerm(); if (!recognized) { - return CompletableFuture.completedFuture(toInstallSnapshotReplyProto(leaderId, getMemberId(), - currentTerm, InstallSnapshotResult.NOT_LEADER)); + return toInstallSnapshotReplyProto(leaderId, getMemberId(), + currentTerm, InstallSnapshotResult.NOT_LEADER); } future = server.changeToFollowerAndPersistMetadata(leaderTerm, true, UpdateType.INSTALL_SNAPSHOT_NOTIFICATION); state.setLeader(leaderId, UpdateType.INSTALL_SNAPSHOT_NOTIFICATION); @@ -272,7 +271,7 @@ private CompletableFuture notifyStateMachineToInstall inProgressInstallSnapshotIndex.compareAndSet(firstAvailableLogIndex, INVALID_LOG_INDEX); LOG.info("{}: InstallSnapshot notification result: {}, current snapshot index: {}", getMemberId(), InstallSnapshotResult.ALREADY_INSTALLED, snapshotIndex); - final InstallSnapshotReplyProto reply = toInstallSnapshotReplyProto(leaderId, getMemberId(), currentTerm, + return toInstallSnapshotReplyProto(leaderId, getMemberId(), currentTerm, InstallSnapshotResult.ALREADY_INSTALLED, snapshotIndex); return future.thenApply(dummy -> reply); } @@ -351,7 +350,7 @@ private CompletableFuture notifyStateMachineToInstall inProgressInstallSnapshotIndex.set(INVALID_LOG_INDEX); server.getStateMachine().event().notifySnapshotInstalled( InstallSnapshotResult.SNAPSHOT_UNAVAILABLE, INVALID_LOG_INDEX, server.getPeer()); - final InstallSnapshotReplyProto reply = toInstallSnapshotReplyProto(leaderId, getMemberId(), + return toInstallSnapshotReplyProto(leaderId, getMemberId(), currentTerm, InstallSnapshotResult.SNAPSHOT_UNAVAILABLE); return future.thenApply(dummy -> reply); } @@ -370,7 +369,7 @@ private CompletableFuture notifyStateMachineToInstall server.getStateMachine().event().notifySnapshotInstalled( InstallSnapshotResult.SNAPSHOT_INSTALLED, latestInstalledIndex, server.getPeer()); installedIndex.set(latestInstalledIndex); - final InstallSnapshotReplyProto reply = toInstallSnapshotReplyProto(leaderId, getMemberId(), + return toInstallSnapshotReplyProto(leaderId, getMemberId(), currentTerm, InstallSnapshotResult.SNAPSHOT_INSTALLED, latestInstalledSnapshotTermIndex.getIndex()); return future.thenApply(dummy -> reply); } @@ -380,7 +379,7 @@ private CompletableFuture notifyStateMachineToInstall LOG.debug("{}: InstallSnapshot notification result: {}", getMemberId(), InstallSnapshotResult.IN_PROGRESS); } - final InstallSnapshotReplyProto reply = toInstallSnapshotReplyProto(leaderId, getMemberId(), + return toInstallSnapshotReplyProto(leaderId, getMemberId(), currentTerm, InstallSnapshotResult.IN_PROGRESS); return future.thenApply(dummy -> reply); } diff --git a/ratis-server/src/main/java/org/apache/ratis/server/util/ServerStringUtils.java b/ratis-server/src/main/java/org/apache/ratis/server/util/ServerStringUtils.java index 50b0918b83..8ca314ff24 100644 --- a/ratis-server/src/main/java/org/apache/ratis/server/util/ServerStringUtils.java +++ b/ratis-server/src/main/java/org/apache/ratis/server/util/ServerStringUtils.java @@ -21,6 +21,7 @@ import org.apache.ratis.proto.RaftProtos.AppendEntriesRequestProto; import org.apache.ratis.proto.RaftProtos.InstallSnapshotReplyProto; import org.apache.ratis.proto.RaftProtos.InstallSnapshotRequestProto; +import org.apache.ratis.proto.RaftProtos.LogEntryProto; import org.apache.ratis.proto.RaftProtos.RequestVoteReplyProto; import org.apache.ratis.proto.RaftProtos.StateMachineLogEntryProto; import org.apache.ratis.protocol.RaftGroupMemberId; @@ -28,9 +29,7 @@ import org.apache.ratis.util.JavaUtils; import org.apache.ratis.util.ProtoUtils; -import java.util.function.Function; - -import static org.apache.ratis.server.raftlog.LogProtoUtils.toLogEntriesShortString; +import java.util.List; /** * This class provides convenient utilities for converting Protocol Buffers messages to strings. @@ -49,12 +48,13 @@ public static String toAppendEntriesRequestString(AppendEntriesRequestProto requ if (request == null) { return null; } + final List entries = request.getEntriesList(); return ProtoUtils.toString(request.getServerRequest()) + "-t" + request.getLeaderTerm() + ",previous=" + TermIndex.valueOf(request.getPreviousLog()) + ",leaderCommit=" + request.getLeaderCommit() + ",initializing? " + request.getInitializing() - + "," + toLogEntriesShortString(request.getEntriesList(), stateMachineToString); + + "," + (entries.isEmpty()? "HEARTBEAT" : "entries: " + LogProtoUtils.toLogEntriesShortString(entries)); } public static String toAppendEntriesReplyString(AppendEntriesReplyProto reply) { From 439ed9ad678737af473999cc63882533ee850057 Mon Sep 17 00:00:00 2001 From: Tsz-Wo Nicholas Sze Date: Wed, 7 Feb 2024 09:20:06 -0800 Subject: [PATCH 338/397] RATIS-2025. Move out assert and proto methods from RaftServerImpl. (#1041) (cherry picked from commit 16b6536e220ccbc8f9228b4fccfab8eb823ee83e) --- .../ratis/server/impl/RaftServerImpl.java | 111 +++++++----------- .../ratis/server/impl/ServerImplUtils.java | 5 - 2 files changed, 45 insertions(+), 71 deletions(-) diff --git a/ratis-server/src/main/java/org/apache/ratis/server/impl/RaftServerImpl.java b/ratis-server/src/main/java/org/apache/ratis/server/impl/RaftServerImpl.java index 927a6a83c6..966c70534d 100644 --- a/ratis-server/src/main/java/org/apache/ratis/server/impl/RaftServerImpl.java +++ b/ratis-server/src/main/java/org/apache/ratis/server/impl/RaftServerImpl.java @@ -18,7 +18,6 @@ package org.apache.ratis.server.impl; import org.apache.ratis.client.impl.ClientProtoUtils; -import org.apache.ratis.client.impl.OrderedAsync; import org.apache.ratis.conf.RaftProperties; import org.apache.ratis.metrics.Timekeeper; import org.apache.ratis.proto.RaftProtos.AppendEntriesReplyProto; @@ -29,7 +28,6 @@ import org.apache.ratis.proto.RaftProtos.InstallSnapshotRequestProto; import org.apache.ratis.proto.RaftProtos.InstallSnapshotResult; import org.apache.ratis.proto.RaftProtos.LogEntryProto; -import org.apache.ratis.proto.RaftProtos.LogInfoProto; import org.apache.ratis.proto.RaftProtos.RaftClientRequestProto; import org.apache.ratis.proto.RaftProtos.RaftClientRequestProto.TypeCase; import org.apache.ratis.proto.RaftProtos.RaftConfigurationProto; @@ -82,9 +80,7 @@ import org.apache.ratis.server.RaftServerRpc; import org.apache.ratis.server.impl.LeaderElection.Phase; import org.apache.ratis.server.impl.RetryCacheImpl.CacheEntry; -import org.apache.ratis.server.impl.ServerImplUtils.NavigableIndices; -import org.apache.ratis.server.leader.LeaderState.StepDownReason; -import org.apache.ratis.server.leader.LogAppender; +import org.apache.ratis.server.leader.LeaderState; import org.apache.ratis.server.metrics.LeaderElectionMetrics; import org.apache.ratis.server.metrics.RaftServerMetricsImpl; import org.apache.ratis.server.protocol.RaftServerAsynchronousProtocol; @@ -101,8 +97,6 @@ import org.apache.ratis.statemachine.impl.TransactionContextImpl; import org.apache.ratis.thirdparty.com.google.common.annotations.VisibleForTesting; import org.apache.ratis.thirdparty.com.google.protobuf.InvalidProtocolBufferException; -import org.apache.ratis.trace.TraceServer; -import org.apache.ratis.trace.TraceUtils; import org.apache.ratis.util.CodeInjectionForTesting; import org.apache.ratis.util.CollectionUtils; import org.apache.ratis.util.ConcurrentUtils; @@ -114,13 +108,45 @@ import org.apache.ratis.util.MemoizedSupplier; import org.apache.ratis.util.Preconditions; import org.apache.ratis.util.ProtoUtils; +import org.apache.ratis.util.ReferenceCountedObject; import org.apache.ratis.util.TimeDuration; import org.apache.ratis.util.function.CheckedSupplier; +import java.io.File; +import java.io.IOException; +import java.nio.file.NoSuchFileException; +import java.util.ArrayList; +import java.util.Collection; +import java.util.Collections; +import java.util.Comparator; +import java.util.List; +import java.util.Map; +import java.util.Objects; +import java.util.Optional; +import java.util.Set; +import java.util.concurrent.CompletableFuture; +import java.util.concurrent.CompletionException; +import java.util.concurrent.ExecutionException; +import java.util.concurrent.ExecutorService; +import java.util.concurrent.ThreadLocalRandom; +import java.util.concurrent.TimeUnit; +import java.util.concurrent.atomic.AtomicBoolean; +import java.util.function.Function; +import java.util.function.Supplier; +import java.util.stream.Collectors; +import java.util.stream.Stream; + +import static org.apache.ratis.server.impl.ServerImplUtils.assertEntries; +import static org.apache.ratis.server.impl.ServerImplUtils.assertGroup; import static org.apache.ratis.server.impl.ServerImplUtils.effectiveCommitIndex; import static org.apache.ratis.server.impl.ServerProtoUtils.toAppendEntriesReplyProto; +import static org.apache.ratis.server.impl.ServerProtoUtils.toReadIndexReplyProto; +import static org.apache.ratis.server.impl.ServerProtoUtils.toReadIndexRequestProto; +import static org.apache.ratis.server.impl.ServerProtoUtils.toRequestVoteReplyProto; +import static org.apache.ratis.server.impl.ServerProtoUtils.toStartLeaderElectionReplyProto; import static org.apache.ratis.server.util.ServerStringUtils.toAppendEntriesReplyString; import static org.apache.ratis.server.util.ServerStringUtils.toAppendEntriesRequestString; +import static org.apache.ratis.server.util.ServerStringUtils.toRequestVoteReplyString; class RaftServerImpl implements RaftServer.Division, RaftServerProtocol, RaftServerAsynchronousProtocol, @@ -735,8 +761,13 @@ private CompletableFuture checkLeaderState(RaftClientRequest re /** * @return null if the server is in leader state. */ - private CompletableFuture checkLeaderState( - RaftClientRequest request, CacheEntry entry, TransactionContextImpl context) { + private CompletableFuture checkLeaderState(RaftClientRequest request, CacheEntry entry) { + try { + assertGroup(getMemberId(), request); + } catch (GroupMismatchException e) { + return RetryCacheImpl.failWithException(e, entry); + } + if (!getInfo().isLeader()) { NotLeaderException exception = generateNotLeaderException(); final RaftClientReply reply = newExceptionReply(request, exception); @@ -781,44 +812,6 @@ void assertLifeCycleState(Set expected) throws ServerNotReadyEx getMemberId() + " is not in " + expected + ": current state is " + c), expected); } - private CompletableFuture getResourceUnavailableReply(String op, - RaftClientRequest request, CacheEntry entry, TransactionContextImpl context) { - final ResourceUnavailableException e = new ResourceUnavailableException(getMemberId() - + ": Failed to " + op + " for " + request); - cancelTransaction(context, e); - return entry.failWithException(e); - } - - private CompletableFuture failWithReply( - RaftClientReply reply, CacheEntry entry, TransactionContextImpl context) { - if (context != null) { - cancelTransaction(context, reply.getException()); - } - - if (entry == null) { - return CompletableFuture.completedFuture(reply); - } - entry.failWithReply(reply); - return entry.getReplyFuture(); - } - - /** Cancel a transaction and notify the state machine. Set exception if provided to the transaction context. */ - private void cancelTransaction(TransactionContextImpl context, Exception exception) { - if (context == null) { - return; - } - - if (exception != null) { - context.setException(exception); - } - - try { - context.cancelTransaction(); - } catch (IOException ioe) { - LOG.warn("{}: Failed to cancel transaction {}", getMemberId(), context, ioe); - } - } - /** * Handle a normal update request from client. */ @@ -1294,7 +1287,6 @@ CompletableFuture takeSnapshotAsync(SnapshotManagementRequest r LOG.info("{}: takeSnapshotAsync {}", getMemberId(), request); assertLifeCycleState(LifeCycle.States.RUNNING); assertGroup(getMemberId(), request); - Objects.requireNonNull(request.getCreate(), "create == null"); final long creationGap = request.getCreate().getCreationGap(); long minGapValue = creationGap > 0? creationGap : RaftServerConfigKeys.Snapshot.creationGap(proxy.getProperties()); @@ -1504,7 +1496,7 @@ private RequestVoteReplyProto requestVote(Phase phase, shouldShutdown = true; } reply = toRequestVoteReplyProto(candidateId, getMemberId(), - voteGranted, state.getCurrentTerm(), shouldShutdown, state.getLastEntry(), callId); + voteGranted, state.getCurrentTerm(), shouldShutdown); if (LOG.isInfoEnabled()) { LOG.info("{} replies to {} vote request: {}. Peer's state: {}", getMemberId(), phase, toRequestVoteReplyString(reply), state); @@ -1542,8 +1534,8 @@ public CompletableFuture appendEntriesAsync( if (!startComplete.get()) { throw new ServerNotReadyException(getMemberId() + ": The server role is not yet initialized."); } - assertGroup(leaderId, leaderGroupId); - validateEntries(r.getLeaderTerm(), previous, r.getEntriesList()); + assertGroup(getMemberId(), leaderId, leaderGroupId); + assertEntries(r, previous, state); return appendEntriesAsync(leaderId, request.getCallId(), previous, requestRef); } catch(Exception t) { @@ -1567,19 +1559,6 @@ public CompletableFuture readIndexAsync(ReadIndexRequestPro return getReadIndex(ClientProtoUtils.toRaftClientRequest(request.getClientRequest()), leader) .thenApply(index -> toReadIndexReplyProto(peerId, getMemberId(), true, index)) - .whenComplete((reply, exception) -> { - if (exception == null) { - // Leader should try to trigger heartbeat immediately after leader replies the ReadIndex to the follower - // so that the follower's commitIndex can be updated to the leader's commitIndex and the follower - // can start applying the logs up until the leader's commitIndex (instead of waiting for the next - // AppendEntries to happen through heartbeat or new transactions (which might increase the latency - // considerably)). - // Note that if the follower commitIndex is already equal to the leader's commitIndex, no heartbeat - // will be triggered, see GrpcLogAppender#isFollowerCommitBehindLastCommitIndex. - RaftPeerId requestorId = RaftPeerId.valueOf(reply.getServerReply().getRequestorId()); - leader.getLogAppender(requestorId).ifPresent(LogAppender::triggerHeartbeat); - } - }) .exceptionally(throwable -> toReadIndexReplyProto(peerId, getMemberId())); } @@ -1792,7 +1771,7 @@ public StartLeaderElectionReplyProto startLeaderElection(StartLeaderElectionRequ if (!request.hasLeaderLastEntry()) { // It should have a leaderLastEntry since there is a placeHolder entry. LOG.warn("{}: leaderLastEntry is missing in {}", getMemberId(), request); - return ServerProtoUtils.toStartLeaderElectionReplyProto(leaderId, getMemberId(), false); + return toStartLeaderElectionReplyProto(leaderId, getMemberId(), false); } final TermIndex leaderLastEntry = TermIndex.valueOf(request.getLeaderLastEntry()); @@ -1806,7 +1785,7 @@ public StartLeaderElectionReplyProto startLeaderElection(StartLeaderElectionRequ assertLifeCycleState(LifeCycle.States.STARTING_OR_RUNNING); final boolean recognized = state.recognizeLeader("startLeaderElection", leaderId, leaderLastEntry.getTerm()); if (!recognized) { - return ServerProtoUtils.toStartLeaderElectionReplyProto(leaderId, getMemberId(), false); + return toStartLeaderElectionReplyProto(leaderId, getMemberId(), false); } if (!getInfo().isFollower()) { diff --git a/ratis-server/src/main/java/org/apache/ratis/server/impl/ServerImplUtils.java b/ratis-server/src/main/java/org/apache/ratis/server/impl/ServerImplUtils.java index 434f98d683..d6035ff778 100644 --- a/ratis-server/src/main/java/org/apache/ratis/server/impl/ServerImplUtils.java +++ b/ratis-server/src/main/java/org/apache/ratis/server/impl/ServerImplUtils.java @@ -43,11 +43,6 @@ import java.util.ArrayList; import java.util.Collections; import java.util.List; -import java.util.Map; -import java.util.NavigableMap; -import java.util.TreeMap; -import java.util.Objects; -import java.util.concurrent.CompletableFuture; import java.util.concurrent.TimeUnit; import java.util.concurrent.atomic.AtomicReference; import java.util.function.Function; From 5907705a1b03efd2e4bd343ca93b11dcbeedeea5 Mon Sep 17 00:00:00 2001 From: Nandakumar Vadivelu Date: Tue, 20 Feb 2024 22:41:53 +0530 Subject: [PATCH 339/397] RATIS-2021. Migrate ratis-test tests to Junit 5 - Part 2. (#1043) (cherry picked from commit 67b76509d1d5b0037b278f1808bb0f5ecd7aceed) --- .../java/org/apache/ratis/RaftBasicTests.java | 1 - .../retry/TestMultipleLinearRandomRetry.java | 10 +- .../apache/ratis/retry/TestRetryPolicy.java | 1 - .../ratis/server/ServerRestartTests.java | 4 +- .../ratis/server/impl/TestRaftServerJmx.java | 6 +- .../raftlog/segmented/TestCacheEviction.java | 24 +-- .../raftlog/segmented/TestLogSegment.java | 4 +- .../segmented/TestSegmentedRaftLog.java | 153 ++---------------- .../segmented/TestSegmentedRaftLogCache.java | 8 +- .../org/apache/ratis/util/TestLifeCycle.java | 7 +- 10 files changed, 42 insertions(+), 176 deletions(-) diff --git a/ratis-server/src/test/java/org/apache/ratis/RaftBasicTests.java b/ratis-server/src/test/java/org/apache/ratis/RaftBasicTests.java index c71b57e826..7fdc17a9f8 100644 --- a/ratis-server/src/test/java/org/apache/ratis/RaftBasicTests.java +++ b/ratis-server/src/test/java/org/apache/ratis/RaftBasicTests.java @@ -46,7 +46,6 @@ import org.junit.jupiter.api.Assertions; import org.junit.jupiter.api.Assumptions; import org.junit.jupiter.api.Test; -import org.junit.jupiter.api.Timeout; import org.slf4j.Logger; import org.slf4j.event.Level; diff --git a/ratis-test/src/test/java/org/apache/ratis/retry/TestMultipleLinearRandomRetry.java b/ratis-test/src/test/java/org/apache/ratis/retry/TestMultipleLinearRandomRetry.java index 621d46b5b3..2a37062602 100644 --- a/ratis-test/src/test/java/org/apache/ratis/retry/TestMultipleLinearRandomRetry.java +++ b/ratis-test/src/test/java/org/apache/ratis/retry/TestMultipleLinearRandomRetry.java @@ -21,7 +21,6 @@ import org.apache.ratis.util.TimeDuration; import org.junit.jupiter.api.Assertions; import org.junit.jupiter.api.Test; -import org.junit.jupiter.api.Timeout; @Timeout(value = 1) public class TestMultipleLinearRandomRetry extends BaseTest { @@ -44,12 +43,9 @@ public void testParseCommaSeparated() { assertLegalInput("[10x100ms, 20x1s, 30x5s]", "100,10, 1s,20, 5s,30"); } - private void assertIllegalInput(String input) { - try { - MultipleLinearRandomRetry.parseCommaSeparated(input); - } catch (IllegalArgumentException e) { - LOG.info("Expected to catch: {}", String.valueOf(e)); - } + private static void assertIllegalInput(String input) { + final MultipleLinearRandomRetry computed = MultipleLinearRandomRetry.parseCommaSeparated(input); + Assertions.assertNull(computed); } private static MultipleLinearRandomRetry assertLegalInput(String expected, String input) { diff --git a/ratis-test/src/test/java/org/apache/ratis/retry/TestRetryPolicy.java b/ratis-test/src/test/java/org/apache/ratis/retry/TestRetryPolicy.java index 43b2fedd1a..e1d41045ff 100644 --- a/ratis-test/src/test/java/org/apache/ratis/retry/TestRetryPolicy.java +++ b/ratis-test/src/test/java/org/apache/ratis/retry/TestRetryPolicy.java @@ -36,7 +36,6 @@ import org.apache.ratis.util.Timestamp; import org.junit.jupiter.api.Assertions; import org.junit.jupiter.api.Test; -import org.junit.jupiter.api.Timeout; import java.io.IOException; import java.util.ArrayList; diff --git a/ratis-test/src/test/java/org/apache/ratis/server/ServerRestartTests.java b/ratis-test/src/test/java/org/apache/ratis/server/ServerRestartTests.java index c9495fdf79..592196034e 100644 --- a/ratis-test/src/test/java/org/apache/ratis/server/ServerRestartTests.java +++ b/ratis-test/src/test/java/org/apache/ratis/server/ServerRestartTests.java @@ -176,7 +176,7 @@ static void assertTruncatedLog(RaftPeerId id, File openLogFile, long lastIndex, } static File getOpenLogFile(RaftServer.Division server) throws Exception { - final List openLogs = SegmentedRaftLogTestUtils.getOpenLogFiles(server); + final List openLogs = getOpenLogFiles(server); Assertions.assertEquals(1, openLogs.size()); return openLogs.get(0).toFile(); } @@ -203,7 +203,7 @@ void runTestRestartWithCorruptedLogHeader(MiniRaftCluster cluster) throws Except 10, HUNDRED_MILLIS, impl.getId() + "-getOpenLogFile", LOG); for(int i = 0; i < SegmentedRaftLogFormat.getHeaderLength(); i++) { assertCorruptedLogHeader(impl.getId(), openLogFile, i, cluster, LOG); - Assertions.assertTrue(SegmentedRaftLogTestUtils.getOpenLogFiles(impl).isEmpty()); + Assertions.assertTrue(getOpenLogFiles(impl).isEmpty()); } } } diff --git a/ratis-test/src/test/java/org/apache/ratis/server/impl/TestRaftServerJmx.java b/ratis-test/src/test/java/org/apache/ratis/server/impl/TestRaftServerJmx.java index b24ec72c06..4221d85cfe 100644 --- a/ratis-test/src/test/java/org/apache/ratis/server/impl/TestRaftServerJmx.java +++ b/ratis-test/src/test/java/org/apache/ratis/server/impl/TestRaftServerJmx.java @@ -40,7 +40,7 @@ public class TestRaftServerJmx extends BaseTest { @Test - @Timeout(value = 30) + @Timeout(value = 30000) public void testJmxBeans() throws Exception { final int numServers = 3; final MiniRaftClusterWithSimulatedRpc cluster @@ -50,7 +50,7 @@ public void testJmxBeans() throws Exception { MBeanServer platformMBeanServer = ManagementFactory.getPlatformMBeanServer(); Set objectInstances = platformMBeanServer.queryMBeans(new ObjectName("Ratis:*"), null); - Assertions.assertEquals(numServers, objectInstances.size()); + Assertions.assertEquals(NUM_SERVERS, objectInstances.size()); for (ObjectInstance instance : objectInstances) { Object groupId = platformMBeanServer.getAttribute(instance.getObjectName(), "GroupId"); @@ -60,7 +60,7 @@ public void testJmxBeans() throws Exception { } @Test - @Timeout(value = 30) + @Timeout(value = 30000) public void testRegister() throws JMException { { final JmxRegister jmx = new JmxRegister(); diff --git a/ratis-test/src/test/java/org/apache/ratis/server/raftlog/segmented/TestCacheEviction.java b/ratis-test/src/test/java/org/apache/ratis/server/raftlog/segmented/TestCacheEviction.java index ae8bf34323..e1afe4c273 100644 --- a/ratis-test/src/test/java/org/apache/ratis/server/raftlog/segmented/TestCacheEviction.java +++ b/ratis-test/src/test/java/org/apache/ratis/server/raftlog/segmented/TestCacheEviction.java @@ -77,35 +77,35 @@ public void testBasicEviction() throws Exception { new boolean[]{true, true, true, true, true}, 0, 10); // case 1, make sure we do not evict cache for segments behind local flushed index - List evicted = POLICY.evict(null, 5, 15, segments, maxCached); + List evicted = policy.evict(null, 5, 15, segments, maxCached); Assertions.assertEquals(0, evicted.size()); // case 2, suppose the local flushed index is in the 3rd segment, then we // can evict the first two segment - evicted = POLICY.evict(null, 25, 30, segments, maxCached); + evicted = policy.evict(null, 25, 30, segments, maxCached); Assertions.assertEquals(2, evicted.size()); Assertions.assertSame(evicted.get(0), segments.get(0)); Assertions.assertSame(evicted.get(1), segments.get(1)); // case 3, similar with case 2, but the local applied index is less than // the local flushed index. - evicted = POLICY.evict(null, 25, 15, segments, maxCached); + evicted = policy.evict(null, 25, 15, segments, maxCached); Assertions.assertEquals(1, evicted.size()); Assertions.assertSame(evicted.get(0), segments.get(0)); // case 4, the local applied index is very small, then evict cache behind it // first and let the state machine load the segments later - evicted = POLICY.evict(null, 35, 5, segments, maxCached); + evicted = policy.evict(null, 35, 5, segments, maxCached); Assertions.assertEquals(1, evicted.size()); Assertions.assertSame(evicted.get(0), segments.get(2)); Mockito.when(segments.get(2).hasCache()).thenReturn(false); - evicted = POLICY.evict(null, 35, 5, segments, maxCached); + evicted = policy.evict(null, 35, 5, segments, maxCached); Assertions.assertEquals(1, evicted.size()); Assertions.assertSame(evicted.get(0), segments.get(1)); Mockito.when(segments.get(1).hasCache()).thenReturn(false); - evicted = POLICY.evict(null, 35, 5, segments, maxCached); + evicted = policy.evict(null, 35, 5, segments, maxCached); Assertions.assertEquals(0, evicted.size()); } @@ -122,33 +122,33 @@ public void testEvictionWithFollowerIndices() throws Exception { Assertions.assertEquals(0, evicted.size()); // case 2, the follower indices are behind the local flushed index - evicted = POLICY.evict(new long[]{30, 40, 45}, 25, 30, segments, maxCached); + evicted = policy.evict(new long[]{30, 40, 45}, 25, 30, segments, maxCached); Assertions.assertEquals(2, evicted.size()); Assertions.assertSame(evicted.get(0), segments.get(0)); Assertions.assertSame(evicted.get(1), segments.get(1)); // case 3, similar with case 3 in basic eviction test - evicted = POLICY.evict(new long[]{30, 40, 45}, 25, 15, segments, maxCached); + evicted = policy.evict(new long[]{30, 40, 45}, 25, 15, segments, maxCached); Assertions.assertEquals(1, evicted.size()); Assertions.assertSame(evicted.get(0), segments.get(0)); // case 4, the followers are slower than local flush - evicted = POLICY.evict(new long[]{15, 45, 45}, 55, 50, segments, maxCached); + evicted = policy.evict(new long[]{15, 45, 45}, 55, 50, segments, maxCached); Assertions.assertEquals(1, evicted.size()); Assertions.assertSame(evicted.get(0), segments.get(0)); Mockito.when(segments.get(0).hasCache()).thenReturn(false); - evicted = POLICY.evict(new long[]{15, 45, 45}, 55, 50, segments, maxCached); + evicted = policy.evict(new long[]{15, 45, 45}, 55, 50, segments, maxCached); Assertions.assertEquals(1, evicted.size()); Assertions.assertSame(evicted.get(0), segments.get(2)); Mockito.when(segments.get(2).hasCache()).thenReturn(false); - evicted = POLICY.evict(new long[]{15, 45, 45}, 55, 50, segments, maxCached); + evicted = policy.evict(new long[]{15, 45, 45}, 55, 50, segments, maxCached); Assertions.assertEquals(1, evicted.size()); Assertions.assertSame(evicted.get(0), segments.get(3)); Mockito.when(segments.get(3).hasCache()).thenReturn(false); - evicted = POLICY.evict(new long[]{15, 45, 45}, 55, 50, segments, maxCached); + evicted = policy.evict(new long[]{15, 45, 45}, 55, 50, segments, maxCached); Assertions.assertEquals(0, evicted.size()); } diff --git a/ratis-test/src/test/java/org/apache/ratis/server/raftlog/segmented/TestLogSegment.java b/ratis-test/src/test/java/org/apache/ratis/server/raftlog/segmented/TestLogSegment.java index 6a75dfb36e..ec1f925eb8 100644 --- a/ratis-test/src/test/java/org/apache/ratis/server/raftlog/segmented/TestLogSegment.java +++ b/ratis-test/src/test/java/org/apache/ratis/server/raftlog/segmented/TestLogSegment.java @@ -238,7 +238,7 @@ public void testAppendWithGap() throws Exception { final StateMachineLogEntryProto m = op.getLogEntryContent(); try { LogEntryProto entry = LogProtoUtils.toLogEntryProto(m, 0, 1001); - segment.appendToOpenSegment(entry, LogSegment.Op.WRITE_CACHE_WITHOUT_STATE_MACHINE_CACHE, true); + segment.appendToOpenSegment(Op.WRITE_CACHE_WITHOUT_STATE_MACHINE_CACHE, ReferenceCountedObject.wrap(entry)); Assertions.fail("should fail since the entry's index needs to be 1000"); } catch (IllegalStateException e) { // the exception is expected. @@ -249,7 +249,7 @@ public void testAppendWithGap() throws Exception { try { entry = LogProtoUtils.toLogEntryProto(m, 0, 1002); - segment.appendToOpenSegment(entry, LogSegment.Op.WRITE_CACHE_WITHOUT_STATE_MACHINE_CACHE, true); + segment.appendToOpenSegment(Op.WRITE_CACHE_WITHOUT_STATE_MACHINE_CACHE, ReferenceCountedObject.wrap(entry)); Assertions.fail("should fail since the entry's index needs to be 1001"); } catch (IllegalStateException e) { // the exception is expected. diff --git a/ratis-test/src/test/java/org/apache/ratis/server/raftlog/segmented/TestSegmentedRaftLog.java b/ratis-test/src/test/java/org/apache/ratis/server/raftlog/segmented/TestSegmentedRaftLog.java index d268377173..5bcfd12de7 100644 --- a/ratis-test/src/test/java/org/apache/ratis/server/raftlog/segmented/TestSegmentedRaftLog.java +++ b/ratis-test/src/test/java/org/apache/ratis/server/raftlog/segmented/TestSegmentedRaftLog.java @@ -77,15 +77,10 @@ import org.junit.jupiter.params.ParameterizedTest; import org.junit.jupiter.params.provider.Arguments; import org.junit.jupiter.params.provider.MethodSource; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; import org.slf4j.event.Level; import static java.lang.Boolean.FALSE; import static java.lang.Boolean.TRUE; -import static org.apache.ratis.server.raftlog.segmented.SegmentedRaftLogWorker.RUN_WORKER; -import static org.junit.jupiter.api.Assertions.assertEquals; -import static org.junit.jupiter.api.Assertions.assertThrows; import static org.junit.jupiter.api.Assertions.assertTrue; import static org.junit.jupiter.params.provider.Arguments.arguments; @@ -234,7 +229,7 @@ public void testLoadLogSegments(Boolean useAsyncFlush, Boolean smSyncFlush) thro // check if log entries are loaded correctly for (LogEntryProto e : entries) { LogEntryProto entry = raftLog.get(e.getIndex()); - assertEquals(e, entry); + Assertions.assertEquals(e, entry); } final LogEntryHeader[] termIndices = raftLog.getEntries(0, 500); @@ -248,7 +243,7 @@ public void testLoadLogSegments(Boolean useAsyncFlush, Boolean smSyncFlush) thro }) .toArray(LogEntryProto[]::new); Assertions.assertArrayEquals(entries, entriesFromLog); - assertEquals(entries[entries.length - 1], getLastEntry(raftLog)); + Assertions.assertEquals(entries[entries.length - 1], getLastEntry(raftLog)); final RatisMetricRegistry metricRegistryForLogWorker = RaftLogMetricsBase.createRegistry(MEMBER_ID); @@ -403,83 +398,7 @@ public void testAppendAndRoll(Boolean useAsyncFlush, Boolean smSyncFlush) throws raftLog.open(RaftLog.INVALID_LOG_INDEX, null); // check if the raft log is correct checkEntries(raftLog, entries, 0, entries.size()); - assertEquals(9, raftLog.getRaftLogCache().getNumOfSegments()); - } - } - - @ParameterizedTest - @MethodSource("data") - public void testPurgeAfterAppendEntry(Boolean useAsyncFlush, Boolean smSyncFlush) throws Exception { - RaftServerConfigKeys.Log.setAsyncFlushEnabled(properties, useAsyncFlush); - RaftServerConfigKeys.Log.StateMachineData.setSync(properties, smSyncFlush); - RaftServerConfigKeys.Log.setPurgeGap(properties, 1); - RaftServerConfigKeys.Log.setForceSyncNum(properties, 128); - - int startTerm = 0; - int endTerm = 2; - int segmentSize = 10; - long endIndexOfClosedSegment = segmentSize * (endTerm - startTerm - 1); - long nextStartIndex = segmentSize * (endTerm - startTerm); - - // append entries and roll logSegment for later purge operation - List ranges0 = prepareRanges(startTerm, endTerm, segmentSize, 0); - List entries0 = prepareLogEntries(ranges0, null); - try (SegmentedRaftLog raftLog = newSegmentedRaftLog()) { - raftLog.open(RaftLog.INVALID_LOG_INDEX, null); - entries0.stream().map(raftLog::appendEntry).forEach(CompletableFuture::join); - } - - // test the pattern in the task queue of SegmentedRaftLogWorker: (WriteLog, ..., PurgeLog) - List ranges = prepareRanges(endTerm - 1, endTerm, 1, nextStartIndex); - List entries = prepareLogEntries(ranges, null); - - try (SegmentedRaftLog raftLog = newSegmentedRaftLog()) { - final CountDownLatch raftLogOpened = new CountDownLatch(1); - final CountDownLatch tasksAdded = new CountDownLatch(1); - - // inject test code to make the pattern (WriteLog, PurgeLog) - final ConcurrentLinkedQueue> appendFutures = new ConcurrentLinkedQueue<>(); - final AtomicReference> purgeFuture = new AtomicReference<>(); - final AtomicInteger tasksCount = new AtomicInteger(0); - CodeInjectionForTesting.put(RUN_WORKER, (localId, remoteId, args) -> { - // wait for raftLog to be opened - try { - if(!raftLogOpened.await(FIVE_SECONDS.getDuration(), FIVE_SECONDS.getUnit())) { - throw new TimeoutException(); - } - } catch (InterruptedException | TimeoutException e) { - LOG.error("an exception occurred", e); - throw new RuntimeException(e); - } - - // add WriteLog and PurgeLog tasks - entries.stream().map(raftLog::appendEntry).forEach(appendFutures::add); - purgeFuture.set(raftLog.purge(endIndexOfClosedSegment)); - - tasksCount.set(((DataBlockingQueue) args[0]).getNumElements()); - tasksAdded.countDown(); - return true; - }); - - // open raftLog - raftLog.open(RaftLog.INVALID_LOG_INDEX, null); - raftLogOpened.countDown(); - - // wait for all tasks to be added - if(!tasksAdded.await(FIVE_SECONDS.getDuration(), FIVE_SECONDS.getUnit())) { - throw new TimeoutException(); - } - assertEquals(entries.size() + 1, tasksCount.get()); - - // check if the purge task is executed - final Long purged = purgeFuture.get().get(); - LOG.info("purgeIndex = {}, purged = {}", endIndexOfClosedSegment, purged); - assertEquals(endIndexOfClosedSegment, raftLog.getRaftLogCache().getStartIndex()); - - // check if the appendEntry futures are done - JavaUtils.allOf(appendFutures).get(FIVE_SECONDS.getDuration(), FIVE_SECONDS.getUnit()); - } finally { - CodeInjectionForTesting.put(RUN_WORKER, (localId, remoteId, args) -> false); + Assertions.assertEquals(9, raftLog.getRaftLogCache().getNumOfSegments()); } } @@ -518,7 +437,7 @@ private void testTruncate(List entries, long fromIndex) raftLog.open(RaftLog.INVALID_LOG_INDEX, null); // check if the raft log is correct if (fromIndex > 0) { - assertEquals(entries.get((int) (fromIndex - 1)), + Assertions.assertEquals(entries.get((int) (fromIndex - 1)), getLastEntry(raftLog)); } else { Assertions.assertNull(raftLog.getLastEntryTermIndex()); @@ -532,7 +451,7 @@ private void checkEntries(RaftLog raftLog, List expected, if (size > 0) { for (int i = offset; i < size + offset; i++) { LogEntryProto entry = raftLog.get(expected.get(i).getIndex()); - assertEquals(expected.get(i), entry); + Assertions.assertEquals(expected.get(i), entry); } final LogEntryHeader[] termIndices = raftLog.getEntries( expected.get(offset).getIndex(), @@ -640,7 +559,7 @@ private void purgeAndVerify(int startTerm, int endTerm, int segmentSize, int pur final CompletableFuture f = raftLog.purge(purgeIndex); final Long purged = f.get(); LOG.info("purgeIndex = {}, purged = {}", purgeIndex, purged); - assertEquals(expectedIndex, raftLog.getRaftLogCache().getStartIndex()); + Assertions.assertEquals(expectedIndex, raftLog.getRaftLogCache().getStartIndex()); } } @@ -684,9 +603,9 @@ public void testAppendEntriesWithInconsistency(Boolean useAsyncFlush, Boolean sm checkFailedEntries(entries, 650, retryCache); checkEntries(raftLog, entries, 0, 650); checkEntries(raftLog, newEntries, 100, 100); - assertEquals(newEntries.get(newEntries.size() - 1), + Assertions.assertEquals(newEntries.get(newEntries.size() - 1), getLastEntry(raftLog)); - assertEquals(newEntries.get(newEntries.size() - 1).getIndex(), + Assertions.assertEquals(newEntries.get(newEntries.size() - 1).getIndex(), raftLog.getFlushIndex()); } @@ -696,57 +615,13 @@ public void testAppendEntriesWithInconsistency(Boolean useAsyncFlush, Boolean sm raftLog.open(RaftLog.INVALID_LOG_INDEX, null); checkEntries(raftLog, entries, 0, 650); checkEntries(raftLog, newEntries, 100, 100); - assertEquals(newEntries.get(newEntries.size() - 1), + Assertions.assertEquals(newEntries.get(newEntries.size() - 1), getLastEntry(raftLog)); - assertEquals(newEntries.get(newEntries.size() - 1).getIndex(), + Assertions.assertEquals(newEntries.get(newEntries.size() - 1).getIndex(), raftLog.getFlushIndex()); SegmentedRaftLogCache cache = raftLog.getRaftLogCache(); - assertEquals(5, cache.getNumOfSegments()); - } - } - - @ParameterizedTest - @MethodSource("data") - public void testAppendEntriesWithGap(Boolean useAsyncFlush, Boolean smSyncFlush) throws Exception { - RaftServerConfigKeys.Log.setAsyncFlushEnabled(properties, useAsyncFlush); - RaftServerConfigKeys.Log.StateMachineData.setSync(properties, smSyncFlush); - // prepare the log for truncation - List ranges = prepareRanges(0, 5, 200, 0); - List entries = prepareLogEntries(ranges, null); - - final RetryCache retryCache = RetryCacheTestUtil.createRetryCache(); - try (SegmentedRaftLog raftLog = - RetryCacheTestUtil.newSegmentedRaftLog(MEMBER_ID, retryCache, storage, properties)) { - raftLog.open(RaftLog.INVALID_LOG_INDEX, null); - entries.forEach(entry -> RetryCacheTestUtil.createEntry(retryCache, entry)); - // append entries to the raftlog - entries.stream().map(raftLog::appendEntry).forEach(CompletableFuture::join); - } - - long lastIndex = ranges.get(ranges.size() - 1).end; - long snapshotIndex = lastIndex + 100; - LogEntryProto entryProto = prepareLogEntry(4, snapshotIndex + 1, null, false); - final LongSupplier getSnapshotIndexFromStateMachine = new LongSupplier() { - @Override - public long getAsLong() { - return snapshotIndex; - } - }; - try (SegmentedRaftLog raftLog = newSegmentedRaftLog(getSnapshotIndexFromStateMachine)) { - raftLog.open(RaftLog.INVALID_LOG_INDEX, null); - // Assert the wrapped exception - IllegalStateException exception = assertThrows(IllegalStateException.class, - () -> raftLog.appendEntry(entryProto)); - // Assert the original cause - assertTrue(exception.getMessage().contains("gap between entries")); - } - - // load the raftlog again and check - try (SegmentedRaftLog raftLog = - RetryCacheTestUtil.newSegmentedRaftLog(MEMBER_ID, retryCache, storage, properties)) { - raftLog.open(RaftLog.INVALID_LOG_INDEX, null); - Assertions.assertEquals(lastIndex, raftLog.getRaftLogCache().getEndIndex()); + Assertions.assertEquals(5, cache.getNumOfSegments()); } } @@ -842,7 +717,7 @@ public void notifyLogFailed(Throwable cause, LogEntryProto entry) { // SegmentedRaftLogWorker should catch TimeoutIOException CompletableFuture f = raftLog.appendEntry(entry); // Wait for async writeStateMachineData to finish - ex = assertThrows(ExecutionException.class, f::get); + ex = Assertions.assertThrows(ExecutionException.class, f::get); } Assertions.assertSame(LifeCycle.State.PAUSED, sm.getLifeCycleState()); Assertions.assertInstanceOf(TimeoutIOException.class, ex.getCause()); @@ -862,9 +737,9 @@ static Thread startAppendEntryThread(RaftLog raftLog, LogEntryProto entry) { void assertIndices(RaftLog raftLog, long expectedFlushIndex, long expectedNextIndex) { LOG.info("assert expectedFlushIndex={}", expectedFlushIndex); - assertEquals(expectedFlushIndex, raftLog.getFlushIndex()); + Assertions.assertEquals(expectedFlushIndex, raftLog.getFlushIndex()); LOG.info("assert expectedNextIndex={}", expectedNextIndex); - assertEquals(expectedNextIndex, raftLog.getNextIndex()); + Assertions.assertEquals(expectedNextIndex, raftLog.getNextIndex()); } void assertIndicesMultipleAttempts(RaftLog raftLog, long expectedFlushIndex, long expectedNextIndex) diff --git a/ratis-test/src/test/java/org/apache/ratis/server/raftlog/segmented/TestSegmentedRaftLogCache.java b/ratis-test/src/test/java/org/apache/ratis/server/raftlog/segmented/TestSegmentedRaftLogCache.java index 3133fb36f6..d8924d71b4 100644 --- a/ratis-test/src/test/java/org/apache/ratis/server/raftlog/segmented/TestSegmentedRaftLogCache.java +++ b/ratis-test/src/test/java/org/apache/ratis/server/raftlog/segmented/TestSegmentedRaftLogCache.java @@ -34,6 +34,7 @@ import org.apache.ratis.server.raftlog.segmented.SegmentedRaftLogCache.TruncationSegments; import org.apache.ratis.server.raftlog.segmented.LogSegment.LogRecord; import org.apache.ratis.proto.RaftProtos.LogEntryProto; +import org.apache.ratis.util.ReferenceCountedObject; import org.junit.jupiter.api.AfterEach; import org.junit.jupiter.api.Assertions; import org.junit.jupiter.api.BeforeEach; @@ -154,9 +155,10 @@ public void testAppendEntry() throws Exception { final SimpleOperation m = new SimpleOperation("m"); try { LogEntryProto entry = LogProtoUtils.toLogEntryProto(m.getLogEntryContent(), 0, 0); - cache.appendEntry(entry, LogSegment.Op.WRITE_CACHE_WITHOUT_STATE_MACHINE_CACHE); + cache.appendEntry(Op.WRITE_CACHE_WITHOUT_STATE_MACHINE_CACHE, ReferenceCountedObject.wrap(entry) + ); Assertions.fail("the open segment is null"); - } catch (IllegalStateException | NullPointerException ignored) { + } catch (IllegalStateException ignored) { } LogSegment openSegment = prepareLogSegment(100, 100, true); @@ -286,7 +288,7 @@ private void testIterator(long startIndex) { TermIndex prev = null; while (iterator.hasNext()) { TermIndex termIndex = iterator.next(); - Assertions.assertEquals(cache.getTermIndex(termIndex.getIndex()), termIndex); + Assertions.assertEquals(cache.getLogRecord(termIndex.getIndex()).getTermIndex(), termIndex); if (prev != null) { Assertions.assertEquals(prev.getIndex() + 1, termIndex.getIndex()); } diff --git a/ratis-test/src/test/java/org/apache/ratis/util/TestLifeCycle.java b/ratis-test/src/test/java/org/apache/ratis/util/TestLifeCycle.java index 201b510571..9f61905e5e 100644 --- a/ratis-test/src/test/java/org/apache/ratis/util/TestLifeCycle.java +++ b/ratis-test/src/test/java/org/apache/ratis/util/TestLifeCycle.java @@ -21,11 +21,6 @@ import org.junit.jupiter.api.Test; import org.junit.jupiter.api.Timeout; -import java.util.Arrays; -import java.util.EnumMap; -import java.util.List; -import java.util.Map; - import static org.apache.ratis.util.LifeCycle.State.*; import static org.junit.jupiter.api.Assertions.assertEquals; import static org.junit.jupiter.api.Assertions.assertFalse; @@ -39,7 +34,7 @@ public class TestLifeCycle { * while this test uses successors. */ @Test - @Timeout(value = 1) + @Timeout(value = 1000) public void testIsValid() { final Map> successors = new EnumMap<>(LifeCycle.State.class); From 20e2bae283821a5dae15b283d0be4b1029a31ce9 Mon Sep 17 00:00:00 2001 From: Duong Nguyen Date: Wed, 21 Feb 2024 10:27:25 -0800 Subject: [PATCH 340/397] RATIS-2020. Refactor TransactionContext to supply LogEntryProto via a ReferenceCountedObject (#1042) (cherry picked from commit 53d35c01e7bffeb596585cb981a2a89f685e9bef) --- .../arithmetic/ArithmeticStateMachine.java | 2 +- .../counter/server/CounterStateMachine.java | 2 +- .../filestore/FileStoreStateMachine.java | 2 +- .../statemachine/TransactionContext.java | 41 ++++++++++++++++- .../ratis/server/impl/LeaderStateImpl.java | 2 +- .../ratis/server/impl/PendingRequest.java | 2 +- .../ratis/server/impl/RaftServerImpl.java | 6 +++ .../apache/ratis/server/impl/ServerState.java | 2 +- .../ratis/server/raftlog/LogProtoUtils.java | 18 ++++++++ .../statemachine/impl/BaseStateMachine.java | 6 +-- .../impl/TransactionContextImpl.java | 44 ++++++++++++++++++- .../apache/ratis/ReadOnlyRequestTests.java | 12 ++--- .../impl/StateMachineShutdownTests.java | 10 ++--- .../impl/SimpleStateMachine4Testing.java | 31 +++++-------- .../ratis/datastream/DataStreamTestUtils.java | 2 +- .../ratis/statemachine/TestStateMachine.java | 2 +- 16 files changed, 138 insertions(+), 46 deletions(-) diff --git a/ratis-examples/src/main/java/org/apache/ratis/examples/arithmetic/ArithmeticStateMachine.java b/ratis-examples/src/main/java/org/apache/ratis/examples/arithmetic/ArithmeticStateMachine.java index fa0dc6d8e2..c4adff5987 100644 --- a/ratis-examples/src/main/java/org/apache/ratis/examples/arithmetic/ArithmeticStateMachine.java +++ b/ratis-examples/src/main/java/org/apache/ratis/examples/arithmetic/ArithmeticStateMachine.java @@ -164,7 +164,7 @@ public void close() { @Override public CompletableFuture applyTransaction(TransactionContext trx) { - final LogEntryProto entry = trx.getLogEntry(); + final LogEntryProto entry = trx.getLogEntryUnsafe(); final AssignmentMessage assignment = new AssignmentMessage(entry.getStateMachineLogEntry().getLogData()); final long index = entry.getIndex(); diff --git a/ratis-examples/src/main/java/org/apache/ratis/examples/counter/server/CounterStateMachine.java b/ratis-examples/src/main/java/org/apache/ratis/examples/counter/server/CounterStateMachine.java index 914180feb8..e8b09c77ae 100644 --- a/ratis-examples/src/main/java/org/apache/ratis/examples/counter/server/CounterStateMachine.java +++ b/ratis-examples/src/main/java/org/apache/ratis/examples/counter/server/CounterStateMachine.java @@ -261,7 +261,7 @@ public TransactionContext startTransaction(RaftClientRequest request) throws IOE */ @Override public CompletableFuture applyTransaction(TransactionContext trx) { - final LogEntryProto entry = trx.getLogEntry(); + final LogEntryProto entry = trx.getLogEntryUnsafe(); //increment the counter and update term-index final TermIndex termIndex = TermIndex.valueOf(entry); final int incremented = incrementCounter(termIndex); diff --git a/ratis-examples/src/main/java/org/apache/ratis/examples/filestore/FileStoreStateMachine.java b/ratis-examples/src/main/java/org/apache/ratis/examples/filestore/FileStoreStateMachine.java index 941b8b9123..5ccebe8e12 100644 --- a/ratis-examples/src/main/java/org/apache/ratis/examples/filestore/FileStoreStateMachine.java +++ b/ratis-examples/src/main/java/org/apache/ratis/examples/filestore/FileStoreStateMachine.java @@ -218,7 +218,7 @@ public CompletableFuture link(DataStream stream, LogEntryProto entry) { @Override public CompletableFuture applyTransaction(TransactionContext trx) { - final LogEntryProto entry = trx.getLogEntry(); + final LogEntryProto entry = trx.getLogEntryUnsafe(); final long index = entry.getIndex(); updateLastAppliedTermIndex(entry.getTerm(), index); diff --git a/ratis-server-api/src/main/java/org/apache/ratis/statemachine/TransactionContext.java b/ratis-server-api/src/main/java/org/apache/ratis/statemachine/TransactionContext.java index 35a40efb57..b9e70e2a59 100644 --- a/ratis-server-api/src/main/java/org/apache/ratis/statemachine/TransactionContext.java +++ b/ratis-server-api/src/main/java/org/apache/ratis/statemachine/TransactionContext.java @@ -27,6 +27,7 @@ import java.io.IOException; import java.util.Objects; +import java.util.Optional; /** * Context for a transaction. @@ -93,11 +94,47 @@ public interface TransactionContext { LogEntryProto initLogEntry(long term, long index); /** - * Returns the committed log entry - * @return the committed log entry + * @return a copy of the committed log entry if it exists; otherwise, returns null + * + * @deprecated Use {@link #getLogEntryRef()} or {@link #getLogEntryUnsafe()} to avoid copying. */ + @Deprecated LogEntryProto getLogEntry(); + /** + * @return the committed log entry if it exists; otherwise, returns null. + * The returned value is safe to use only before {@link StateMachine#applyTransaction} returns. + * Once {@link StateMachine#applyTransaction} has returned, it is unsafe to use the log entry + * since the underlying buffers can possiby be released. + */ + default LogEntryProto getLogEntryUnsafe() { + return getLogEntryRef().get(); + } + + /** + * Get a {@link ReferenceCountedObject} to the committed log entry. + * + * It is safe to access the log entry by calling {@link ReferenceCountedObject#get()} + * (without {@link ReferenceCountedObject#retain()}) + * inside the scope of {@link StateMachine#applyTransaction}. + * + * If the log entry is needed after {@link StateMachine#applyTransaction} returns, + * e.g. for asynchronous computation or caching, + * the caller must invoke {@link ReferenceCountedObject#retain()} and {@link ReferenceCountedObject#release()}. + * + * @return a reference to the committed log entry if it exists; otherwise, returns null. + */ + default ReferenceCountedObject getLogEntryRef() { + return Optional.ofNullable(getLogEntryUnsafe()).map(this::wrap).orElse(null); + } + + /** Wrap the given log entry as a {@link ReferenceCountedObject} for retaining it for later use. */ + default ReferenceCountedObject wrap(LogEntryProto entry) { + Preconditions.assertSame(getLogEntry().getTerm(), entry.getTerm(), "entry.term"); + Preconditions.assertSame(getLogEntry().getIndex(), entry.getIndex(), "entry.index"); + return ReferenceCountedObject.wrap(entry); + } + /** * Sets whether to commit the transaction to the RAFT log or not * @param shouldCommit true if the transaction is supposed to be committed to the RAFT log diff --git a/ratis-server/src/main/java/org/apache/ratis/server/impl/LeaderStateImpl.java b/ratis-server/src/main/java/org/apache/ratis/server/impl/LeaderStateImpl.java index b716781086..5dcb3251c9 100644 --- a/ratis-server/src/main/java/org/apache/ratis/server/impl/LeaderStateImpl.java +++ b/ratis-server/src/main/java/org/apache/ratis/server/impl/LeaderStateImpl.java @@ -557,7 +557,7 @@ PendingRequests.Permit tryAcquirePendingRequest(Message message) { PendingRequest addPendingRequest(PendingRequests.Permit permit, RaftClientRequest request, TransactionContext entry) { if (LOG.isDebugEnabled()) { LOG.debug("{}: addPendingRequest at {}, entry={}", this, request, - LogProtoUtils.toLogEntryString(entry.getLogEntry())); + LogProtoUtils.toLogEntryString(entry.getLogEntryUnsafe())); } return pendingRequests.add(permit, request, entry); } diff --git a/ratis-server/src/main/java/org/apache/ratis/server/impl/PendingRequest.java b/ratis-server/src/main/java/org/apache/ratis/server/impl/PendingRequest.java index ed13b10113..d72fcde90b 100644 --- a/ratis-server/src/main/java/org/apache/ratis/server/impl/PendingRequest.java +++ b/ratis-server/src/main/java/org/apache/ratis/server/impl/PendingRequest.java @@ -38,7 +38,7 @@ class PendingRequest { private final CompletableFuture futureToReturn; PendingRequest(RaftClientRequest request, TransactionContext entry) { - this.termIndex = entry == null? null: TermIndex.valueOf(entry.getLogEntry()); + this.termIndex = entry == null? null: TermIndex.valueOf(entry.getLogEntryUnsafe()); this.request = request; this.entry = entry; if (request.is(TypeCase.FORWARD)) { diff --git a/ratis-server/src/main/java/org/apache/ratis/server/impl/RaftServerImpl.java b/ratis-server/src/main/java/org/apache/ratis/server/impl/RaftServerImpl.java index 966c70534d..fa88f5241c 100644 --- a/ratis-server/src/main/java/org/apache/ratis/server/impl/RaftServerImpl.java +++ b/ratis-server/src/main/java/org/apache/ratis/server/impl/RaftServerImpl.java @@ -1899,6 +1899,10 @@ CompletableFuture applyLogToStateMachine(LogEntryProto next) throws Raf final ClientInvocationId invocationId = ClientInvocationId.valueOf(next.getStateMachineLogEntry()); writeIndexCache.add(invocationId.getClientId(), ((TransactionContextImpl) trx).getLogIndexFuture()); + // TODO: RaftLog to provide the log entry as a ReferenceCountedObject as per RATIS-2028. + ReferenceCountedObject ref = ReferenceCountedObject.wrap(next); + ((TransactionContextImpl) trx).setDelegatedRef(ref); + ref.retain(); try { // Let the StateMachine inject logic for committed transactions in sequential order. trx = stateMachine.applyTransactionSerial(trx); @@ -1907,6 +1911,8 @@ CompletableFuture applyLogToStateMachine(LogEntryProto next) throws Raf messageFuture = replyPendingRequest(invocationId, TermIndex.valueOf(next), stateMachineFuture); } catch (Exception e) { throw new RaftLogIOException(e); + } finally { + ref.release(); } break; case METADATAENTRY: diff --git a/ratis-server/src/main/java/org/apache/ratis/server/impl/ServerState.java b/ratis-server/src/main/java/org/apache/ratis/server/impl/ServerState.java index bcf11baf7a..cef0af7304 100644 --- a/ratis-server/src/main/java/org/apache/ratis/server/impl/ServerState.java +++ b/ratis-server/src/main/java/org/apache/ratis/server/impl/ServerState.java @@ -327,7 +327,7 @@ TermIndex getLastEntry() { void appendLog(TransactionContext operation) throws StateMachineException { getLog().append(currentTerm.get(), operation); - Objects.requireNonNull(operation.getLogEntry()); + Objects.requireNonNull(operation.getLogEntryUnsafe(), "transaction-logEntry"); } /** @return true iff the given peer id is recognized as the leader. */ diff --git a/ratis-server/src/main/java/org/apache/ratis/server/raftlog/LogProtoUtils.java b/ratis-server/src/main/java/org/apache/ratis/server/raftlog/LogProtoUtils.java index 3705c3bd4b..610b02001a 100644 --- a/ratis-server/src/main/java/org/apache/ratis/server/raftlog/LogProtoUtils.java +++ b/ratis-server/src/main/java/org/apache/ratis/server/raftlog/LogProtoUtils.java @@ -26,6 +26,7 @@ import org.apache.ratis.server.protocol.TermIndex; import org.apache.ratis.thirdparty.com.google.protobuf.AbstractMessage; import org.apache.ratis.thirdparty.com.google.protobuf.ByteString; +import org.apache.ratis.thirdparty.com.google.protobuf.InvalidProtocolBufferException; import org.apache.ratis.util.Preconditions; import org.apache.ratis.util.ProtoUtils; @@ -240,4 +241,21 @@ public static RaftConfiguration toRaftConfiguration(LogEntryProto entry) { final List oldListener = ProtoUtils.toRaftPeers(proto.getOldListenersList()); return ServerImplUtils.newRaftConfiguration(conf, listener, entry.getIndex(), oldConf, oldListener); } + + public static LogEntryProto copy(LogEntryProto proto) { + if (proto == null) { + return null; + } + + if (!proto.hasStateMachineLogEntry() && !proto.hasMetadataEntry() && !proto.hasConfigurationEntry()) { + // empty entry, just return as is. + return proto; + } + + try { + return LogEntryProto.parseFrom(proto.toByteString()); + } catch (InvalidProtocolBufferException e) { + throw new IllegalArgumentException("Failed to copy log entry " + TermIndex.valueOf(proto), e); + } + } } diff --git a/ratis-server/src/main/java/org/apache/ratis/statemachine/impl/BaseStateMachine.java b/ratis-server/src/main/java/org/apache/ratis/statemachine/impl/BaseStateMachine.java index 3f18ee538b..b97749f262 100644 --- a/ratis-server/src/main/java/org/apache/ratis/statemachine/impl/BaseStateMachine.java +++ b/ratis-server/src/main/java/org/apache/ratis/statemachine/impl/BaseStateMachine.java @@ -18,7 +18,7 @@ package org.apache.ratis.statemachine.impl; -import org.apache.ratis.proto.RaftProtos; +import org.apache.ratis.proto.RaftProtos.LogEntryProto; import org.apache.ratis.protocol.Message; import org.apache.ratis.protocol.RaftClientRequest; import org.apache.ratis.protocol.RaftGroupId; @@ -115,10 +115,10 @@ public TransactionContext applyTransactionSerial(TransactionContext trx) throws @Override public CompletableFuture applyTransaction(TransactionContext trx) { // return the same message contained in the entry - RaftProtos.LogEntryProto entry = Objects.requireNonNull(trx.getLogEntry()); + final LogEntryProto entry = Objects.requireNonNull(trx.getLogEntryUnsafe()); updateLastAppliedTermIndex(entry.getTerm(), entry.getIndex()); return CompletableFuture.completedFuture( - Message.valueOf(trx.getLogEntry().getStateMachineLogEntry().getLogData())); + Message.valueOf(entry.getStateMachineLogEntry().getLogData())); } @Override diff --git a/ratis-server/src/main/java/org/apache/ratis/statemachine/impl/TransactionContextImpl.java b/ratis-server/src/main/java/org/apache/ratis/statemachine/impl/TransactionContextImpl.java index 8497b12f4d..ac8d534f26 100644 --- a/ratis-server/src/main/java/org/apache/ratis/statemachine/impl/TransactionContextImpl.java +++ b/ratis-server/src/main/java/org/apache/ratis/statemachine/impl/TransactionContextImpl.java @@ -25,11 +25,13 @@ import org.apache.ratis.statemachine.StateMachine; import org.apache.ratis.statemachine.TransactionContext; import org.apache.ratis.thirdparty.com.google.protobuf.ByteString; +import org.apache.ratis.util.MemoizedSupplier; import org.apache.ratis.util.Preconditions; import java.io.IOException; import java.util.Objects; import java.util.concurrent.CompletableFuture; +import java.util.function.Supplier; /** * Implementation of {@link TransactionContext} @@ -71,6 +73,11 @@ public class TransactionContextImpl implements TransactionContext { /** Committed LogEntry. */ @SuppressWarnings({"squid:S3077"}) // Suppress volatile for generic type private volatile LogEntryProto logEntry; + /** Committed LogEntry copy. */ + private volatile Supplier logEntryCopy; + + /** For wrapping {@link #logEntry} in order to release the underlying buffer. */ + private volatile ReferenceCountedObject delegatedRef; private final CompletableFuture logIndexFuture = new CompletableFuture<>(); @@ -112,7 +119,7 @@ private static StateMachineLogEntryProto get(StateMachineLogEntryProto stateMach */ TransactionContextImpl(RaftPeerRole serverRole, StateMachine stateMachine, LogEntryProto logEntry) { this(serverRole, null, stateMachine, logEntry.getStateMachineLogEntry()); - this.logEntry = logEntry; + setLogEntry(logEntry); this.logIndexFuture.complete(logEntry.getIndex()); } @@ -126,6 +133,22 @@ public RaftClientRequest getClientRequest() { return clientRequest; } + public void setDelegatedRef(ReferenceCountedObject ref) { + this.delegatedRef = ref; + } + + @Override + public ReferenceCountedObject wrap(LogEntryProto entry) { + if (delegatedRef == null) { + return TransactionContext.super.wrap(entry); + } + final LogEntryProto expected = getLogEntryUnsafe(); + Objects.requireNonNull(expected, "logEntry == null"); + Preconditions.assertSame(expected.getTerm(), entry.getTerm(), "entry.term"); + Preconditions.assertSame(expected.getIndex(), entry.getIndex(), "entry.index"); + return delegatedRef.delegate(entry); + } + @Override public StateMachineLogEntryProto getStateMachineLogEntry() { return stateMachineLogEntry; @@ -154,18 +177,31 @@ public LogEntryProto initLogEntry(long term, long index) { Objects.requireNonNull(stateMachineLogEntry, "stateMachineLogEntry == null"); logIndexFuture.complete(index); - return logEntry = LogProtoUtils.toLogEntryProto(stateMachineLogEntry, term, index); + return setLogEntry(LogProtoUtils.toLogEntryProto(stateMachineLogEntry, term, index)); } public CompletableFuture getLogIndexFuture() { return logIndexFuture; } + private LogEntryProto setLogEntry(LogEntryProto entry) { + this.logEntry = entry; + this.logEntryCopy = MemoizedSupplier.valueOf(() -> LogProtoUtils.copy(entry)); + return entry; + } + + @Override public LogEntryProto getLogEntry() { + return logEntryCopy == null ? null : logEntryCopy.get(); + } + + @Override + public LogEntryProto getLogEntryUnsafe() { return logEntry; } + @Override public TransactionContext setException(Exception ioe) { this.exception = ioe; @@ -193,4 +229,8 @@ public TransactionContext preAppendTransaction() throws IOException { public TransactionContext cancelTransaction() throws IOException { return stateMachine.cancelTransaction(this); } + + public static LogEntryProto getLogEntry(TransactionContext context) { + return ((TransactionContextImpl) context).logEntry; + } } diff --git a/ratis-server/src/test/java/org/apache/ratis/ReadOnlyRequestTests.java b/ratis-server/src/test/java/org/apache/ratis/ReadOnlyRequestTests.java index 94e9433b15..a2060da9ec 100644 --- a/ratis-server/src/test/java/org/apache/ratis/ReadOnlyRequestTests.java +++ b/ratis-server/src/test/java/org/apache/ratis/ReadOnlyRequestTests.java @@ -19,7 +19,7 @@ import org.apache.ratis.client.RaftClient; import org.apache.ratis.conf.RaftProperties; -import org.apache.ratis.proto.RaftProtos.LogEntryProto; +import org.apache.ratis.proto.RaftProtos; import org.apache.ratis.protocol.Message; import org.apache.ratis.protocol.RaftClientReply; import org.apache.ratis.protocol.RaftPeerId; @@ -214,12 +214,12 @@ private long timeoutIncrement() { @Override public CompletableFuture applyTransaction(TransactionContext trx) { - final LogEntryProto logEntry = trx.getLogEntry(); - final TermIndex ti = TermIndex.valueOf(logEntry); - updateLastAppliedTermIndex(ti); - LOG.info("{}: updateLastAppliedTermIndex {}", getId(), ti); + final RaftProtos.LogEntryProto logEntry = trx.getLogEntryUnsafe(); + LOG.debug("apply trx with index=" + logEntry.getIndex()); + updateLastAppliedTermIndex(logEntry.getTerm(), logEntry.getIndex()); - final String command = logEntry.getStateMachineLogEntry().getLogData().toString(StandardCharsets.UTF_8); + String command = logEntry.getStateMachineLogEntry() + .getLogData().toString(StandardCharsets.UTF_8); final long updatedCount; if (command.equals(INCREMENT_STRING)) { diff --git a/ratis-server/src/test/java/org/apache/ratis/server/impl/StateMachineShutdownTests.java b/ratis-server/src/test/java/org/apache/ratis/server/impl/StateMachineShutdownTests.java index fc00b70bab..9fa629420b 100644 --- a/ratis-server/src/test/java/org/apache/ratis/server/impl/StateMachineShutdownTests.java +++ b/ratis-server/src/test/java/org/apache/ratis/server/impl/StateMachineShutdownTests.java @@ -93,12 +93,10 @@ public CompletableFuture applyTransaction(TransactionContext trx) { } } } - numTxns.computeIfAbsent(getId(), (k) -> new AtomicLong()).incrementAndGet(); - appliedTxns.put(entry.getIndex(), entry.getTerm()); - updateTxns(); - future.complete(new RaftTestUtil.SimpleMessage("done")); - }); - return future; + } + final RaftProtos.LogEntryProto entry = trx.getLogEntryUnsafe(); + updateLastAppliedTermIndex(entry.getTerm(), entry.getIndex()); + return CompletableFuture.completedFuture(new RaftTestUtil.SimpleMessage("done")); } public void unBlockApplyTxn(long txnId) { diff --git a/ratis-server/src/test/java/org/apache/ratis/statemachine/impl/SimpleStateMachine4Testing.java b/ratis-server/src/test/java/org/apache/ratis/statemachine/impl/SimpleStateMachine4Testing.java index 9ef6e33048..57219ce21f 100644 --- a/ratis-server/src/test/java/org/apache/ratis/statemachine/impl/SimpleStateMachine4Testing.java +++ b/ratis-server/src/test/java/org/apache/ratis/statemachine/impl/SimpleStateMachine4Testing.java @@ -84,7 +84,7 @@ public static SimpleStateMachine4Testing get(RaftServer.Division s) { return (SimpleStateMachine4Testing)s.getStateMachine(); } - private final SortedMap indexMap = Collections.synchronizedSortedMap(new TreeMap<>()); + private final SortedMap> indexMap = Collections.synchronizedSortedMap(new TreeMap<>()); private final SortedMap dataMap = Collections.synchronizedSortedMap(new TreeMap<>()); private final Daemon checkpointer; private final SimpleStateMachineStorage storage = new SimpleStateMachineStorage(); @@ -199,8 +199,9 @@ public RoleInfoProto getLeaderElectionTimeoutInfo() { return leaderElectionTimeoutInfo; } - private void put(LogEntryProto entry) { - final LogEntryProto previous = indexMap.put(entry.getIndex(), entry); + private void put(ReferenceCountedObject entryRef) { + LogEntryProto entry = entryRef.retain(); + final ReferenceCountedObject previous = indexMap.put(entry.getIndex(), entryRef); Preconditions.assertNull(previous, "previous"); final String s = entry.getStateMachineLogEntry().getLogData().toStringUtf8(); dataMap.put(s, entry); @@ -249,27 +250,17 @@ public synchronized void reinitialize() throws IOException { @Override public CompletableFuture applyTransaction(TransactionContext trx) { blocking.await(Blocking.Type.APPLY_TRANSACTION); - LogEntryProto entry = Objects.requireNonNull(trx.getLogEntry()); + ReferenceCountedObject entryRef = Objects.requireNonNull(trx.getLogEntryRef()); + LogEntryProto entry = entryRef.get(); LOG.info("applyTransaction for log index {}", entry.getIndex()); - // TODO: Logs kept in StateMachine's cache may be corrupted. Copy for now to have the test pass. - // Use ReferenceCount per RATIS-1997. - LogEntryProto copied = copy(entry); - put(copied); + put(entryRef); updateLastAppliedTermIndex(entry.getTerm(), entry.getIndex()); final SimpleMessage m = new SimpleMessage(entry.getIndex() + " OK"); return collecting.collect(Collecting.Type.APPLY_TRANSACTION, m); } - private LogEntryProto copy(LogEntryProto log) { - try { - return LogEntryProto.parseFrom(log.toByteString()); - } catch (InvalidProtocolBufferException e) { - throw new IllegalStateException("Error copying log entry", e); - } - } - @Override public long takeSnapshot() { final TermIndex termIndex = getLastAppliedTermIndex(); @@ -283,7 +274,8 @@ public long takeSnapshot() { LOG.debug("Taking a snapshot with {}, file:{}", termIndex, snapshotFile); try (SegmentedRaftLogOutputStream out = new SegmentedRaftLogOutputStream(snapshotFile, false, segmentMaxSize, preallocatedSize, ByteBuffer.allocateDirect(bufferSize))) { - for (final LogEntryProto entry : indexMap.values()) { + for (final ReferenceCountedObject entryRef : indexMap.values()) { + LogEntryProto entry = entryRef.get(); if (entry.getIndex() > endIndex) { break; } else { @@ -318,7 +310,7 @@ private synchronized void loadSnapshot(SingleFileSnapshotInfo snapshot) throws I snapshot.getFile().getPath().toFile(), 0, endIndex, false)) { LogEntryProto entry; while ((entry = in.nextEntry()) != null) { - put(entry); + put(ReferenceCountedObject.wrap(entry)); updateLastAppliedTermIndex(entry.getTerm(), entry.getIndex()); } } @@ -393,10 +385,11 @@ public void close() { running = false; checkpointer.interrupt(); }); + indexMap.values().forEach(ReferenceCountedObject::release); } public LogEntryProto[] getContent() { - return indexMap.values().toArray(new LogEntryProto[0]); + return indexMap.values().stream().map(ReferenceCountedObject::get).toArray(LogEntryProto[]::new); } public void blockStartTransaction() { diff --git a/ratis-test/src/test/java/org/apache/ratis/datastream/DataStreamTestUtils.java b/ratis-test/src/test/java/org/apache/ratis/datastream/DataStreamTestUtils.java index f038dc5a45..6bcda74541 100644 --- a/ratis-test/src/test/java/org/apache/ratis/datastream/DataStreamTestUtils.java +++ b/ratis-test/src/test/java/org/apache/ratis/datastream/DataStreamTestUtils.java @@ -169,7 +169,7 @@ public CompletableFuture link(DataStream stream, LogEntryProto entry) { @Override public CompletableFuture applyTransaction(TransactionContext trx) { - final LogEntryProto entry = Objects.requireNonNull(trx.getLogEntry()); + final LogEntryProto entry = Objects.requireNonNull(trx.getLogEntryUnsafe()); updateLastAppliedTermIndex(entry.getTerm(), entry.getIndex()); final SingleDataStream s = getSingleDataStream(ClientInvocationId.valueOf(entry.getStateMachineLogEntry())); final ByteString bytesWritten = bytesWritten2ByteString(s.getDataChannel().getBytesWritten()); diff --git a/ratis-test/src/test/java/org/apache/ratis/statemachine/TestStateMachine.java b/ratis-test/src/test/java/org/apache/ratis/statemachine/TestStateMachine.java index 094189827d..07ea4edbcb 100644 --- a/ratis-test/src/test/java/org/apache/ratis/statemachine/TestStateMachine.java +++ b/ratis-test/src/test/java/org/apache/ratis/statemachine/TestStateMachine.java @@ -91,7 +91,7 @@ public TransactionContext startTransaction(RaftClientRequest request) { @Override public CompletableFuture applyTransaction(TransactionContext trx) { try { - assertNotNull(trx.getLogEntry()); + assertNotNull(trx.getLogEntryUnsafe()); assertNotNull(trx.getStateMachineLogEntry()); Object context = trx.getStateMachineContext(); if (isLeader.get()) { From 3c397869a2104e0808243599cf41edd010a87a09 Mon Sep 17 00:00:00 2001 From: Duong Nguyen Date: Mon, 11 Mar 2024 17:58:00 -0700 Subject: [PATCH 341/397] RATIS-2028. Refactor RaftLog to supply log as ReferenceCountedObject (#1045) (cherry picked from commit e199daaf2a288508bd3e119e9c252c70ee90937d) --- .../apache/ratis/server/raftlog/RaftLog.java | 17 +++++- .../ratis/server/impl/RaftServerImpl.java | 16 +++--- .../server/impl/StateMachineUpdater.java | 19 ++++--- .../ratis/server/raftlog/RaftLogBase.java | 18 +++++- .../server/raftlog/memory/MemoryRaftLog.java | 35 ++++++++++-- .../server/raftlog/segmented/LogSegment.java | 55 +++++++++++++++---- .../raftlog/segmented/SegmentedRaftLog.java | 37 +++++++++++-- .../statemachine/RaftSnapshotBaseTest.java | 8 +-- .../ratis/datastream/DataStreamTestUtils.java | 11 +++- .../ratis/server/ServerRestartTests.java | 10 ++-- .../raftlog/segmented/TestLogSegment.java | 4 +- .../segmented/TestSegmentedRaftLog.java | 7 ++- .../segmented/TestSegmentedRaftLogCache.java | 4 +- 13 files changed, 184 insertions(+), 57 deletions(-) diff --git a/ratis-server-api/src/main/java/org/apache/ratis/server/raftlog/RaftLog.java b/ratis-server-api/src/main/java/org/apache/ratis/server/raftlog/RaftLog.java index e194f865ed..37c10e05ef 100644 --- a/ratis-server-api/src/main/java/org/apache/ratis/server/raftlog/RaftLog.java +++ b/ratis-server-api/src/main/java/org/apache/ratis/server/raftlog/RaftLog.java @@ -21,6 +21,7 @@ import org.apache.ratis.server.metrics.RaftLogMetrics; import org.apache.ratis.server.protocol.TermIndex; import org.apache.ratis.server.storage.RaftStorageMetadata; +import org.apache.ratis.util.ReferenceCountedObject; import org.apache.ratis.util.TimeDuration; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -60,10 +61,24 @@ default boolean contains(TermIndex ti) { /** * @return null if the log entry is not found in this log; - * otherwise, return the log entry corresponding to the given index. + * otherwise, return a copy of the log entry corresponding to the given index. + * @deprecated use {@link RaftLog#retainLog(long)} instead in order to avoid copying. */ + @Deprecated LogEntryProto get(long index) throws RaftLogIOException; + /** + * @return a retained {@link ReferenceCountedObject} to the log entry corresponding to the given index if it exists; + * otherwise, return null. + * Since the returned reference is retained, the caller must call {@link ReferenceCountedObject#release()}} + * after use. + */ + default ReferenceCountedObject retainLog(long index) throws RaftLogIOException { + ReferenceCountedObject wrap = ReferenceCountedObject.wrap(get(index)); + wrap.retain(); + return wrap; + } + /** * @return null if the log entry is not found in this log; * otherwise, return the {@link EntryWithData} corresponding to the given index. diff --git a/ratis-server/src/main/java/org/apache/ratis/server/impl/RaftServerImpl.java b/ratis-server/src/main/java/org/apache/ratis/server/impl/RaftServerImpl.java index fa88f5241c..a10066a39f 100644 --- a/ratis-server/src/main/java/org/apache/ratis/server/impl/RaftServerImpl.java +++ b/ratis-server/src/main/java/org/apache/ratis/server/impl/RaftServerImpl.java @@ -1881,8 +1881,12 @@ TransactionContext getTransactionContext(LogEntryProto entry, Boolean createNew) MemoizedSupplier.valueOf(() -> stateMachine.startTransaction(entry, getInfo().getCurrentRole()))); } - CompletableFuture applyLogToStateMachine(LogEntryProto next) throws RaftLogIOException { - CompletableFuture messageFuture = null; + CompletableFuture applyLogToStateMachine(ReferenceCountedObject nextRef) + throws RaftLogIOException { + LogEntryProto next = nextRef.get(); + if (!next.hasStateMachineLogEntry()) { + stateMachine.event().notifyTermIndexUpdated(next.getTerm(), next.getIndex()); + } switch (next.getLogEntryBodyCase()) { case CONFIGURATIONENTRY: @@ -1898,11 +1902,7 @@ CompletableFuture applyLogToStateMachine(LogEntryProto next) throws Raf Objects.requireNonNull(trx, "trx == null"); final ClientInvocationId invocationId = ClientInvocationId.valueOf(next.getStateMachineLogEntry()); writeIndexCache.add(invocationId.getClientId(), ((TransactionContextImpl) trx).getLogIndexFuture()); - - // TODO: RaftLog to provide the log entry as a ReferenceCountedObject as per RATIS-2028. - ReferenceCountedObject ref = ReferenceCountedObject.wrap(next); - ((TransactionContextImpl) trx).setDelegatedRef(ref); - ref.retain(); + ((TransactionContextImpl) trx).setDelegatedRef(nextRef); try { // Let the StateMachine inject logic for committed transactions in sequential order. trx = stateMachine.applyTransactionSerial(trx); @@ -1911,8 +1911,6 @@ CompletableFuture applyLogToStateMachine(LogEntryProto next) throws Raf messageFuture = replyPendingRequest(invocationId, TermIndex.valueOf(next), stateMachineFuture); } catch (Exception e) { throw new RaftLogIOException(e); - } finally { - ref.release(); } break; case METADATAENTRY: diff --git a/ratis-server/src/main/java/org/apache/ratis/server/impl/StateMachineUpdater.java b/ratis-server/src/main/java/org/apache/ratis/server/impl/StateMachineUpdater.java index 041693195f..125776555f 100644 --- a/ratis-server/src/main/java/org/apache/ratis/server/impl/StateMachineUpdater.java +++ b/ratis-server/src/main/java/org/apache/ratis/server/impl/StateMachineUpdater.java @@ -244,10 +244,17 @@ private CompletableFuture applyLog(CompletableFuture applyLogFutures final long committed = raftLog.getLastCommittedIndex(); for(long applied; (applied = getLastAppliedIndex()) < committed && state == State.RUNNING && !shouldStop(); ) { final long nextIndex = applied + 1; - final LogEntryProto next = raftLog.get(nextIndex); - if (next != null) { + final ReferenceCountedObject next = raftLog.retainLog(nextIndex); + if (next == null) { + LOG.debug("{}: logEntry {} is null. There may be snapshot to load. state:{}", + this, nextIndex, state); + break; + } + + try { + final LogEntryProto entry = next.get(); if (LOG.isTraceEnabled()) { - LOG.trace("{}: applying nextIndex={}, nextLog={}", this, nextIndex, LogProtoUtils.toLogEntryString(next)); + LOG.trace("{}: applying nextIndex={}, nextLog={}", this, nextIndex, LogProtoUtils.toLogEntryString(entry)); } else { LOG.debug("{}: applying nextIndex={}", this, nextIndex); } @@ -266,10 +273,8 @@ private CompletableFuture applyLog(CompletableFuture applyLogFutures } else { notifyAppliedIndex(incremented); } - } else { - LOG.debug("{}: logEntry {} is null. There may be snapshot to load. state:{}", - this, nextIndex, state); - break; + } finally { + next.release(); } } return applyLogFutures; diff --git a/ratis-server/src/main/java/org/apache/ratis/server/raftlog/RaftLogBase.java b/ratis-server/src/main/java/org/apache/ratis/server/raftlog/RaftLogBase.java index c6c33e6714..c4751770e4 100644 --- a/ratis-server/src/main/java/org/apache/ratis/server/raftlog/RaftLogBase.java +++ b/ratis-server/src/main/java/org/apache/ratis/server/raftlog/RaftLogBase.java @@ -243,9 +243,21 @@ private boolean shouldAppendMetadata(long newCommitIndex) { // do not log the first conf entry return false; } - final LogEntryProto last = lastMetadataEntry.get(); - // do not log entries with a smaller commit index. - return last == null || newCommitIndex > last.getMetadataEntry().getCommitIndex(); + ReferenceCountedObject ref = null; + try { + ref = retainLog(newCommitIndex); + if (ref.get().hasMetadataEntry()) { + // do not log the metadata entry + return false; + } + } catch(RaftLogIOException e) { + LOG.error("Failed to get log entry for index " + newCommitIndex, e); + } finally { + if (ref != null) { + ref.release(); + } + } + return true; } @Override diff --git a/ratis-server/src/main/java/org/apache/ratis/server/raftlog/memory/MemoryRaftLog.java b/ratis-server/src/main/java/org/apache/ratis/server/raftlog/memory/MemoryRaftLog.java index fc7973aab6..feedaeee4a 100644 --- a/ratis-server/src/main/java/org/apache/ratis/server/raftlog/memory/MemoryRaftLog.java +++ b/ratis-server/src/main/java/org/apache/ratis/server/raftlog/memory/MemoryRaftLog.java @@ -22,8 +22,10 @@ import org.apache.ratis.server.metrics.RaftLogMetricsBase; import org.apache.ratis.server.protocol.TermIndex; import org.apache.ratis.proto.RaftProtos.LogEntryProto; +import org.apache.ratis.server.raftlog.LogProtoUtils; import org.apache.ratis.server.raftlog.RaftLogBase; import org.apache.ratis.server.raftlog.LogEntryHeader; +import org.apache.ratis.server.raftlog.RaftLogIOException; import org.apache.ratis.server.storage.RaftStorageMetadata; import org.apache.ratis.statemachine.TransactionContext; import org.apache.ratis.util.AutoCloseableLock; @@ -45,8 +47,13 @@ public class MemoryRaftLog extends RaftLogBase { static class EntryList { private final List> entries = new ArrayList<>(); + ReferenceCountedObject getRef(int i) { + return i >= 0 && i < entries.size() ? entries.get(i) : null; + } + LogEntryProto get(int i) { - return i >= 0 && i < entries.size() ? entries.get(i).get() : null; + final ReferenceCountedObject ref = getRef(i); + return ref != null ? ref.get() : null; } TermIndex getTermIndex(int i) { @@ -108,16 +115,34 @@ public RaftLogMetricsBase getRaftLogMetrics() { } @Override - public LogEntryProto get(long index) { + public LogEntryProto get(long index) throws RaftLogIOException { + final ReferenceCountedObject ref = retainLog(index); + try { + return LogProtoUtils.copy(ref.get()); + } finally { + ref.release(); + } + } + + @Override + public ReferenceCountedObject retainLog(long index) { checkLogState(); - try(AutoCloseableLock readLock = readLock()) { - return entries.get(Math.toIntExact(index)); + try (AutoCloseableLock readLock = readLock()) { + ReferenceCountedObject ref = entries.getRef(Math.toIntExact(index)); + ref.retain(); + return ref; } } @Override public EntryWithData getEntryWithData(long index) { - return newEntryWithData(get(index), null); + // TODO. The reference counted object should be passed to LogAppender RATIS-2026. + ReferenceCountedObject ref = retainLog(index); + try { + return newEntryWithData(ref.get(), null); + } finally { + ref.release(); + } } @Override diff --git a/ratis-server/src/main/java/org/apache/ratis/server/raftlog/segmented/LogSegment.java b/ratis-server/src/main/java/org/apache/ratis/server/raftlog/segmented/LogSegment.java index 860bd196d8..601e3aeca6 100644 --- a/ratis-server/src/main/java/org/apache/ratis/server/raftlog/segmented/LogSegment.java +++ b/ratis-server/src/main/java/org/apache/ratis/server/raftlog/segmented/LogSegment.java @@ -262,7 +262,7 @@ private void assertSegment(long expectedStart, int expectedEntryCount, boolean c * * In the future we can make the cache loader configurable if necessary. */ - class LogEntryLoader extends CacheLoader { + class LogEntryLoader extends CacheLoader> { private final SegmentedRaftLogMetrics raftLogMetrics; LogEntryLoader(SegmentedRaftLogMetrics raftLogMetrics) { @@ -270,27 +270,58 @@ class LogEntryLoader extends CacheLoader { } @Override - public LogEntryProto load(LogRecord key) throws IOException { + public ReferenceCountedObject load(LogRecord key) throws IOException { final File file = getFile(); // note the loading should not exceed the endIndex: it is possible that // the on-disk log file should be truncated but has not been done yet. - final AtomicReference toReturn = new AtomicReference<>(); + final AtomicReference> toReturn = new AtomicReference<>(); final LogSegmentStartEnd startEnd = LogSegmentStartEnd.valueOf(startIndex, endIndex, isOpen); readSegmentFile(file, startEnd, maxOpSize, getLogCorruptionPolicy(), raftLogMetrics, entry -> { final TermIndex ti = TermIndex.valueOf(entry); putEntryCache(ti, entry, Op.LOAD_SEGMENT_FILE); if (ti.equals(key.getTermIndex())) { - toReturn.set(entry); + entryRef.retain(); + toReturn.set(entryRef); } entryRef.release(); }); loadingTimes.incrementAndGet(); - final LogEntryProto proto = toReturn.get(); - if (proto == null) { - throw new RaftLogIOException("Failed to load log entry " + key); - } - return proto; + return Objects.requireNonNull(toReturn.get()); + } + } + + static class EntryCache { + private final Map> map = new ConcurrentHashMap<>(); + private final AtomicLong size = new AtomicLong(); + + long size() { + return size.get(); + } + + ReferenceCountedObject get(TermIndex ti) { + return map.get(ti); + } + + void clear() { + map.values().forEach(ReferenceCountedObject::release); + map.clear(); + size.set(0); + } + + void put(TermIndex key, ReferenceCountedObject valueRef, Op op) { + valueRef.retain(); + Optional.ofNullable(map.put(key, valueRef)).ifPresent(this::release); + size.getAndAdd(getEntrySize(valueRef.get(), op)); + } + + private void release(ReferenceCountedObject entry) { + size.getAndAdd(-getEntrySize(entry.get(), Op.REMOVE_CACHE)); + entry.release(); + } + + void remove(TermIndex key) { + Optional.ofNullable(map.remove(key)).ifPresent(this::release); } } @@ -391,15 +422,15 @@ void verifyEntryIndex(long entryIndex) { } } - LogEntryProto getEntryFromCache(TermIndex ti) { + ReferenceCountedObject getEntryFromCache(TermIndex ti) { return entryCache.get(ti); } /** * Acquire LogSegment's monitor so that there is no concurrent loading. */ - synchronized LogEntryProto loadCache(LogRecord record) throws RaftLogIOException { - LogEntryProto entry = entryCache.get(record.getTermIndex()); + synchronized ReferenceCountedObject loadCache(LogRecord record) throws RaftLogIOException { + ReferenceCountedObject entry = entryCache.get(record.getTermIndex()); if (entry != null) { return entry; } diff --git a/ratis-server/src/main/java/org/apache/ratis/server/raftlog/segmented/SegmentedRaftLog.java b/ratis-server/src/main/java/org/apache/ratis/server/raftlog/segmented/SegmentedRaftLog.java index 8879024771..786257c0eb 100644 --- a/ratis-server/src/main/java/org/apache/ratis/server/raftlog/segmented/SegmentedRaftLog.java +++ b/ratis-server/src/main/java/org/apache/ratis/server/raftlog/segmented/SegmentedRaftLog.java @@ -179,11 +179,17 @@ public long getLastAppliedIndex() { @Override public void notifyTruncatedLogEntry(TermIndex ti) { + ReferenceCountedObject ref = null; try { - final LogEntryProto entry = get(ti.getIndex()); + ref = retainLog(ti.getIndex()); + final LogEntryProto entry = ref != null ? ref.get() : null; notifyTruncatedLogEntry.accept(entry); } catch (RaftLogIOException e) { LOG.error("{}: Failed to read log {}", getName(), ti, e); + } finally { + if (ref != null) { + ref.release(); + } } } @@ -279,6 +285,19 @@ private void loadLogSegments(long lastIndexInSnapshot, @Override public LogEntryProto get(long index) throws RaftLogIOException { + final ReferenceCountedObject ref = retainLog(index); + if (ref == null) { + return null; + } + try { + return LogProtoUtils.copy(ref.get()); + } finally { + ref.release(); + } + } + + @Override + public ReferenceCountedObject retainLog(long index) throws RaftLogIOException { checkLogState(); final LogSegment segment; final LogRecord record; @@ -291,9 +310,10 @@ record = segment.getLogRecord(index); if (record == null) { return null; } - final LogEntryProto entry = segment.getEntryFromCache(record.getTermIndex()); + final ReferenceCountedObject entry = segment.getEntryFromCache(record.getTermIndex()); if (entry != null) { getRaftLogMetrics().onRaftLogCacheHit(); + entry.retain(); return entry; } } @@ -306,10 +326,19 @@ record = segment.getLogRecord(index); @Override public EntryWithData getEntryWithData(long index) throws RaftLogIOException { - final LogEntryProto entry = get(index); - if (entry == null) { + final ReferenceCountedObject entryRef = retainLog(index); + if (entryRef == null) { throw new RaftLogIOException("Log entry not found: index = " + index); } + try { + // TODO. The reference counted object should be passed to LogAppender RATIS-2026. + return getEntryWithData(entryRef.get()); + } finally { + entryRef.release(); + } + } + + private EntryWithData getEntryWithData(LogEntryProto entry) throws RaftLogIOException { if (!LogProtoUtils.isStateMachineDataEmpty(entry)) { return newEntryWithData(entry, null); } diff --git a/ratis-server/src/test/java/org/apache/ratis/statemachine/RaftSnapshotBaseTest.java b/ratis-server/src/test/java/org/apache/ratis/statemachine/RaftSnapshotBaseTest.java index 09b5ee59a6..03c500f5db 100644 --- a/ratis-server/src/test/java/org/apache/ratis/statemachine/RaftSnapshotBaseTest.java +++ b/ratis-server/src/test/java/org/apache/ratis/statemachine/RaftSnapshotBaseTest.java @@ -21,6 +21,7 @@ import static org.apache.ratis.server.impl.StateMachineMetrics.RATIS_STATEMACHINE_METRICS_DESC; import static org.apache.ratis.server.impl.StateMachineMetrics.STATEMACHINE_TAKE_SNAPSHOT_TIMER; import static org.apache.ratis.metrics.RatisMetrics.RATIS_APPLICATION_NAME_METRICS; +import static org.apache.ratis.server.storage.RaftStorageTestUtils.getLogUnsafe; import org.apache.ratis.BaseTest; import org.apache.ratis.metrics.LongCounter; @@ -44,6 +45,7 @@ import org.apache.ratis.server.raftlog.RaftLog; import org.apache.ratis.server.raftlog.segmented.LogSegmentPath; import org.apache.ratis.proto.RaftProtos.LogEntryProto; +import org.apache.ratis.server.storage.RaftStorageTestUtils; import org.apache.ratis.statemachine.impl.SimpleStateMachine4Testing; import org.apache.ratis.statemachine.impl.SimpleStateMachineStorage; import org.apache.ratis.util.FileUtils; @@ -99,10 +101,8 @@ public static void assertLeaderContent(MiniRaftCluster cluster) throws Exception public static void checkMetadataEntry(RaftServer.Division server) throws Exception { final RaftLog log = server.getRaftLog(); final long lastIndex = log.getLastEntryTermIndex().getIndex(); - final LogEntryProto e = log.get(lastIndex); - Assertions.assertTrue(e.hasMetadataEntry()); - Assertions.assertEquals(log.getLastCommittedIndex() - 1, e.getMetadataEntry().getCommitIndex()); - } + final LogEntryProto e = getLogUnsafe(log, lastIndex); + Assert.assertTrue(e.hasMetadataEntry()); public static void assertLogContent(RaftServer.Division server, boolean isLeader) throws Exception { JavaUtils.attempt(() -> checkMetadataEntry(server), 50, HUNDRED_MILLIS, "checkMetadataEntry", LOG); diff --git a/ratis-test/src/test/java/org/apache/ratis/datastream/DataStreamTestUtils.java b/ratis-test/src/test/java/org/apache/ratis/datastream/DataStreamTestUtils.java index 6bcda74541..989b6cd2b2 100644 --- a/ratis-test/src/test/java/org/apache/ratis/datastream/DataStreamTestUtils.java +++ b/ratis-test/src/test/java/org/apache/ratis/datastream/DataStreamTestUtils.java @@ -169,7 +169,7 @@ public CompletableFuture link(DataStream stream, LogEntryProto entry) { @Override public CompletableFuture applyTransaction(TransactionContext trx) { - final LogEntryProto entry = Objects.requireNonNull(trx.getLogEntryUnsafe()); + final LogEntryProto entry = Objects.requireNonNull(trx.getLogEntry()); updateLastAppliedTermIndex(entry.getTerm(), entry.getIndex()); final SingleDataStream s = getSingleDataStream(ClientInvocationId.valueOf(entry.getStateMachineLogEntry())); final ByteString bytesWritten = bytesWritten2ByteString(s.getDataChannel().getBytesWritten()); @@ -419,4 +419,13 @@ static void assertLogEntry(RaftServer.Division division, SingleDataStream stream final LogEntryProto entryFromLog = searchLogEntry(ClientInvocationId.valueOf(request), division.getRaftLog()); Assertions.assertEquals(entryFromStream, entryFromLog); } + + ResourceLeakDetector.LeakListener LEAK_LISTENER = (resourceType, records) -> { + throw new IllegalStateException("Leak detected for resource type: " + resourceType + records); + }; + + static void enableResourceLeakDetector() { + ResourceLeakDetector.setLevel(Level.PARANOID); + ByteBufUtil.setLeakListener(DataStreamTestUtils.LEAK_LISTENER); + } } diff --git a/ratis-test/src/test/java/org/apache/ratis/server/ServerRestartTests.java b/ratis-test/src/test/java/org/apache/ratis/server/ServerRestartTests.java index 592196034e..f7a8bbaeca 100644 --- a/ratis-test/src/test/java/org/apache/ratis/server/ServerRestartTests.java +++ b/ratis-test/src/test/java/org/apache/ratis/server/ServerRestartTests.java @@ -64,6 +64,8 @@ import java.util.concurrent.atomic.AtomicInteger; import java.util.function.Supplier; +import static org.apache.ratis.server.storage.RaftStorageTestUtils.getLogUnsafe; + /** * Test restarting raft peers. */ @@ -261,10 +263,10 @@ void runTestRestartCommitIndex(MiniRaftCluster cluster) throws Exception { final long lastIndex = leaderLog.getLastEntryTermIndex().getIndex(); LOG.info("{}: leader lastIndex={}", leaderId, lastIndex); - final LogEntryProto lastEntry = leaderLog.get(lastIndex); + final LogEntryProto lastEntry = getLogUnsafe(leaderLog, lastIndex); LOG.info("{}: leader lastEntry entry[{}] = {}", leaderId, lastIndex, LogProtoUtils.toLogEntryString(lastEntry)); final long loggedCommitIndex = lastEntry.getMetadataEntry().getCommitIndex(); - final LogEntryProto lastCommittedEntry = leaderLog.get(loggedCommitIndex); + final LogEntryProto lastCommittedEntry = getLogUnsafe(leaderLog, loggedCommitIndex); LOG.info("{}: leader lastCommittedEntry = entry[{}] = {}", leaderId, loggedCommitIndex, LogProtoUtils.toLogEntryString(lastCommittedEntry)); @@ -310,11 +312,11 @@ void runTestRestartCommitIndex(MiniRaftCluster cluster) throws Exception { static void assertLastLogEntry(RaftServer.Division server) throws RaftLogIOException { final RaftLog raftLog = server.getRaftLog(); final long lastIndex = raftLog.getLastEntryTermIndex().getIndex(); - final LogEntryProto lastEntry = raftLog.get(lastIndex); + final LogEntryProto lastEntry = getLogUnsafe(raftLog, lastIndex); Assertions.assertTrue(lastEntry.hasMetadataEntry()); final long loggedCommitIndex = lastEntry.getMetadataEntry().getCommitIndex(); - final LogEntryProto lastCommittedEntry = raftLog.get(loggedCommitIndex); + final LogEntryProto lastCommittedEntry = getLogUnsafe(raftLog, loggedCommitIndex); Assertions.assertTrue(lastCommittedEntry.hasStateMachineLogEntry()); final SimpleStateMachine4Testing leaderStateMachine = SimpleStateMachine4Testing.get(server); diff --git a/ratis-test/src/test/java/org/apache/ratis/server/raftlog/segmented/TestLogSegment.java b/ratis-test/src/test/java/org/apache/ratis/server/raftlog/segmented/TestLogSegment.java index ec1f925eb8..5d87fde3c4 100644 --- a/ratis-test/src/test/java/org/apache/ratis/server/raftlog/segmented/TestLogSegment.java +++ b/ratis-test/src/test/java/org/apache/ratis/server/raftlog/segmented/TestLogSegment.java @@ -142,11 +142,11 @@ static void checkLogSegment(LogSegment segment, long start, long end, Assertions.assertEquals(term, ti.getTerm()); Assertions.assertEquals(offset, record.getOffset()); - LogEntryProto entry = segment.getEntryFromCache(ti); + ReferenceCountedObject entry = segment.getEntryFromCache(ti); if (entry == null) { entry = segment.loadCache(record); } - offset += getEntrySize(entry, LogSegment.Op.WRITE_CACHE_WITHOUT_STATE_MACHINE_CACHE); + offset += getEntrySize(entry.get(), Op.WRITE_CACHE_WITHOUT_STATE_MACHINE_CACHE); } } diff --git a/ratis-test/src/test/java/org/apache/ratis/server/raftlog/segmented/TestSegmentedRaftLog.java b/ratis-test/src/test/java/org/apache/ratis/server/raftlog/segmented/TestSegmentedRaftLog.java index 5bcfd12de7..4e53fccd69 100644 --- a/ratis-test/src/test/java/org/apache/ratis/server/raftlog/segmented/TestSegmentedRaftLog.java +++ b/ratis-test/src/test/java/org/apache/ratis/server/raftlog/segmented/TestSegmentedRaftLog.java @@ -81,6 +81,7 @@ import static java.lang.Boolean.FALSE; import static java.lang.Boolean.TRUE; +import static org.apache.ratis.server.storage.RaftStorageTestUtils.getLogUnsafe; import static org.junit.jupiter.api.Assertions.assertTrue; import static org.junit.jupiter.params.provider.Arguments.arguments; @@ -211,7 +212,7 @@ static List prepareRanges(int startTerm, int endTerm, int segmentS private LogEntryProto getLastEntry(SegmentedRaftLog raftLog) throws IOException { - return raftLog.get(raftLog.getLastEntryTermIndex().getIndex()); + return getLogUnsafe(raftLog, raftLog.getLastEntryTermIndex().getIndex()); } @ParameterizedTest @@ -236,7 +237,7 @@ public void testLoadLogSegments(Boolean useAsyncFlush, Boolean smSyncFlush) thro LogEntryProto[] entriesFromLog = Arrays.stream(termIndices) .map(ti -> { try { - return raftLog.get(ti.getIndex()); + return getLogUnsafe(raftLog, ti.getIndex()); } catch (IOException e) { throw new RuntimeException(e); } @@ -459,7 +460,7 @@ private void checkEntries(RaftLog raftLog, List expected, LogEntryProto[] entriesFromLog = Arrays.stream(termIndices) .map(ti -> { try { - return raftLog.get(ti.getIndex()); + return getLogUnsafe(raftLog, ti.getIndex()); } catch (IOException e) { throw new RuntimeException(e); } diff --git a/ratis-test/src/test/java/org/apache/ratis/server/raftlog/segmented/TestSegmentedRaftLogCache.java b/ratis-test/src/test/java/org/apache/ratis/server/raftlog/segmented/TestSegmentedRaftLogCache.java index d8924d71b4..da0c6c8a3f 100644 --- a/ratis-test/src/test/java/org/apache/ratis/server/raftlog/segmented/TestSegmentedRaftLogCache.java +++ b/ratis-test/src/test/java/org/apache/ratis/server/raftlog/segmented/TestSegmentedRaftLogCache.java @@ -79,8 +79,8 @@ private void checkCache(long start, long end, int segmentSize) { for (long index = start; index <= end; index++) { final LogSegment segment = cache.getSegment(index); final LogRecord record = segment.getLogRecord(index); - final LogEntryProto entry = segment.getEntryFromCache(record.getTermIndex()); - Assertions.assertEquals(index, entry.getIndex()); + final ReferenceCountedObject entry = segment.getEntryFromCache(record.getTermIndex()); + Assertions.assertEquals(index, entry.get().getIndex()); } long[] offsets = new long[]{start, start + 1, start + (end - start) / 2, From 40f3c63b18b9fed5720f2ed7f909d36334e98963 Mon Sep 17 00:00:00 2001 From: Duong Nguyen Date: Tue, 26 Mar 2024 20:18:16 -0700 Subject: [PATCH 342/397] RATIS-2026. LogAppender to consume log entries with reference count (#1049) (cherry picked from commit 397c2efdd79ff41ebf57a38bcc3681f2126091ec) --- .../ratis/grpc/server/GrpcLogAppender.java | 29 ++++++++---- .../ratis/server/leader/LogAppender.java | 2 + .../apache/ratis/server/raftlog/RaftLog.java | 23 ++++++++++ .../ratis/server/leader/LogAppenderBase.java | 46 ++++++++++++++++--- .../server/leader/LogAppenderDefault.java | 38 +++++++++------ .../ratis/server/raftlog/RaftLogBase.java | 13 ++++-- .../server/raftlog/memory/MemoryRaftLog.java | 16 +++---- .../raftlog/segmented/SegmentedRaftLog.java | 18 ++++---- 8 files changed, 131 insertions(+), 54 deletions(-) diff --git a/ratis-grpc/src/main/java/org/apache/ratis/grpc/server/GrpcLogAppender.java b/ratis-grpc/src/main/java/org/apache/ratis/grpc/server/GrpcLogAppender.java index 053cc5c0f4..e75da86236 100644 --- a/ratis-grpc/src/main/java/org/apache/ratis/grpc/server/GrpcLogAppender.java +++ b/ratis-grpc/src/main/java/org/apache/ratis/grpc/server/GrpcLogAppender.java @@ -392,30 +392,39 @@ public Comparator getCallIdComparator() { } private void appendLog(boolean heartbeat) throws IOException { - final AppendEntriesRequestProto pending; + ReferenceCountedObject pending = null; final AppendEntriesRequest request; try (AutoCloseableLock writeLock = lock.writeLock(caller, LOG::trace)) { // Prepare and send the append request. // Note changes on follower's nextIndex and ops on pendingRequests should always be done under the write-lock - pending = newAppendEntriesRequest(callId.getAndIncrement(), heartbeat); + pending = nextAppendEntriesRequest(callId.getAndIncrement(), heartbeat); if (pending == null) { return; } - request = new AppendEntriesRequest(pending, getFollowerId(), grpcServerMetrics); + request = new AppendEntriesRequest(pending.get(), getFollowerId(), grpcServerMetrics); pendingRequests.put(request); - increaseNextIndex(pending); + increaseNextIndex(pending.get()); if (appendLogRequestObserver == null) { appendLogRequestObserver = new StreamObservers( getClient(), new AppendLogResponseHandler(), useSeparateHBChannel, getWaitTimeMin()); } + } catch(Exception e) { + if (pending != null) { + pending.release(); + } + throw e; } - final TimeDuration remaining = getRemainingWaitTime(); - if (remaining.isPositive()) { - sleep(remaining, heartbeat); - } - if (isRunning()) { - sendRequest(request, pending); + try { + final TimeDuration remaining = getRemainingWaitTime(); + if (remaining.isPositive()) { + sleep(remaining, heartbeat); + } + if (isRunning()) { + sendRequest(request, pending.get()); + } + } finally { + pending.release(); } } diff --git a/ratis-server-api/src/main/java/org/apache/ratis/server/leader/LogAppender.java b/ratis-server-api/src/main/java/org/apache/ratis/server/leader/LogAppender.java index 33914fde7f..dc189a14aa 100644 --- a/ratis-server-api/src/main/java/org/apache/ratis/server/leader/LogAppender.java +++ b/ratis-server-api/src/main/java/org/apache/ratis/server/leader/LogAppender.java @@ -125,7 +125,9 @@ default RaftPeerId getFollowerId() { * @param heartbeat the returned request must be a heartbeat. * * @return a new {@link AppendEntriesRequestProto} object. + * @deprecated this is no longer a public API. */ + @Deprecated AppendEntriesRequestProto newAppendEntriesRequest(long callId, boolean heartbeat) throws RaftLogIOException; /** @return a new {@link InstallSnapshotRequestProto} object. */ diff --git a/ratis-server-api/src/main/java/org/apache/ratis/server/raftlog/RaftLog.java b/ratis-server-api/src/main/java/org/apache/ratis/server/raftlog/RaftLog.java index 37c10e05ef..07446282e7 100644 --- a/ratis-server-api/src/main/java/org/apache/ratis/server/raftlog/RaftLog.java +++ b/ratis-server-api/src/main/java/org/apache/ratis/server/raftlog/RaftLog.java @@ -82,9 +82,23 @@ default ReferenceCountedObject retainLog(long index) throws RaftL /** * @return null if the log entry is not found in this log; * otherwise, return the {@link EntryWithData} corresponding to the given index. + * @deprecated use {@link #retainEntryWithData(long)}. */ + @Deprecated EntryWithData getEntryWithData(long index) throws RaftLogIOException; + /** + * @return null if the log entry is not found in this log; + * otherwise, return a retained reference of the {@link EntryWithData} corresponding to the given index. + * Since the returned reference is retained, the caller must call {@link ReferenceCountedObject#release()}} + * after use. + */ + default ReferenceCountedObject retainEntryWithData(long index) throws RaftLogIOException { + final ReferenceCountedObject wrap = ReferenceCountedObject.wrap(getEntryWithData(index)); + wrap.retain(); + return wrap; +} + /** * @param startIndex the starting log index (inclusive) * @param endIndex the ending log index (exclusive) @@ -175,6 +189,15 @@ default long getNextIndex() { * containing both the log entry and the state machine data. */ interface EntryWithData { + /** @return the index of this entry. */ + default long getIndex() { + try { + return getEntry(TimeDuration.ONE_MINUTE).getIndex(); + } catch (Exception e) { + throw new IllegalStateException("Failed to getIndex", e); + } + } + /** @return the serialized size including both log entry and state machine data. */ int getSerializedSize(); diff --git a/ratis-server/src/main/java/org/apache/ratis/server/leader/LogAppenderBase.java b/ratis-server/src/main/java/org/apache/ratis/server/leader/LogAppenderBase.java index f65ac1863c..7e87825b29 100644 --- a/ratis-server/src/main/java/org/apache/ratis/server/leader/LogAppenderBase.java +++ b/ratis-server/src/main/java/org/apache/ratis/server/leader/LogAppenderBase.java @@ -33,11 +33,14 @@ import org.apache.ratis.util.JavaUtils; import org.apache.ratis.util.LifeCycle; import org.apache.ratis.util.Preconditions; +import org.apache.ratis.util.ReferenceCountedObject; import org.apache.ratis.util.SizeInBytes; import org.apache.ratis.util.TimeDuration; import java.util.Collections; +import java.util.HashMap; import java.util.List; +import java.util.Map; import java.util.Objects; import java.util.concurrent.CompletableFuture; import java.util.concurrent.TimeUnit; @@ -208,16 +211,35 @@ protected LongUnaryOperator getNextIndexForError(long newNextIndex) { }; } - @Override - public AppendEntriesRequestProto newAppendEntriesRequest(long callId, boolean heartbeat) + public AppendEntriesRequestProto newAppendEntriesRequest(long callId, boolean heartbeat) { + throw new UnsupportedOperationException("Use nextAppendEntriesRequest(" + callId + ", " + heartbeat +") instead."); + } + +/** + * Create a {@link AppendEntriesRequestProto} object using the {@link FollowerInfo} of this {@link LogAppender}. + * The {@link AppendEntriesRequestProto} object may contain zero or more log entries. + * When there is zero log entries, the {@link AppendEntriesRequestProto} object is a heartbeat. + * + * @param callId The call id of the returned request. + * @param heartbeat the returned request must be a heartbeat. + * + * @return a retained reference of {@link AppendEntriesRequestProto} object. + * Since the returned reference is retained, the caller must call {@link ReferenceCountedObject#release()}} + * after use. + */ + protected ReferenceCountedObject nextAppendEntriesRequest(long callId, boolean heartbeat) throws RaftLogIOException { final long heartbeatWaitTimeMs = getHeartbeatWaitTimeMs(); final TermIndex previous = getPrevious(follower.getNextIndex()); if (heartbeatWaitTimeMs <= 0L || heartbeat) { // heartbeat - return leaderState.newAppendEntriesRequestProto(follower, Collections.emptyList(), - hasPendingDataRequests()? null : previous, callId); + AppendEntriesRequestProto heartbeatRequest = + leaderState.newAppendEntriesRequestProto(follower, Collections.emptyList(), + hasPendingDataRequests() ? null : previous, callId); + ReferenceCountedObject ref = ReferenceCountedObject.wrap(heartbeatRequest); + ref.retain(); + return ref; } Preconditions.assertTrue(buffer.isEmpty(), () -> "buffer has " + buffer.getNumElements() + " elements."); @@ -234,10 +256,14 @@ public AppendEntriesRequestProto newAppendEntriesRequest(long callId, boolean he } final long halfMs = heartbeatWaitTimeMs/2; - for (long next = followerNext; leaderNext > next && getHeartbeatWaitTimeMs() - halfMs > 0; ) { - if (!buffer.offer(getRaftLog().getEntryWithData(next++))) { + final Map> offered = new HashMap<>(); + for (long next = followerNext; leaderNext > next && getHeartbeatWaitTimeMs() - halfMs > 0; next++) { + final ReferenceCountedObject entryWithData = getRaftLog().retainEntryWithData(next); + if (!buffer.offer(entryWithData.get())) { + entryWithData.release(); break; } + offered.put(next, entryWithData); } if (buffer.isEmpty()) { return null; @@ -246,9 +272,15 @@ public AppendEntriesRequestProto newAppendEntriesRequest(long callId, boolean he final List protos = buffer.pollList(getHeartbeatWaitTimeMs(), EntryWithData::getEntry, (entry, time, exception) -> LOG.warn("Failed to get " + entry + " in " + time.toString(TimeUnit.MILLISECONDS, 3), exception)); + for (EntryWithData entry : buffer) { + // Release remaining entries. + offered.remove(entry.getIndex()).release(); + } buffer.clear(); assertProtos(protos, followerNext, previous, snapshotIndex); - return leaderState.newAppendEntriesRequestProto(follower, protos, previous, callId); + AppendEntriesRequestProto appendEntriesProto = + leaderState.newAppendEntriesRequestProto(follower, protos, previous, callId); + return ReferenceCountedObject.delegateFrom(offered.values(), appendEntriesProto); } private void assertProtos(List protos, long nextIndex, TermIndex previous, long snapshotIndex) { diff --git a/ratis-server/src/main/java/org/apache/ratis/server/leader/LogAppenderDefault.java b/ratis-server/src/main/java/org/apache/ratis/server/leader/LogAppenderDefault.java index 9d1edd4695..be92423beb 100644 --- a/ratis-server/src/main/java/org/apache/ratis/server/leader/LogAppenderDefault.java +++ b/ratis-server/src/main/java/org/apache/ratis/server/leader/LogAppenderDefault.java @@ -27,6 +27,7 @@ import org.apache.ratis.server.raftlog.RaftLogIOException; import org.apache.ratis.server.util.ServerStringUtils; import org.apache.ratis.statemachine.SnapshotInfo; +import org.apache.ratis.util.ReferenceCountedObject; import org.apache.ratis.util.Timestamp; import java.io.IOException; @@ -60,11 +61,15 @@ private AppendEntriesReplyProto sendAppendEntriesWithRetries(AtomicLong requestF throws InterruptedException, InterruptedIOException, RaftLogIOException { int retry = 0; - AppendEntriesRequestProto request = newAppendEntriesRequest(CallId.getAndIncrement(), false); + ReferenceCountedObject request = nextAppendEntriesRequest( + CallId.getAndIncrement(), false); while (isRunning()) { // keep retrying for IOException try { - if (request == null || request.getEntriesCount() == 0) { - request = newAppendEntriesRequest(CallId.getAndIncrement(), false); + if (request == null || request.get().getEntriesCount() == 0) { + if (request != null) { + request.release(); + } + request = nextAppendEntriesRequest(CallId.getAndIncrement(), false); } if (request == null) { @@ -75,18 +80,9 @@ private AppendEntriesReplyProto sendAppendEntriesWithRetries(AtomicLong requestF return null; } - resetHeartbeatTrigger(); - final Timestamp sendTime = Timestamp.currentTime(); - getFollower().updateLastRpcSendTime(request.getEntriesCount() == 0); - final AppendEntriesRequestProto proto = request; - final AppendEntriesReplyProto reply = getServerRpc().appendEntries(proto); - final long first = proto.getEntriesCount() > 0 ? proto.getEntries(0).getIndex() : RaftLog.INVALID_LOG_INDEX; - requestFirstIndex.set(first); - getFollower().updateLastRpcResponseTime(); - getFollower().updateLastRespondedAppendEntriesSendTime(sendTime); - - getLeaderState().onFollowerCommitIndex(getFollower(), reply.getFollowerCommit()); - return reply; + AppendEntriesReplyProto r = sendAppendEntries(request.get()); + request.release(); + return r; } catch (InterruptedIOException | RaftLogIOException e) { throw e; } catch (IOException ioe) { @@ -103,6 +99,18 @@ private AppendEntriesReplyProto sendAppendEntriesWithRetries(AtomicLong requestF return null; } + private AppendEntriesReplyProto sendAppendEntries(AppendEntriesRequestProto request) throws IOException { + resetHeartbeatTrigger(); + final Timestamp sendTime = Timestamp.currentTime(); + getFollower().updateLastRpcSendTime(request.getEntriesCount() == 0); + final AppendEntriesReplyProto r = getServerRpc().appendEntries(request); + getFollower().updateLastRpcResponseTime(); + getFollower().updateLastRespondedAppendEntriesSendTime(sendTime); + + getLeaderState().onFollowerCommitIndex(getFollower(), r.getFollowerCommit()); + return r; + } + private InstallSnapshotReplyProto installSnapshot(SnapshotInfo snapshot) throws InterruptedIOException { String requestId = UUID.randomUUID().toString(); InstallSnapshotReplyProto reply = null; diff --git a/ratis-server/src/main/java/org/apache/ratis/server/raftlog/RaftLogBase.java b/ratis-server/src/main/java/org/apache/ratis/server/raftlog/RaftLogBase.java index c4751770e4..2b5cf4fdbc 100644 --- a/ratis-server/src/main/java/org/apache/ratis/server/raftlog/RaftLogBase.java +++ b/ratis-server/src/main/java/org/apache/ratis/server/raftlog/RaftLogBase.java @@ -441,11 +441,16 @@ class EntryWithDataImpl implements EntryWithData { private ByteString checkStateMachineData(ByteString data) { if (data == null) { - throw new IllegalStateException("State machine data is null for log entry " + logEntry); + throw new IllegalStateException("State machine data is null for log entry " + this); } return data; } + @Override + public long getIndex() { + return logEntry.getIndex(); + } + @Override public int getSerializedSize() { return LogProtoUtils.getSerializedSize(logEntry); @@ -453,11 +458,11 @@ public int getSerializedSize() { @Override public LogEntryProto getEntry(TimeDuration timeout) throws RaftLogIOException, TimeoutException { - LogEntryProto entryProto; if (future == null) { return logEntry; } + final LogEntryProto entryProto; try { entryProto = future.thenApply(data -> LogProtoUtils.addStateMachineData(data, logEntry)) .get(timeout.getDuration(), timeout.getUnit()); @@ -470,14 +475,14 @@ public LogEntryProto getEntry(TimeDuration timeout) throws RaftLogIOException, T if (e instanceof InterruptedException) { Thread.currentThread().interrupt(); } - final String err = getName() + ": Failed readStateMachineData for " + toLogEntryString(logEntry); + final String err = getName() + ": Failed readStateMachineData for " + this; LOG.error(err, e); throw new RaftLogIOException(err, JavaUtils.unwrapCompletionException(e)); } // by this time we have already read the state machine data, // so the log entry data should be set now if (LogProtoUtils.isStateMachineDataEmpty(entryProto)) { - final String err = getName() + ": State machine data not set for " + toLogEntryString(logEntry); + final String err = getName() + ": State machine data not set for " + this; LOG.error(err); throw new RaftLogIOException(err); } diff --git a/ratis-server/src/main/java/org/apache/ratis/server/raftlog/memory/MemoryRaftLog.java b/ratis-server/src/main/java/org/apache/ratis/server/raftlog/memory/MemoryRaftLog.java index feedaeee4a..55036fac56 100644 --- a/ratis-server/src/main/java/org/apache/ratis/server/raftlog/memory/MemoryRaftLog.java +++ b/ratis-server/src/main/java/org/apache/ratis/server/raftlog/memory/MemoryRaftLog.java @@ -135,14 +135,14 @@ public ReferenceCountedObject retainLog(long index) { } @Override - public EntryWithData getEntryWithData(long index) { - // TODO. The reference counted object should be passed to LogAppender RATIS-2026. - ReferenceCountedObject ref = retainLog(index); - try { - return newEntryWithData(ref.get(), null); - } finally { - ref.release(); - } + public EntryWithData getEntryWithData(long index) throws RaftLogIOException { + throw new UnsupportedOperationException("Use retainEntryWithData(" + index + ") instead."); + } + + @Override + public ReferenceCountedObject retainEntryWithData(long index) { + final ReferenceCountedObject ref = retainLog(index); + return ref.delegate(newEntryWithData(ref.get(), null)); } @Override diff --git a/ratis-server/src/main/java/org/apache/ratis/server/raftlog/segmented/SegmentedRaftLog.java b/ratis-server/src/main/java/org/apache/ratis/server/raftlog/segmented/SegmentedRaftLog.java index 786257c0eb..eff52931ab 100644 --- a/ratis-server/src/main/java/org/apache/ratis/server/raftlog/segmented/SegmentedRaftLog.java +++ b/ratis-server/src/main/java/org/apache/ratis/server/raftlog/segmented/SegmentedRaftLog.java @@ -326,21 +326,19 @@ record = segment.getLogRecord(index); @Override public EntryWithData getEntryWithData(long index) throws RaftLogIOException { + throw new UnsupportedOperationException("Use retainEntryWithData(" + index + ") instead."); + } + + @Override + public ReferenceCountedObject retainEntryWithData(long index) throws RaftLogIOException { final ReferenceCountedObject entryRef = retainLog(index); if (entryRef == null) { throw new RaftLogIOException("Log entry not found: index = " + index); } - try { - // TODO. The reference counted object should be passed to LogAppender RATIS-2026. - return getEntryWithData(entryRef.get()); - } finally { - entryRef.release(); - } - } - private EntryWithData getEntryWithData(LogEntryProto entry) throws RaftLogIOException { + final LogEntryProto entry = entryRef.get(); if (!LogProtoUtils.isStateMachineDataEmpty(entry)) { - return newEntryWithData(entry, null); + return entryRef.delegate(newEntryWithData(entry, null)); } try { @@ -351,7 +349,7 @@ private EntryWithData getEntryWithData(LogEntryProto entry) throws RaftLogIOExce throw new CompletionException("Failed to read state machine data for log entry " + entry, ex); }); } - return newEntryWithData(entry, future); + return entryRef.delegate(newEntryWithData(entry, future)); } catch (Exception e) { final String err = getName() + ": Failed readStateMachineData for " + toLogEntryString(entry); LOG.error(err, e); From 05e0d6f3acfec1aed24646c8c5b23fc1a6648f38 Mon Sep 17 00:00:00 2001 From: Duong Nguyen Date: Tue, 2 Apr 2024 07:09:24 -0700 Subject: [PATCH 343/397] RATIS-1979. Allow StateMachine.read to return a ReferentCountedObject (#1062) (cherry picked from commit c73a3eb8c026133e0e9b8ada1a2aeb467812cb37) --- .../ratis/examples/filestore/FileInfo.java | 3 +- .../ratis/statemachine/StateMachine.java | 22 +++++++ .../ratis/server/raftlog/LogProtoUtils.java | 10 ++- .../ratis/server/raftlog/RaftLogBase.java | 62 ++++++++++++++++--- .../server/raftlog/memory/MemoryRaftLog.java | 2 +- .../server/raftlog/segmented/LogSegment.java | 21 ++++--- .../raftlog/segmented/SegmentedRaftLog.java | 10 +-- 7 files changed, 106 insertions(+), 24 deletions(-) diff --git a/ratis-examples/src/main/java/org/apache/ratis/examples/filestore/FileInfo.java b/ratis-examples/src/main/java/org/apache/ratis/examples/filestore/FileInfo.java index c7d8cb7cd1..bba001002a 100644 --- a/ratis-examples/src/main/java/org/apache/ratis/examples/filestore/FileInfo.java +++ b/ratis-examples/src/main/java/org/apache/ratis/examples/filestore/FileInfo.java @@ -19,6 +19,7 @@ import org.apache.ratis.protocol.RaftPeerId; import org.apache.ratis.thirdparty.com.google.protobuf.ByteString; +import org.apache.ratis.thirdparty.com.google.protobuf.UnsafeByteOperations; import org.apache.ratis.util.CollectionUtils; import org.apache.ratis.util.JavaUtils; import org.apache.ratis.util.LogUtils; @@ -84,7 +85,7 @@ ByteString read(CheckedFunction resolver, long offset, final ByteBuffer buffer = ByteBuffer.allocateDirect(FileStoreCommon.getChunkSize(length)); in.position(offset).read(buffer); buffer.flip(); - return ByteString.copyFrom(buffer); + return UnsafeByteOperations.unsafeWrap(buffer); } } diff --git a/ratis-server-api/src/main/java/org/apache/ratis/statemachine/StateMachine.java b/ratis-server-api/src/main/java/org/apache/ratis/statemachine/StateMachine.java index 176811eb18..3960ab8287 100644 --- a/ratis-server-api/src/main/java/org/apache/ratis/statemachine/StateMachine.java +++ b/ratis-server-api/src/main/java/org/apache/ratis/statemachine/StateMachine.java @@ -88,6 +88,28 @@ default CompletableFuture read(LogEntryProto entry, TransactionConte return read(entry); } + /** + * Read asynchronously the state machine data from this state machine. + * StateMachines implement this method when the read result contains retained resources that should be released + * after use. + * + * @return a future for the read task. The result of the future is a {@link ReferenceCountedObject} wrapping the + * read result. Client code of this method must call {@link ReferenceCountedObject#release()} after + * use. + */ + default CompletableFuture> retainRead(LogEntryProto entry, + TransactionContext context) { + return read(entry, context).thenApply(r -> { + if (r == null) { + return null; + } + ReferenceCountedObject ref = ReferenceCountedObject.wrap(r); + ref.retain(); + return ref; + + }); + } + /** * Write asynchronously the state machine data in the given log entry to this state machine. * diff --git a/ratis-server/src/main/java/org/apache/ratis/server/raftlog/LogProtoUtils.java b/ratis-server/src/main/java/org/apache/ratis/server/raftlog/LogProtoUtils.java index 610b02001a..c3943f7d9f 100644 --- a/ratis-server/src/main/java/org/apache/ratis/server/raftlog/LogProtoUtils.java +++ b/ratis-server/src/main/java/org/apache/ratis/server/raftlog/LogProtoUtils.java @@ -156,8 +156,9 @@ public static LogEntryProto removeStateMachineData(LogEntryProto entry) { } private static LogEntryProto replaceStateMachineDataWithSerializedSize(LogEntryProto entry) { - return replaceStateMachineEntry(entry, + LogEntryProto replaced = replaceStateMachineEntry(entry, StateMachineEntryProto.newBuilder().setLogEntryProtoSerializedSize(entry.getSerializedSize())); + return copy(replaced); } private static LogEntryProto replaceStateMachineEntry(LogEntryProto proto, StateMachineEntryProto.Builder newEntry) { @@ -179,6 +180,13 @@ static LogEntryProto addStateMachineData(ByteString stateMachineData, LogEntryPr return replaceStateMachineEntry(entry, StateMachineEntryProto.newBuilder().setStateMachineData(stateMachineData)); } + public static boolean hasStateMachineData(LogEntryProto entry) { + return getStateMachineEntry(entry) + .map(StateMachineEntryProto::getStateMachineData) + .map(data -> !data.isEmpty()) + .orElse(false); + } + public static boolean isStateMachineDataEmpty(LogEntryProto entry) { return getStateMachineEntry(entry) .map(StateMachineEntryProto::getStateMachineData) diff --git a/ratis-server/src/main/java/org/apache/ratis/server/raftlog/RaftLogBase.java b/ratis-server/src/main/java/org/apache/ratis/server/raftlog/RaftLogBase.java index 2b5cf4fdbc..cdf501301d 100644 --- a/ratis-server/src/main/java/org/apache/ratis/server/raftlog/RaftLogBase.java +++ b/ratis-server/src/main/java/org/apache/ratis/server/raftlog/RaftLogBase.java @@ -17,6 +17,7 @@ */ package org.apache.ratis.server.raftlog; +import java.util.Objects; import java.util.concurrent.atomic.AtomicReference; import org.apache.ratis.conf.RaftProperties; import org.apache.ratis.proto.RaftProtos.LogEntryProto; @@ -423,8 +424,43 @@ public String getName() { return name; } - protected EntryWithData newEntryWithData(LogEntryProto logEntry, CompletableFuture future) { - return new EntryWithDataImpl(logEntry, future); + protected ReferenceCountedObject newEntryWithData(ReferenceCountedObject retained) { + return retained.delegate(new EntryWithDataImpl(retained.get(), null)); + } + + protected ReferenceCountedObject newEntryWithData(ReferenceCountedObject retained, + CompletableFuture> stateMachineDataFuture) { + final EntryWithDataImpl impl = new EntryWithDataImpl(retained.get(), stateMachineDataFuture); + return new ReferenceCountedObject() { + private CompletableFuture> future + = Objects.requireNonNull(stateMachineDataFuture, "stateMachineDataFuture == null"); + + @Override + public EntryWithData get() { + return impl; + } + + synchronized void updateFuture(Consumer> action) { + future = future.whenComplete((ref, e) -> { + if (ref != null) { + action.accept(ref); + } + }); + } + + @Override + public EntryWithData retain() { + retained.retain(); + updateFuture(ReferenceCountedObject::retain); + return impl; + } + + @Override + public boolean release() { + updateFuture(ReferenceCountedObject::release); + return retained.release(); + } + }; } /** @@ -432,14 +468,14 @@ protected EntryWithData newEntryWithData(LogEntryProto logEntry, CompletableFutu */ class EntryWithDataImpl implements EntryWithData { private final LogEntryProto logEntry; - private final CompletableFuture future; + private final CompletableFuture> future; - EntryWithDataImpl(LogEntryProto logEntry, CompletableFuture future) { + EntryWithDataImpl(LogEntryProto logEntry, CompletableFuture> future) { this.logEntry = logEntry; this.future = future == null? null: future.thenApply(this::checkStateMachineData); } - private ByteString checkStateMachineData(ByteString data) { + private ReferenceCountedObject checkStateMachineData(ReferenceCountedObject data) { if (data == null) { throw new IllegalStateException("State machine data is null for log entry " + this); } @@ -463,18 +499,21 @@ public LogEntryProto getEntry(TimeDuration timeout) throws RaftLogIOException, T } final LogEntryProto entryProto; + ReferenceCountedObject data; try { - entryProto = future.thenApply(data -> LogProtoUtils.addStateMachineData(data, logEntry)) - .get(timeout.getDuration(), timeout.getUnit()); + data = future.get(timeout.getDuration(), timeout.getUnit()); + entryProto = LogProtoUtils.addStateMachineData(data.get(), logEntry); } catch (TimeoutException t) { if (timeout.compareTo(stateMachineDataReadTimeout) > 0) { getRaftLogMetrics().onStateMachineDataReadTimeout(); } + discardData(); throw t; } catch (Exception e) { if (e instanceof InterruptedException) { Thread.currentThread().interrupt(); } + discardData(); final String err = getName() + ": Failed readStateMachineData for " + this; LOG.error(err, e); throw new RaftLogIOException(err, JavaUtils.unwrapCompletionException(e)); @@ -484,11 +523,20 @@ public LogEntryProto getEntry(TimeDuration timeout) throws RaftLogIOException, T if (LogProtoUtils.isStateMachineDataEmpty(entryProto)) { final String err = getName() + ": State machine data not set for " + this; LOG.error(err); + data.release(); throw new RaftLogIOException(err); } return entryProto; } + private void discardData() { + future.whenComplete((r, ex) -> { + if (r != null) { + r.release(); + } + }); + } + @Override public String toString() { return toLogEntryString(logEntry); diff --git a/ratis-server/src/main/java/org/apache/ratis/server/raftlog/memory/MemoryRaftLog.java b/ratis-server/src/main/java/org/apache/ratis/server/raftlog/memory/MemoryRaftLog.java index 55036fac56..2aac6c1b1f 100644 --- a/ratis-server/src/main/java/org/apache/ratis/server/raftlog/memory/MemoryRaftLog.java +++ b/ratis-server/src/main/java/org/apache/ratis/server/raftlog/memory/MemoryRaftLog.java @@ -142,7 +142,7 @@ public EntryWithData getEntryWithData(long index) throws RaftLogIOException { @Override public ReferenceCountedObject retainEntryWithData(long index) { final ReferenceCountedObject ref = retainLog(index); - return ref.delegate(newEntryWithData(ref.get(), null)); + return newEntryWithData(ref); } @Override diff --git a/ratis-server/src/main/java/org/apache/ratis/server/raftlog/segmented/LogSegment.java b/ratis-server/src/main/java/org/apache/ratis/server/raftlog/segmented/LogSegment.java index 601e3aeca6..76a0dd35c1 100644 --- a/ratis-server/src/main/java/org/apache/ratis/server/raftlog/segmented/LogSegment.java +++ b/ratis-server/src/main/java/org/apache/ratis/server/raftlog/segmented/LogSegment.java @@ -69,15 +69,18 @@ enum Op { } static long getEntrySize(LogEntryProto entry, Op op) { - LogEntryProto e = entry; - if (op == Op.CHECK_SEGMENT_FILE_FULL) { - e = LogProtoUtils.removeStateMachineData(entry); - } else if (op == Op.LOAD_SEGMENT_FILE || op == Op.WRITE_CACHE_WITH_STATE_MACHINE_CACHE) { - Preconditions.assertTrue(entry == LogProtoUtils.removeStateMachineData(entry), - () -> "Unexpected LogEntryProto with StateMachine data: op=" + op + ", entry=" + entry); - } else { - Preconditions.assertTrue(op == Op.WRITE_CACHE_WITHOUT_STATE_MACHINE_CACHE || op == Op.REMOVE_CACHE, - () -> "Unexpected op " + op + ", entry=" + entry); + switch (op) { + case CHECK_SEGMENT_FILE_FULL: + case LOAD_SEGMENT_FILE: + case WRITE_CACHE_WITH_STATE_MACHINE_CACHE: + Preconditions.assertTrue(!LogProtoUtils.hasStateMachineData(entry), + () -> "Unexpected LogEntryProto with StateMachine data: op=" + op + ", entry=" + entry); + break; + case WRITE_CACHE_WITHOUT_STATE_MACHINE_CACHE: + case REMOVE_CACHE: + break; + default: + throw new IllegalStateException("Unexpected op " + op + ", entry=" + entry); } final int serialized = e.getSerializedSize(); return serialized + CodedOutputStream.computeUInt32SizeNoTag(serialized) + 4L; diff --git a/ratis-server/src/main/java/org/apache/ratis/server/raftlog/segmented/SegmentedRaftLog.java b/ratis-server/src/main/java/org/apache/ratis/server/raftlog/segmented/SegmentedRaftLog.java index eff52931ab..bc22eaaf30 100644 --- a/ratis-server/src/main/java/org/apache/ratis/server/raftlog/segmented/SegmentedRaftLog.java +++ b/ratis-server/src/main/java/org/apache/ratis/server/raftlog/segmented/SegmentedRaftLog.java @@ -338,18 +338,18 @@ public ReferenceCountedObject retainEntryWithData(long index) thr final LogEntryProto entry = entryRef.get(); if (!LogProtoUtils.isStateMachineDataEmpty(entry)) { - return entryRef.delegate(newEntryWithData(entry, null)); + return newEntryWithData(entryRef); } try { - CompletableFuture future = null; + CompletableFuture> future = null; if (stateMachine != null) { - future = stateMachine.data().read(entry, server.getTransactionContext(entry, false)).exceptionally(ex -> { + future = stateMachine.data().retainRead(entry, server.getTransactionContext(entry, false)).exceptionally(ex -> { stateMachine.event().notifyLogFailed(ex, entry); throw new CompletionException("Failed to read state machine data for log entry " + entry, ex); }); } - return entryRef.delegate(newEntryWithData(entry, future)); + return future != null? newEntryWithData(entryRef, future): newEntryWithData(entryRef); } catch (Exception e) { final String err = getName() + ": Failed readStateMachineData for " + toLogEntryString(entry); LOG.error(err, e); @@ -464,7 +464,7 @@ protected CompletableFuture appendEntryImpl(ReferenceCountedObject Date: Thu, 4 Apr 2024 00:16:16 +0800 Subject: [PATCH 344/397] RATIS-2051. Fix Sonar bugs for Ratis. (#1059) (cherry picked from commit f40424422b692349b5496ee30e24335c8186093b) --- .../src/main/java/org/apache/ratis/util/LeakDetector.java | 1 + .../org/apache/ratis/server/impl/ConfigurationManager.java | 6 ++++-- .../ratis/statemachine/impl/TransactionContextImpl.java | 2 ++ 3 files changed, 7 insertions(+), 2 deletions(-) diff --git a/ratis-common/src/main/java/org/apache/ratis/util/LeakDetector.java b/ratis-common/src/main/java/org/apache/ratis/util/LeakDetector.java index d801868163..82202f2884 100644 --- a/ratis-common/src/main/java/org/apache/ratis/util/LeakDetector.java +++ b/ratis-common/src/main/java/org/apache/ratis/util/LeakDetector.java @@ -84,6 +84,7 @@ private void run() { tracker.reportLeak(); } } catch (InterruptedException e) { + Thread.currentThread().interrupt(); LOG.warn("Thread interrupted, exiting.", e); break; } diff --git a/ratis-server/src/main/java/org/apache/ratis/server/impl/ConfigurationManager.java b/ratis-server/src/main/java/org/apache/ratis/server/impl/ConfigurationManager.java index 2ba8107837..5cf3535a61 100644 --- a/ratis-server/src/main/java/org/apache/ratis/server/impl/ConfigurationManager.java +++ b/ratis-server/src/main/java/org/apache/ratis/server/impl/ConfigurationManager.java @@ -44,9 +44,11 @@ public class ConfigurationManager { * The current raft configuration. If configurations is not empty, should be * the last entry of the map. Otherwise is initialConf. */ - private RaftConfigurationImpl currentConf; + @SuppressWarnings({"squid:S3077"}) // Suppress volatile for generic type + private volatile RaftConfigurationImpl currentConf; /** Cache the peer corresponding to {@link #id}. */ - private RaftPeer currentPeer; + @SuppressWarnings({"squid:S3077"}) // Suppress volatile for generic type + private volatile RaftPeer currentPeer; ConfigurationManager(RaftPeerId id, RaftConfigurationImpl initialConf) { this.id = id; diff --git a/ratis-server/src/main/java/org/apache/ratis/statemachine/impl/TransactionContextImpl.java b/ratis-server/src/main/java/org/apache/ratis/statemachine/impl/TransactionContextImpl.java index ac8d534f26..18ed297f87 100644 --- a/ratis-server/src/main/java/org/apache/ratis/statemachine/impl/TransactionContextImpl.java +++ b/ratis-server/src/main/java/org/apache/ratis/statemachine/impl/TransactionContextImpl.java @@ -74,9 +74,11 @@ public class TransactionContextImpl implements TransactionContext { @SuppressWarnings({"squid:S3077"}) // Suppress volatile for generic type private volatile LogEntryProto logEntry; /** Committed LogEntry copy. */ + @SuppressWarnings({"squid:S3077"}) // Suppress volatile for generic type private volatile Supplier logEntryCopy; /** For wrapping {@link #logEntry} in order to release the underlying buffer. */ + @SuppressWarnings({"squid:S3077"}) // Suppress volatile for generic type private volatile ReferenceCountedObject delegatedRef; private final CompletableFuture logIndexFuture = new CompletableFuture<>(); From 7e188c83140f6eb46cef1faaf3b9f0f0ddca77fc Mon Sep 17 00:00:00 2001 From: Duong Nguyen Date: Wed, 17 Apr 2024 10:10:13 -0700 Subject: [PATCH 345/397] RATIS-2059. Missing reference count when putting log entries to cache on follower. (#1067) (cherry picked from commit ee09075c0e930756e5ad390fc9250bd5d1ffea52) --- .../apache/ratis/server/raftlog/segmented/SegmentedRaftLog.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/ratis-server/src/main/java/org/apache/ratis/server/raftlog/segmented/SegmentedRaftLog.java b/ratis-server/src/main/java/org/apache/ratis/server/raftlog/segmented/SegmentedRaftLog.java index bc22eaaf30..e3f2d96199 100644 --- a/ratis-server/src/main/java/org/apache/ratis/server/raftlog/segmented/SegmentedRaftLog.java +++ b/ratis-server/src/main/java/org/apache/ratis/server/raftlog/segmented/SegmentedRaftLog.java @@ -461,7 +461,7 @@ protected CompletableFuture appendEntryImpl(ReferenceCountedObject Date: Fri, 19 Apr 2024 23:20:16 +0800 Subject: [PATCH 346/397] RATIS-2055. Move notifyTermIndexUpdated after leader.checkReady (#1068) (cherry picked from commit 8704cd23d3b38a90780ef34a7ed5f7c417ea1207) --- .../java/org/apache/ratis/server/impl/RaftServerImpl.java | 4 +--- 1 file changed, 1 insertion(+), 3 deletions(-) diff --git a/ratis-server/src/main/java/org/apache/ratis/server/impl/RaftServerImpl.java b/ratis-server/src/main/java/org/apache/ratis/server/impl/RaftServerImpl.java index a10066a39f..9394e2f8cb 100644 --- a/ratis-server/src/main/java/org/apache/ratis/server/impl/RaftServerImpl.java +++ b/ratis-server/src/main/java/org/apache/ratis/server/impl/RaftServerImpl.java @@ -1884,9 +1884,7 @@ TransactionContext getTransactionContext(LogEntryProto entry, Boolean createNew) CompletableFuture applyLogToStateMachine(ReferenceCountedObject nextRef) throws RaftLogIOException { LogEntryProto next = nextRef.get(); - if (!next.hasStateMachineLogEntry()) { - stateMachine.event().notifyTermIndexUpdated(next.getTerm(), next.getIndex()); - } + CompletableFuture messageFuture = null; switch (next.getLogEntryBodyCase()) { case CONFIGURATIONENTRY: From c0e6f48dc9e4e140a124bf0a1dbafdc48f56fec4 Mon Sep 17 00:00:00 2001 From: Duong Nguyen Date: Fri, 3 May 2024 11:48:52 -0700 Subject: [PATCH 347/397] RATIS-2068. Avoid logging raw StateMachine data body. (#1074) (cherry picked from commit 51244e42cabb98f7ac9ca144e311222d9c6e5a45) --- .../ratis/server/impl/RaftServerImpl.java | 5 +++-- .../ratis/server/raftlog/LogProtoUtils.java | 20 +++---------------- .../ratis/server/util/ServerStringUtils.java | 5 +++-- 3 files changed, 9 insertions(+), 21 deletions(-) diff --git a/ratis-server/src/main/java/org/apache/ratis/server/impl/RaftServerImpl.java b/ratis-server/src/main/java/org/apache/ratis/server/impl/RaftServerImpl.java index 9394e2f8cb..50fd46ed33 100644 --- a/ratis-server/src/main/java/org/apache/ratis/server/impl/RaftServerImpl.java +++ b/ratis-server/src/main/java/org/apache/ratis/server/impl/RaftServerImpl.java @@ -1539,7 +1539,8 @@ public CompletableFuture appendEntriesAsync( return appendEntriesAsync(leaderId, request.getCallId(), previous, requestRef); } catch(Exception t) { - LOG.error("{}: Failed appendEntries* {}", getMemberId(), toAppendEntriesRequestString(r), t); + LOG.error("{}: Failed appendEntries* {}", getMemberId(), + toAppendEntriesRequestString(r, stateMachine::toStateMachineLogEntryString), t); throw IOUtils.asIOException(t); } finally { requestRef.release(); @@ -1598,7 +1599,7 @@ private CompletableFuture appendEntriesAsync(RaftPeerId final List entries = proto.getEntriesList(); final boolean isHeartbeat = entries.isEmpty(); logAppendEntries(isHeartbeat, () -> getMemberId() + ": appendEntries* " - + toAppendEntriesRequestString(proto)); + + toAppendEntriesRequestString(proto, stateMachine::toStateMachineLogEntryString)); final long leaderTerm = proto.getLeaderTerm(); final long currentTerm; diff --git a/ratis-server/src/main/java/org/apache/ratis/server/raftlog/LogProtoUtils.java b/ratis-server/src/main/java/org/apache/ratis/server/raftlog/LogProtoUtils.java index c3943f7d9f..e969eaa484 100644 --- a/ratis-server/src/main/java/org/apache/ratis/server/raftlog/LogProtoUtils.java +++ b/ratis-server/src/main/java/org/apache/ratis/server/raftlog/LogProtoUtils.java @@ -88,23 +88,9 @@ public static String toLogEntriesString(List entries) { public static String toLogEntriesShortString(List entries, Function stateMachineToString) { - if (entries == null) { - return null; - } - return toLogEntryTermIndexString(entries) - + (entries.isEmpty() ? "" : ", first=" + toLogEntryString(entries.get(0), stateMachineToString)); - } - - public static String toLogEntryTermIndexString(List entries) { - final int n = entries.size(); - return n == 0 ? toLogEntryTermIndexString(n, null, null) - : toLogEntryTermIndexString(n, TermIndex.valueOf(entries.get(0)), TermIndex.valueOf(entries.get(n - 1))); - } - - public static String toLogEntryTermIndexString(int n, TermIndex first, TermIndex last) { - return n == 0 ? "HEARTBEAT" - : n == 1 ? "entry=" + first - : n + " entries=" + first + "..." + last; + return entries == null ? null + : entries.isEmpty()? "" + : "size=" + entries.size() + ", first=" + toLogEntryString(entries.get(0), stateMachineToString); } public static LogEntryProto toLogEntryProto(RaftConfiguration conf, Long term, long index) { diff --git a/ratis-server/src/main/java/org/apache/ratis/server/util/ServerStringUtils.java b/ratis-server/src/main/java/org/apache/ratis/server/util/ServerStringUtils.java index 8ca314ff24..d3c7a4bc70 100644 --- a/ratis-server/src/main/java/org/apache/ratis/server/util/ServerStringUtils.java +++ b/ratis-server/src/main/java/org/apache/ratis/server/util/ServerStringUtils.java @@ -24,12 +24,12 @@ import org.apache.ratis.proto.RaftProtos.LogEntryProto; import org.apache.ratis.proto.RaftProtos.RequestVoteReplyProto; import org.apache.ratis.proto.RaftProtos.StateMachineLogEntryProto; -import org.apache.ratis.protocol.RaftGroupMemberId; import org.apache.ratis.server.protocol.TermIndex; import org.apache.ratis.util.JavaUtils; import org.apache.ratis.util.ProtoUtils; import java.util.List; +import java.util.function.Function; /** * This class provides convenient utilities for converting Protocol Buffers messages to strings. @@ -54,7 +54,8 @@ public static String toAppendEntriesRequestString(AppendEntriesRequestProto requ + ",previous=" + TermIndex.valueOf(request.getPreviousLog()) + ",leaderCommit=" + request.getLeaderCommit() + ",initializing? " + request.getInitializing() - + "," + (entries.isEmpty()? "HEARTBEAT" : "entries: " + LogProtoUtils.toLogEntriesShortString(entries)); + + "," + (entries.isEmpty()? "HEARTBEAT" : "entries: " + + LogProtoUtils.toLogEntriesShortString(entries, stateMachineToString)); } public static String toAppendEntriesReplyString(AppendEntriesReplyProto reply) { From e2ba1beba0c154d9a7fb0b0f3673161969d5bd64 Mon Sep 17 00:00:00 2001 From: Tsz-Wo Nicholas Sze Date: Fri, 10 May 2024 08:09:20 -0700 Subject: [PATCH 348/397] RATIS-2080. Reuse LeaderElection executor. (#1082) (cherry picked from commit 8c9c801e5b8edc68971911e5819e89e74e114683) --- .../ratis/server/impl/LeaderElection.java | 58 +++++++------------ .../ratis/server/impl/RaftServerImpl.java | 23 +++++++- 2 files changed, 41 insertions(+), 40 deletions(-) diff --git a/ratis-server/src/main/java/org/apache/ratis/server/impl/LeaderElection.java b/ratis-server/src/main/java/org/apache/ratis/server/impl/LeaderElection.java index 385d338335..acf1b4f3d5 100644 --- a/ratis-server/src/main/java/org/apache/ratis/server/impl/LeaderElection.java +++ b/ratis-server/src/main/java/org/apache/ratis/server/impl/LeaderElection.java @@ -43,16 +43,17 @@ import java.util.ArrayList; import java.util.Collection; import java.util.HashMap; +import java.util.Iterator; import java.util.List; import java.util.Map; import java.util.Optional; import java.util.concurrent.Callable; import java.util.concurrent.CompletableFuture; import java.util.concurrent.ExecutionException; -import java.util.concurrent.ExecutorCompletionService; import java.util.concurrent.ExecutorService; import java.util.concurrent.Executors; import java.util.concurrent.Future; +import java.util.concurrent.TimeoutException; import java.util.concurrent.atomic.AtomicInteger; import java.util.Set; import java.util.stream.Collectors; @@ -247,28 +248,22 @@ public String toString() { } static class Executor { - private final ExecutorCompletionService service; private final ExecutorService executor; private final AtomicInteger count = new AtomicInteger(); Executor(Object name, int size) { Preconditions.assertTrue(size > 0); - executor = Executors.newFixedThreadPool(size, r -> + executor = Executors.newCachedThreadPool(r -> Daemon.newBuilder().setName(name + "-" + count.incrementAndGet()).setRunnable(r).build()); - service = new ExecutorCompletionService<>(executor); } - void shutdown() { - executor.shutdownNow(); + ExecutorService getExecutor() { + return executor; } - void submit(Callable task) { - service.submit(task); - } - - Future poll(TimeDuration waitTime) throws InterruptedException { - return service.poll(waitTime.getDuration(), waitTime.getUnit()); + Future submit(Callable task) { + return executor.submit(task); } } @@ -425,14 +420,10 @@ private ResultAndTerm submitRequestAndWaitResult(Phase phase, RaftConfigurationI if (others.isEmpty()) { r = new ResultAndTerm(Result.PASSED, electionTerm); } else { - final TermIndex lastEntry = server.getLastEntry(); - final Executor voteExecutor = new Executor(this, others.size()); - try { - final int submitted = submitRequests(phase, electionTerm, lastEntry, others, voteExecutor); - r = waitForResults(phase, electionTerm, submitted, conf, voteExecutor); - } finally { - voteExecutor.shutdown(); - } + final TermIndex lastEntry = server.getState().getLastEntry(); + final List> submitted = submitRequests( + phase, electionTerm, lastEntry, others, server.getLeaderElectionExecutor()); + r = waitForResults(phase, electionTerm, submitted, conf); } return r; @@ -482,14 +473,13 @@ private boolean askForVotes(Phase phase, int round) throws InterruptedException, } } - private int submitRequests(Phase phase, long electionTerm, TermIndex lastEntry, + private List> submitRequests(Phase phase, long electionTerm, TermIndex lastEntry, Collection others, Executor voteExecutor) { - int submitted = 0; + final List> submitted = new ArrayList<>(others.size()); for (final RaftPeer peer : others) { final RequestVoteRequestProto r = ServerProtoUtils.toRequestVoteRequestProto( server.getMemberId(), peer.getId(), electionTerm, lastEntry, phase == Phase.PRE_VOTE); - voteExecutor.submit(() -> server.requestVote(r)); - submitted++; + submitted.add(voteExecutor.submit(() -> server.getServerRpc().requestVote(r))); } return submitted; } @@ -503,21 +493,17 @@ private Set getHigherPriorityPeers(RaftConfiguration conf) { .collect(Collectors.toSet()); } - private ResultAndTerm waitForResults(Phase phase, long electionTerm, int submitted, - RaftConfigurationImpl conf, Executor voteExecutor) throws InterruptedException { + private ResultAndTerm waitForResults(Phase phase, long electionTerm, List> submitted, + RaftConfigurationImpl conf) throws InterruptedException { final Timestamp timeout = Timestamp.currentTime().addTime(server.getRandomElectionTimeout()); final Map responses = new HashMap<>(); final List exceptions = new ArrayList<>(); - int waitForNum = submitted; Collection votedPeers = new ArrayList<>(); Collection rejectedPeers = new ArrayList<>(); Set higherPriorityPeers = getHigherPriorityPeers(conf); final boolean singleMode = conf.isSingleMode(server.getId()); - // true iff this server does not have any commits - final boolean emptyCommit = server.getLastCommittedIndex() < RaftLog.LEAST_VALID_LOG_INDEX; - - while (waitForNum > 0 && shouldRun(electionTerm)) { + for(Iterator> i = submitted.iterator(); i.hasNext() && shouldRun(electionTerm); ) { final TimeDuration waitTime = timeout.elapsedTime().apply(n -> -n); if (waitTime.isNonPositive()) { if (conf.hasMajority(votedPeers, server.getId())) { @@ -533,12 +519,9 @@ private ResultAndTerm waitForResults(Phase phase, long electionTerm, int submitt } try { - final Future future = voteExecutor.poll(waitTime); - if (future == null) { - continue; // poll timeout, continue to return Result.TIMEOUT - } + final Future future = i.next(); + final RequestVoteReplyProto r = future.get(waitTime.getDuration(), waitTime.getUnit()); - final RequestVoteReplyProto r = future.get(); final RaftPeerId replierId = RaftPeerId.valueOf(r.getServerReply().getReplyId()); final RequestVoteReplyProto previous = responses.putIfAbsent(replierId, r); if (previous != null) { @@ -585,8 +568,9 @@ private ResultAndTerm waitForResults(Phase phase, long electionTerm, int submitt } catch(ExecutionException e) { LogUtils.infoOrTrace(LOG, () -> this + " got exception when requesting votes", e); exceptions.add(e); + } catch (TimeoutException e) { + // get timeout, continue to return Result.TIMEOUT } - waitForNum--; } // received all the responses if (conf.hasMajority(votedPeers, server.getId())) { diff --git a/ratis-server/src/main/java/org/apache/ratis/server/impl/RaftServerImpl.java b/ratis-server/src/main/java/org/apache/ratis/server/impl/RaftServerImpl.java index 50fd46ed33..88f20280fd 100644 --- a/ratis-server/src/main/java/org/apache/ratis/server/impl/RaftServerImpl.java +++ b/ratis-server/src/main/java/org/apache/ratis/server/impl/RaftServerImpl.java @@ -250,6 +250,9 @@ public long[] getFollowerMatchIndices() { private final ExecutorService serverExecutor; private final ExecutorService clientExecutor; + private final MemoizedSupplier leaderElectionExecutor; + + private final AtomicBoolean firstElectionSinceStartup = new AtomicBoolean(true); private final ThreadGroup threadGroup; RaftServerImpl(RaftGroup group, StateMachine stateMachine, RaftServerProxy proxy, RaftStorage.StartupOption option) @@ -286,15 +289,17 @@ public long[] getFollowerMatchIndices() { this.appendLogTermIndices = RaftServerConfigKeys.Log.appendEntriesComposeEnabled(properties) ? new NavigableIndices() : null; + final RaftGroupMemberId memberId = getMemberId(); this.serverExecutor = ConcurrentUtils.newThreadPoolWithMax( RaftServerConfigKeys.ThreadPool.serverCached(properties), RaftServerConfigKeys.ThreadPool.serverSize(properties), - id + "-server"); + memberId + "-server"); this.clientExecutor = ConcurrentUtils.newThreadPoolWithMax( RaftServerConfigKeys.ThreadPool.clientCached(properties), RaftServerConfigKeys.ThreadPool.clientSize(properties), - id + "-client"); - this.threadGroup = new ThreadGroup(proxy.getThreadGroup(), getMemberId().toString()); + memberId + "-client"); + this.leaderElectionExecutor = MemoizedSupplier.valueOf( + () -> new LeaderElection.Executor(memberId + "-election" , group.getPeers().size())); } private long getCommitIndex(RaftPeerId id) { @@ -557,6 +562,14 @@ public void close() { } catch (Exception e) { LOG.warn("{}: Failed to shutdown serverExecutor", getMemberId(), e); } + + if (leaderElectionExecutor.isInitialized()) { + try { + ConcurrentUtils.shutdownAndWait(leaderElectionExecutor.get().getExecutor()); + } catch (Exception e) { + LOG.warn("{}: Failed to shutdown leaderElectionExecutor", getMemberId(), e); + } + } closeFinishedLatch.countDown(); }); } @@ -1593,6 +1606,10 @@ ExecutorService getServerExecutor() { return serverExecutor; } + LeaderElection.Executor getLeaderElectionExecutor() { + return leaderElectionExecutor.get(); + } + private CompletableFuture appendEntriesAsync(RaftPeerId leaderId, long callId, TermIndex previous, ReferenceCountedObject requestRef) throws IOException { final AppendEntriesRequestProto proto = requestRef.get(); From 740a7eee3850408fc20402fee756c979755fa8b0 Mon Sep 17 00:00:00 2001 From: Tsz-Wo Nicholas Sze Date: Fri, 10 May 2024 13:49:19 -0700 Subject: [PATCH 349/397] Revert "RATIS-2080. Reuse LeaderElection executor. (#1082)" This reverts commit 8c9c801e5b8edc68971911e5819e89e74e114683. (cherry picked from commit 33cb58678b499ecc5816079e5f93bf03debc8b48) --- .../ratis/server/impl/LeaderElection.java | 53 ++++++++++++------- .../ratis/server/impl/RaftServerImpl.java | 24 ++------- 2 files changed, 37 insertions(+), 40 deletions(-) diff --git a/ratis-server/src/main/java/org/apache/ratis/server/impl/LeaderElection.java b/ratis-server/src/main/java/org/apache/ratis/server/impl/LeaderElection.java index acf1b4f3d5..e6c3ce5f0b 100644 --- a/ratis-server/src/main/java/org/apache/ratis/server/impl/LeaderElection.java +++ b/ratis-server/src/main/java/org/apache/ratis/server/impl/LeaderElection.java @@ -43,17 +43,16 @@ import java.util.ArrayList; import java.util.Collection; import java.util.HashMap; -import java.util.Iterator; import java.util.List; import java.util.Map; import java.util.Optional; import java.util.concurrent.Callable; import java.util.concurrent.CompletableFuture; import java.util.concurrent.ExecutionException; +import java.util.concurrent.ExecutorCompletionService; import java.util.concurrent.ExecutorService; import java.util.concurrent.Executors; import java.util.concurrent.Future; -import java.util.concurrent.TimeoutException; import java.util.concurrent.atomic.AtomicInteger; import java.util.Set; import java.util.stream.Collectors; @@ -248,22 +247,28 @@ public String toString() { } static class Executor { + private final ExecutorCompletionService service; private final ExecutorService executor; private final AtomicInteger count = new AtomicInteger(); Executor(Object name, int size) { Preconditions.assertTrue(size > 0); - executor = Executors.newCachedThreadPool(r -> + executor = Executors.newFixedThreadPool(size, r -> Daemon.newBuilder().setName(name + "-" + count.incrementAndGet()).setRunnable(r).build()); + service = new ExecutorCompletionService<>(executor); } - ExecutorService getExecutor() { - return executor; + void shutdown() { + executor.shutdown(); } - Future submit(Callable task) { - return executor.submit(task); + void submit(Callable task) { + service.submit(task); + } + + Future poll(TimeDuration waitTime) throws InterruptedException { + return service.poll(waitTime.getDuration(), waitTime.getUnit()); } } @@ -421,9 +426,13 @@ private ResultAndTerm submitRequestAndWaitResult(Phase phase, RaftConfigurationI r = new ResultAndTerm(Result.PASSED, electionTerm); } else { final TermIndex lastEntry = server.getState().getLastEntry(); - final List> submitted = submitRequests( - phase, electionTerm, lastEntry, others, server.getLeaderElectionExecutor()); - r = waitForResults(phase, electionTerm, submitted, conf); + final Executor voteExecutor = new Executor(this, others.size()); + try { + final int submitted = submitRequests(phase, electionTerm, lastEntry, others, voteExecutor); + r = waitForResults(phase, electionTerm, submitted, conf, voteExecutor); + } finally { + voteExecutor.shutdown(); + } } return r; @@ -473,13 +482,14 @@ private boolean askForVotes(Phase phase, int round) throws InterruptedException, } } - private List> submitRequests(Phase phase, long electionTerm, TermIndex lastEntry, + private int submitRequests(Phase phase, long electionTerm, TermIndex lastEntry, Collection others, Executor voteExecutor) { - final List> submitted = new ArrayList<>(others.size()); + int submitted = 0; for (final RaftPeer peer : others) { final RequestVoteRequestProto r = ServerProtoUtils.toRequestVoteRequestProto( server.getMemberId(), peer.getId(), electionTerm, lastEntry, phase == Phase.PRE_VOTE); - submitted.add(voteExecutor.submit(() -> server.getServerRpc().requestVote(r))); + voteExecutor.submit(() -> server.getServerRpc().requestVote(r)); + submitted++; } return submitted; } @@ -493,17 +503,18 @@ private Set getHigherPriorityPeers(RaftConfiguration conf) { .collect(Collectors.toSet()); } - private ResultAndTerm waitForResults(Phase phase, long electionTerm, List> submitted, - RaftConfigurationImpl conf) throws InterruptedException { + private ResultAndTerm waitForResults(Phase phase, long electionTerm, int submitted, + RaftConfigurationImpl conf, Executor voteExecutor) throws InterruptedException { final Timestamp timeout = Timestamp.currentTime().addTime(server.getRandomElectionTimeout()); final Map responses = new HashMap<>(); final List exceptions = new ArrayList<>(); + int waitForNum = submitted; Collection votedPeers = new ArrayList<>(); Collection rejectedPeers = new ArrayList<>(); Set higherPriorityPeers = getHigherPriorityPeers(conf); final boolean singleMode = conf.isSingleMode(server.getId()); - for(Iterator> i = submitted.iterator(); i.hasNext() && shouldRun(electionTerm); ) { + while (waitForNum > 0 && shouldRun(electionTerm)) { final TimeDuration waitTime = timeout.elapsedTime().apply(n -> -n); if (waitTime.isNonPositive()) { if (conf.hasMajority(votedPeers, server.getId())) { @@ -519,9 +530,12 @@ private ResultAndTerm waitForResults(Phase phase, long electionTerm, List future = i.next(); - final RequestVoteReplyProto r = future.get(waitTime.getDuration(), waitTime.getUnit()); + final Future future = voteExecutor.poll(waitTime); + if (future == null) { + continue; // poll timeout, continue to return Result.TIMEOUT + } + final RequestVoteReplyProto r = future.get(); final RaftPeerId replierId = RaftPeerId.valueOf(r.getServerReply().getReplyId()); final RequestVoteReplyProto previous = responses.putIfAbsent(replierId, r); if (previous != null) { @@ -568,9 +582,8 @@ private ResultAndTerm waitForResults(Phase phase, long electionTerm, List this + " got exception when requesting votes", e); exceptions.add(e); - } catch (TimeoutException e) { - // get timeout, continue to return Result.TIMEOUT } + waitForNum--; } // received all the responses if (conf.hasMajority(votedPeers, server.getId())) { diff --git a/ratis-server/src/main/java/org/apache/ratis/server/impl/RaftServerImpl.java b/ratis-server/src/main/java/org/apache/ratis/server/impl/RaftServerImpl.java index 88f20280fd..7ce3f5feb0 100644 --- a/ratis-server/src/main/java/org/apache/ratis/server/impl/RaftServerImpl.java +++ b/ratis-server/src/main/java/org/apache/ratis/server/impl/RaftServerImpl.java @@ -250,7 +250,6 @@ public long[] getFollowerMatchIndices() { private final ExecutorService serverExecutor; private final ExecutorService clientExecutor; - private final MemoizedSupplier leaderElectionExecutor; private final AtomicBoolean firstElectionSinceStartup = new AtomicBoolean(true); private final ThreadGroup threadGroup; @@ -289,17 +288,14 @@ public long[] getFollowerMatchIndices() { this.appendLogTermIndices = RaftServerConfigKeys.Log.appendEntriesComposeEnabled(properties) ? new NavigableIndices() : null; - final RaftGroupMemberId memberId = getMemberId(); this.serverExecutor = ConcurrentUtils.newThreadPoolWithMax( RaftServerConfigKeys.ThreadPool.serverCached(properties), RaftServerConfigKeys.ThreadPool.serverSize(properties), - memberId + "-server"); + id + "-server"); this.clientExecutor = ConcurrentUtils.newThreadPoolWithMax( RaftServerConfigKeys.ThreadPool.clientCached(properties), RaftServerConfigKeys.ThreadPool.clientSize(properties), - memberId + "-client"); - this.leaderElectionExecutor = MemoizedSupplier.valueOf( - () -> new LeaderElection.Executor(memberId + "-election" , group.getPeers().size())); + id + "-client"); } private long getCommitIndex(RaftPeerId id) { @@ -555,20 +551,12 @@ public void close() { try { ConcurrentUtils.shutdownAndWait(clientExecutor); } catch (Exception e) { - LOG.warn("{}: Failed to shutdown clientExecutor", getMemberId(), e); + LOG.warn(getMemberId() + ": Failed to shutdown clientExecutor", e); } try { ConcurrentUtils.shutdownAndWait(serverExecutor); } catch (Exception e) { - LOG.warn("{}: Failed to shutdown serverExecutor", getMemberId(), e); - } - - if (leaderElectionExecutor.isInitialized()) { - try { - ConcurrentUtils.shutdownAndWait(leaderElectionExecutor.get().getExecutor()); - } catch (Exception e) { - LOG.warn("{}: Failed to shutdown leaderElectionExecutor", getMemberId(), e); - } + LOG.warn(getMemberId() + ": Failed to shutdown serverExecutor", e); } closeFinishedLatch.countDown(); }); @@ -1606,10 +1594,6 @@ ExecutorService getServerExecutor() { return serverExecutor; } - LeaderElection.Executor getLeaderElectionExecutor() { - return leaderElectionExecutor.get(); - } - private CompletableFuture appendEntriesAsync(RaftPeerId leaderId, long callId, TermIndex previous, ReferenceCountedObject requestRef) throws IOException { final AppendEntriesRequestProto proto = requestRef.get(); From 0350055feb84e3e57602c9ddb90c7848eb7377d0 Mon Sep 17 00:00:00 2001 From: Duong Nguyen Date: Sat, 11 May 2024 14:13:34 -0700 Subject: [PATCH 350/397] RATIS-2077. Timedout StateMachine retainRead is released twice (#1081) (cherry picked from commit 7f04919ffe99a24ea4cb226f27f7c97ba58c2417) --- .../ratis/server/leader/LogAppenderBase.java | 25 +++++++++++++------ .../ratis/server/raftlog/RaftLogBase.java | 11 -------- 2 files changed, 18 insertions(+), 18 deletions(-) diff --git a/ratis-server/src/main/java/org/apache/ratis/server/leader/LogAppenderBase.java b/ratis-server/src/main/java/org/apache/ratis/server/leader/LogAppenderBase.java index 7e87825b29..e424be86cc 100644 --- a/ratis-server/src/main/java/org/apache/ratis/server/leader/LogAppenderBase.java +++ b/ratis-server/src/main/java/org/apache/ratis/server/leader/LogAppenderBase.java @@ -42,6 +42,7 @@ import java.util.List; import java.util.Map; import java.util.Objects; +import java.util.Optional; import java.util.concurrent.CompletableFuture; import java.util.concurrent.TimeUnit; import java.util.concurrent.atomic.AtomicBoolean; @@ -269,14 +270,24 @@ protected ReferenceCountedObject nextAppendEntriesReq return null; } - final List protos = buffer.pollList(getHeartbeatWaitTimeMs(), EntryWithData::getEntry, - (entry, time, exception) -> LOG.warn("Failed to get " + entry - + " in " + time.toString(TimeUnit.MILLISECONDS, 3), exception)); - for (EntryWithData entry : buffer) { - // Release remaining entries. - offered.remove(entry.getIndex()).release(); + final List protos; + try { + protos = buffer.pollList(getHeartbeatWaitTimeMs(), EntryWithData::getEntry, + (entry, time, exception) -> LOG.warn("Failed to get {} in {}", + entry, time.toString(TimeUnit.MILLISECONDS, 3), exception)); + } catch (RaftLogIOException e) { + for (ReferenceCountedObject ref : offered.values()) { + ref.release(); + } + offered.clear(); + throw e; + } finally { + for (EntryWithData entry : buffer) { + // Release remaining entries. + Optional.ofNullable(offered.remove(entry.getIndex())).ifPresent(ReferenceCountedObject::release); + } + buffer.clear(); } - buffer.clear(); assertProtos(protos, followerNext, previous, snapshotIndex); AppendEntriesRequestProto appendEntriesProto = leaderState.newAppendEntriesRequestProto(follower, protos, previous, callId); diff --git a/ratis-server/src/main/java/org/apache/ratis/server/raftlog/RaftLogBase.java b/ratis-server/src/main/java/org/apache/ratis/server/raftlog/RaftLogBase.java index cdf501301d..024845fac4 100644 --- a/ratis-server/src/main/java/org/apache/ratis/server/raftlog/RaftLogBase.java +++ b/ratis-server/src/main/java/org/apache/ratis/server/raftlog/RaftLogBase.java @@ -507,13 +507,11 @@ public LogEntryProto getEntry(TimeDuration timeout) throws RaftLogIOException, T if (timeout.compareTo(stateMachineDataReadTimeout) > 0) { getRaftLogMetrics().onStateMachineDataReadTimeout(); } - discardData(); throw t; } catch (Exception e) { if (e instanceof InterruptedException) { Thread.currentThread().interrupt(); } - discardData(); final String err = getName() + ": Failed readStateMachineData for " + this; LOG.error(err, e); throw new RaftLogIOException(err, JavaUtils.unwrapCompletionException(e)); @@ -523,20 +521,11 @@ public LogEntryProto getEntry(TimeDuration timeout) throws RaftLogIOException, T if (LogProtoUtils.isStateMachineDataEmpty(entryProto)) { final String err = getName() + ": State machine data not set for " + this; LOG.error(err); - data.release(); throw new RaftLogIOException(err); } return entryProto; } - private void discardData() { - future.whenComplete((r, ex) -> { - if (r != null) { - r.release(); - } - }); - } - @Override public String toString() { return toLogEntryString(logEntry); From 163cfd94ab3907cd2e79cfd8c09704242e7319ce Mon Sep 17 00:00:00 2001 From: Symious <14933944+symious@users.noreply.github.com> Date: Tue, 14 May 2024 09:15:50 +0800 Subject: [PATCH 351/397] RATIS-2088. Fix name and imports related check in test-directory (#1092) (cherry picked from commit 29bba5943ae9c7b5d538da62de683edc2eb9d8f0) --- .../test/java/org/apache/ratis/BaseTest.java | 13 ++++++++ .../apache/ratis/util/TestRefCountingMap.java | 1 + .../ratis/grpc/MiniRaftClusterWithGrpc.java | 6 +--- .../InstallSnapshotNotificationTests.java | 10 +++---- .../apache/ratis/OutputStreamBaseTest.java | 4 +-- .../ratis/server/impl/MiniRaftCluster.java | 2 -- .../statemachine/RaftSnapshotBaseTest.java | 1 - .../impl/SimpleStateMachine4Testing.java | 1 - .../ratis/server/impl/TestRaftServerJmx.java | 6 ++-- .../raftlog/segmented/TestCacheEviction.java | 30 +++++++++---------- .../segmented/TestSegmentedRaftLog.java | 9 ++---- 11 files changed, 43 insertions(+), 40 deletions(-) diff --git a/ratis-common/src/test/java/org/apache/ratis/BaseTest.java b/ratis-common/src/test/java/org/apache/ratis/BaseTest.java index 52b986ab9b..8773fc101e 100644 --- a/ratis-common/src/test/java/org/apache/ratis/BaseTest.java +++ b/ratis-common/src/test/java/org/apache/ratis/BaseTest.java @@ -99,6 +99,19 @@ public void assertNoFailures() { ExitUtils.assertNotTerminated(); } + // Retained to support junit 4 tests. + @Rule + public final org.junit.rules.Timeout globalTimeout = new org.junit.rules.Timeout( + getGlobalTimeoutSeconds(), TimeUnit.SECONDS ); + + // Retained to support junit 4 tests. + @Rule + public final TestName testName = new TestName(); + + public int getGlobalTimeoutSeconds() { + return 100; + } + private static final Supplier ROOT_TEST_DIR = JavaUtils.memoize( () -> JavaUtils.callAsUnchecked(() -> { final File dir = new File(System.getProperty("test.build.data", "target/test/data"), diff --git a/ratis-common/src/test/java/org/apache/ratis/util/TestRefCountingMap.java b/ratis-common/src/test/java/org/apache/ratis/util/TestRefCountingMap.java index 217d6e3da8..3c3cbdc6ee 100644 --- a/ratis-common/src/test/java/org/apache/ratis/util/TestRefCountingMap.java +++ b/ratis-common/src/test/java/org/apache/ratis/util/TestRefCountingMap.java @@ -26,6 +26,7 @@ import java.util.Collection; import java.util.Set; +import org.apache.ratis.thirdparty.com.google.common.collect.Lists; import org.junit.jupiter.api.BeforeEach; import org.junit.jupiter.api.Test; diff --git a/ratis-grpc/src/test/java/org/apache/ratis/grpc/MiniRaftClusterWithGrpc.java b/ratis-grpc/src/test/java/org/apache/ratis/grpc/MiniRaftClusterWithGrpc.java index 20ac883589..aeee7c050c 100644 --- a/ratis-grpc/src/test/java/org/apache/ratis/grpc/MiniRaftClusterWithGrpc.java +++ b/ratis-grpc/src/test/java/org/apache/ratis/grpc/MiniRaftClusterWithGrpc.java @@ -62,11 +62,7 @@ default Factory getFactory() { } public static final DelayLocalExecutionInjection SEND_SERVER_REQUEST_INJECTION = - new DelayLocalExecutionInjection(GrpcServicesImpl.GRPC_SEND_SERVER_REQUEST); - - public MiniRaftClusterWithGrpc(String[] ids, RaftProperties properties, Parameters parameters) { - this(ids, new String[0], properties, parameters); - } + new DelayLocalExecutionInjection(GrpcService.GRPC_SEND_SERVER_REQUEST); protected MiniRaftClusterWithGrpc(String[] ids, String[] listenerIds, RaftProperties properties, Parameters parameters) { diff --git a/ratis-server/src/test/java/org/apache/ratis/InstallSnapshotNotificationTests.java b/ratis-server/src/test/java/org/apache/ratis/InstallSnapshotNotificationTests.java index 931bf6317f..f4fae7bb06 100644 --- a/ratis-server/src/test/java/org/apache/ratis/InstallSnapshotNotificationTests.java +++ b/ratis-server/src/test/java/org/apache/ratis/InstallSnapshotNotificationTests.java @@ -238,7 +238,7 @@ private void testAddNewFollowers(CLUSTER cluster, int numRequests) throws Except final SnapshotInfo leaderSnapshotInfo = cluster.getLeader().getStateMachine().getLatestSnapshot(); LOG.info("LeaderSnapshotInfo: {}", leaderSnapshotInfo.getTermIndex()); final boolean set = LEADER_SNAPSHOT_INFO_REF.compareAndSet(null, leaderSnapshotInfo); - Assertions.assertTrue(set); + Assert.assertTrue(set); // Add new peer(s) final PeerChanges change = cluster.addNewPeers(1, true); @@ -349,9 +349,9 @@ private void testInstallSnapshotNotificationCount(CLUSTER cluster) throws Except long snapshotIndex = cluster.getLeader().getStateMachine().takeSnapshot(); Assertions.assertEquals(20, snapshotIndex); final SnapshotInfo leaderSnapshotInfo = cluster.getLeader().getStateMachine().getLatestSnapshot(); - Assertions.assertEquals(20, leaderSnapshotInfo.getIndex()); + Assert.assertEquals(20, leaderSnapshotInfo.getIndex()); final boolean set = LEADER_SNAPSHOT_INFO_REF.compareAndSet(null, leaderSnapshotInfo); - Assertions.assertTrue(set); + Assert.assertTrue(set); // Wait for the snapshot to be done. final RaftServer.Division leader = cluster.getLeader(); @@ -472,7 +472,7 @@ private void testInstallSnapshotInstalledEvent(CLUSTER cluster) throws Exception final SnapshotInfo leaderSnapshotInfo = cluster.getLeader().getStateMachine().getLatestSnapshot(); LOG.info("LeaderSnapshotInfo: {}", leaderSnapshotInfo.getTermIndex()); final boolean set = LEADER_SNAPSHOT_INFO_REF.compareAndSet(null, leaderSnapshotInfo); - Assertions.assertTrue(set); + Assert.assertTrue(set); // add one new peer final PeerChanges change = cluster.addNewPeers(1, true); @@ -547,7 +547,7 @@ private void testInstallSnapshotDuringBootstrap(CLUSTER cluster) throws Exceptio final SnapshotInfo leaderSnapshotInfo = cluster.getLeader().getStateMachine().getLatestSnapshot(); final boolean set = LEADER_SNAPSHOT_INFO_REF.compareAndSet(null, leaderSnapshotInfo); - Assertions.assertTrue(set); + Assert.assertTrue(set); // Add new peer(s) final int numNewPeers = 1; diff --git a/ratis-server/src/test/java/org/apache/ratis/OutputStreamBaseTest.java b/ratis-server/src/test/java/org/apache/ratis/OutputStreamBaseTest.java index 9821126ce6..874c953553 100644 --- a/ratis-server/src/test/java/org/apache/ratis/OutputStreamBaseTest.java +++ b/ratis-server/src/test/java/org/apache/ratis/OutputStreamBaseTest.java @@ -26,8 +26,8 @@ import org.apache.ratis.server.raftlog.RaftLog; import org.apache.ratis.util.SizeInBytes; import org.apache.ratis.util.StringUtils; -import org.junit.jupiter.api.Assertions; -import org.junit.jupiter.api.Test; +import org.junit.Assert; +import org.junit.Test; import java.io.IOException; import java.io.OutputStream; diff --git a/ratis-server/src/test/java/org/apache/ratis/server/impl/MiniRaftCluster.java b/ratis-server/src/test/java/org/apache/ratis/server/impl/MiniRaftCluster.java index b47d13e814..bc432c10f4 100644 --- a/ratis-server/src/test/java/org/apache/ratis/server/impl/MiniRaftCluster.java +++ b/ratis-server/src/test/java/org/apache/ratis/server/impl/MiniRaftCluster.java @@ -95,8 +95,6 @@ public abstract class MiniRaftCluster implements Closeable { TimeDuration.valueOf(100, TimeUnit.MILLISECONDS); static final AtomicInteger THREAD_COUNT = new AtomicInteger(0); - static final AtomicInteger CLIENT_ID = new AtomicInteger(0); - public abstract static class Factory { public interface Get { Supplier PROPERTIES = JavaUtils.memoize(RaftProperties::new); diff --git a/ratis-server/src/test/java/org/apache/ratis/statemachine/RaftSnapshotBaseTest.java b/ratis-server/src/test/java/org/apache/ratis/statemachine/RaftSnapshotBaseTest.java index 03c500f5db..b4469b800d 100644 --- a/ratis-server/src/test/java/org/apache/ratis/statemachine/RaftSnapshotBaseTest.java +++ b/ratis-server/src/test/java/org/apache/ratis/statemachine/RaftSnapshotBaseTest.java @@ -45,7 +45,6 @@ import org.apache.ratis.server.raftlog.RaftLog; import org.apache.ratis.server.raftlog.segmented.LogSegmentPath; import org.apache.ratis.proto.RaftProtos.LogEntryProto; -import org.apache.ratis.server.storage.RaftStorageTestUtils; import org.apache.ratis.statemachine.impl.SimpleStateMachine4Testing; import org.apache.ratis.statemachine.impl.SimpleStateMachineStorage; import org.apache.ratis.util.FileUtils; diff --git a/ratis-server/src/test/java/org/apache/ratis/statemachine/impl/SimpleStateMachine4Testing.java b/ratis-server/src/test/java/org/apache/ratis/statemachine/impl/SimpleStateMachine4Testing.java index 57219ce21f..71fbe216ca 100644 --- a/ratis-server/src/test/java/org/apache/ratis/statemachine/impl/SimpleStateMachine4Testing.java +++ b/ratis-server/src/test/java/org/apache/ratis/statemachine/impl/SimpleStateMachine4Testing.java @@ -42,7 +42,6 @@ import org.apache.ratis.statemachine.StateMachine; import org.apache.ratis.statemachine.TransactionContext; import org.apache.ratis.thirdparty.com.google.protobuf.ByteString; -import org.apache.ratis.thirdparty.com.google.protobuf.InvalidProtocolBufferException; import org.apache.ratis.util.Daemon; import org.apache.ratis.util.JavaUtils; import org.apache.ratis.util.LifeCycle; diff --git a/ratis-test/src/test/java/org/apache/ratis/server/impl/TestRaftServerJmx.java b/ratis-test/src/test/java/org/apache/ratis/server/impl/TestRaftServerJmx.java index 4221d85cfe..b24ec72c06 100644 --- a/ratis-test/src/test/java/org/apache/ratis/server/impl/TestRaftServerJmx.java +++ b/ratis-test/src/test/java/org/apache/ratis/server/impl/TestRaftServerJmx.java @@ -40,7 +40,7 @@ public class TestRaftServerJmx extends BaseTest { @Test - @Timeout(value = 30000) + @Timeout(value = 30) public void testJmxBeans() throws Exception { final int numServers = 3; final MiniRaftClusterWithSimulatedRpc cluster @@ -50,7 +50,7 @@ public void testJmxBeans() throws Exception { MBeanServer platformMBeanServer = ManagementFactory.getPlatformMBeanServer(); Set objectInstances = platformMBeanServer.queryMBeans(new ObjectName("Ratis:*"), null); - Assertions.assertEquals(NUM_SERVERS, objectInstances.size()); + Assertions.assertEquals(numServers, objectInstances.size()); for (ObjectInstance instance : objectInstances) { Object groupId = platformMBeanServer.getAttribute(instance.getObjectName(), "GroupId"); @@ -60,7 +60,7 @@ public void testJmxBeans() throws Exception { } @Test - @Timeout(value = 30000) + @Timeout(value = 30) public void testRegister() throws JMException { { final JmxRegister jmx = new JmxRegister(); diff --git a/ratis-test/src/test/java/org/apache/ratis/server/raftlog/segmented/TestCacheEviction.java b/ratis-test/src/test/java/org/apache/ratis/server/raftlog/segmented/TestCacheEviction.java index e1afe4c273..163c25da90 100644 --- a/ratis-test/src/test/java/org/apache/ratis/server/raftlog/segmented/TestCacheEviction.java +++ b/ratis-test/src/test/java/org/apache/ratis/server/raftlog/segmented/TestCacheEviction.java @@ -38,7 +38,6 @@ import org.apache.ratis.statemachine.impl.SimpleStateMachine4Testing; import org.apache.ratis.statemachine.StateMachine; import org.apache.ratis.util.JavaUtils; -import org.apache.ratis.util.ReferenceCountedObject; import org.apache.ratis.util.SizeInBytes; import org.junit.jupiter.api.Assertions; import org.junit.jupiter.api.Test; @@ -52,6 +51,7 @@ import static org.apache.ratis.server.raftlog.segmented.SegmentedRaftLogTestUtils.MAX_OP_SIZE; +@SuppressWarnings({"deprecation"}) public class TestCacheEviction extends BaseTest { private static final CacheInvalidationPolicy POLICY = new CacheInvalidationPolicyDefault(); @@ -77,35 +77,35 @@ public void testBasicEviction() throws Exception { new boolean[]{true, true, true, true, true}, 0, 10); // case 1, make sure we do not evict cache for segments behind local flushed index - List evicted = policy.evict(null, 5, 15, segments, maxCached); + List evicted = POLICY.evict(null, 5, 15, segments, maxCached); Assertions.assertEquals(0, evicted.size()); // case 2, suppose the local flushed index is in the 3rd segment, then we // can evict the first two segment - evicted = policy.evict(null, 25, 30, segments, maxCached); + evicted = POLICY.evict(null, 25, 30, segments, maxCached); Assertions.assertEquals(2, evicted.size()); Assertions.assertSame(evicted.get(0), segments.get(0)); Assertions.assertSame(evicted.get(1), segments.get(1)); // case 3, similar with case 2, but the local applied index is less than // the local flushed index. - evicted = policy.evict(null, 25, 15, segments, maxCached); + evicted = POLICY.evict(null, 25, 15, segments, maxCached); Assertions.assertEquals(1, evicted.size()); Assertions.assertSame(evicted.get(0), segments.get(0)); // case 4, the local applied index is very small, then evict cache behind it // first and let the state machine load the segments later - evicted = policy.evict(null, 35, 5, segments, maxCached); + evicted = POLICY.evict(null, 35, 5, segments, maxCached); Assertions.assertEquals(1, evicted.size()); Assertions.assertSame(evicted.get(0), segments.get(2)); Mockito.when(segments.get(2).hasCache()).thenReturn(false); - evicted = policy.evict(null, 35, 5, segments, maxCached); + evicted = POLICY.evict(null, 35, 5, segments, maxCached); Assertions.assertEquals(1, evicted.size()); Assertions.assertSame(evicted.get(0), segments.get(1)); Mockito.when(segments.get(1).hasCache()).thenReturn(false); - evicted = policy.evict(null, 35, 5, segments, maxCached); + evicted = POLICY.evict(null, 35, 5, segments, maxCached); Assertions.assertEquals(0, evicted.size()); } @@ -122,33 +122,33 @@ public void testEvictionWithFollowerIndices() throws Exception { Assertions.assertEquals(0, evicted.size()); // case 2, the follower indices are behind the local flushed index - evicted = policy.evict(new long[]{30, 40, 45}, 25, 30, segments, maxCached); + evicted = POLICY.evict(new long[]{30, 40, 45}, 25, 30, segments, maxCached); Assertions.assertEquals(2, evicted.size()); Assertions.assertSame(evicted.get(0), segments.get(0)); Assertions.assertSame(evicted.get(1), segments.get(1)); // case 3, similar with case 3 in basic eviction test - evicted = policy.evict(new long[]{30, 40, 45}, 25, 15, segments, maxCached); + evicted = POLICY.evict(new long[]{30, 40, 45}, 25, 15, segments, maxCached); Assertions.assertEquals(1, evicted.size()); Assertions.assertSame(evicted.get(0), segments.get(0)); // case 4, the followers are slower than local flush - evicted = policy.evict(new long[]{15, 45, 45}, 55, 50, segments, maxCached); + evicted = POLICY.evict(new long[]{15, 45, 45}, 55, 50, segments, maxCached); Assertions.assertEquals(1, evicted.size()); Assertions.assertSame(evicted.get(0), segments.get(0)); Mockito.when(segments.get(0).hasCache()).thenReturn(false); - evicted = policy.evict(new long[]{15, 45, 45}, 55, 50, segments, maxCached); + evicted = POLICY.evict(new long[]{15, 45, 45}, 55, 50, segments, maxCached); Assertions.assertEquals(1, evicted.size()); Assertions.assertSame(evicted.get(0), segments.get(2)); Mockito.when(segments.get(2).hasCache()).thenReturn(false); - evicted = policy.evict(new long[]{15, 45, 45}, 55, 50, segments, maxCached); + evicted = POLICY.evict(new long[]{15, 45, 45}, 55, 50, segments, maxCached); Assertions.assertEquals(1, evicted.size()); Assertions.assertSame(evicted.get(0), segments.get(3)); Mockito.when(segments.get(3).hasCache()).thenReturn(false); - evicted = policy.evict(new long[]{15, 45, 45}, 55, 50, segments, maxCached); + evicted = POLICY.evict(new long[]{15, 45, 45}, 55, 50, segments, maxCached); Assertions.assertEquals(0, evicted.size()); } @@ -175,7 +175,7 @@ public void testEvictionInSegmentedLog() throws Exception { raftLog.open(RaftLog.INVALID_LOG_INDEX, null); List slist = TestSegmentedRaftLog.prepareRanges(0, maxCachedNum, 7, 0); List entries = generateEntries(slist); - raftLog.append(ReferenceCountedObject.wrap(entries)).forEach(CompletableFuture::join); + raftLog.append(entries).forEach(CompletableFuture::join); // check the current cached segment number: the last segment is still open Assertions.assertEquals(maxCachedNum - 1, @@ -185,7 +185,7 @@ public void testEvictionInSegmentedLog() throws Exception { Mockito.when(info.getFollowerNextIndices()).thenReturn(new long[]{21, 40, 40}); slist = TestSegmentedRaftLog.prepareRanges(maxCachedNum, maxCachedNum + 2, 7, 7 * maxCachedNum); entries = generateEntries(slist); - raftLog.append(ReferenceCountedObject.wrap(entries)).forEach(CompletableFuture::join); + raftLog.append(entries).forEach(CompletableFuture::join); // check the cached segment number again. since the slowest follower is on // index 21, the eviction should happen and evict 3 segments diff --git a/ratis-test/src/test/java/org/apache/ratis/server/raftlog/segmented/TestSegmentedRaftLog.java b/ratis-test/src/test/java/org/apache/ratis/server/raftlog/segmented/TestSegmentedRaftLog.java index 4e53fccd69..d22ed7f3b7 100644 --- a/ratis-test/src/test/java/org/apache/ratis/server/raftlog/segmented/TestSegmentedRaftLog.java +++ b/ratis-test/src/test/java/org/apache/ratis/server/raftlog/segmented/TestSegmentedRaftLog.java @@ -577,8 +577,7 @@ public void testAppendEntriesWithInconsistency(Boolean useAsyncFlush, Boolean sm List entries = prepareLogEntries(ranges, null); final RetryCache retryCache = RetryCacheTestUtil.createRetryCache(); - try (SegmentedRaftLog raftLog = - RetryCacheTestUtil.newSegmentedRaftLog(MEMBER_ID, retryCache, storage, properties)) { + try (SegmentedRaftLog raftLog = RetryCacheTestUtil.newSegmentedRaftLog(MEMBER_ID, retryCache, storage, properties)) { raftLog.open(RaftLog.INVALID_LOG_INDEX, null); entries.forEach(entry -> RetryCacheTestUtil.createEntry(retryCache, entry)); // append entries to the raftlog @@ -593,8 +592,7 @@ public void testAppendEntriesWithInconsistency(Boolean useAsyncFlush, Boolean sm List newEntries = prepareLogEntries( Arrays.asList(r1, r2, r3), null); - try (SegmentedRaftLog raftLog = - RetryCacheTestUtil.newSegmentedRaftLog(MEMBER_ID, retryCache, storage, properties)) { + try (SegmentedRaftLog raftLog = RetryCacheTestUtil.newSegmentedRaftLog(MEMBER_ID, retryCache, storage, properties)) { raftLog.open(RaftLog.INVALID_LOG_INDEX, null); LOG.info("newEntries[0] = {}", newEntries.get(0)); final int last = newEntries.size() - 1; @@ -611,8 +609,7 @@ public void testAppendEntriesWithInconsistency(Boolean useAsyncFlush, Boolean sm } // load the raftlog again and check - try (SegmentedRaftLog raftLog = - RetryCacheTestUtil.newSegmentedRaftLog(MEMBER_ID, retryCache, storage, properties)) { + try (SegmentedRaftLog raftLog = RetryCacheTestUtil.newSegmentedRaftLog(MEMBER_ID, retryCache, storage, properties)) { raftLog.open(RaftLog.INVALID_LOG_INDEX, null); checkEntries(raftLog, entries, 0, 650); checkEntries(raftLog, newEntries, 100, 100); From 5a88741a9a54f6ba585c3a5147043f44b05fd2b3 Mon Sep 17 00:00:00 2001 From: Duong Nguyen Date: Thu, 23 May 2024 00:29:45 -0700 Subject: [PATCH 352/397] RATIS-2092. Add metrics to expose number of zero-copy unclosed messages (#1095) (cherry picked from commit 53d5af75a2d86efed78931028a504cecd16018f5) --- .../ratis/grpc/metrics/ZeroCopyMetrics.java | 6 +++++ .../server/GrpcClientProtocolService.java | 1 + .../server/GrpcServerProtocolService.java | 1 + .../grpc/util/ZeroCopyMessageMarshaller.java | 23 ++----------------- 4 files changed, 10 insertions(+), 21 deletions(-) diff --git a/ratis-grpc/src/main/java/org/apache/ratis/grpc/metrics/ZeroCopyMetrics.java b/ratis-grpc/src/main/java/org/apache/ratis/grpc/metrics/ZeroCopyMetrics.java index fec2135a98..1fcc317f9d 100644 --- a/ratis-grpc/src/main/java/org/apache/ratis/grpc/metrics/ZeroCopyMetrics.java +++ b/ratis-grpc/src/main/java/org/apache/ratis/grpc/metrics/ZeroCopyMetrics.java @@ -24,6 +24,8 @@ import org.apache.ratis.thirdparty.com.google.common.annotations.VisibleForTesting; import org.apache.ratis.thirdparty.com.google.protobuf.AbstractMessage; +import java.util.function.Supplier; + public class ZeroCopyMetrics extends RatisMetrics { private static final String RATIS_GRPC_METRICS_APP_NAME = "ratis_grpc"; private static final String RATIS_GRPC_METRICS_COMP_NAME = "zero_copy"; @@ -43,6 +45,10 @@ private static RatisMetricRegistry createRegistry() { RATIS_GRPC_METRICS_COMP_NAME, RATIS_GRPC_METRICS_DESC)); } + public void addUnreleased(String name, Supplier unreleased) { + getRegistry().gauge(name + "_num_unreleased_messages", () -> unreleased); + } + public void onZeroCopyMessage(AbstractMessage ignored) { zeroCopyMessages.inc(); diff --git a/ratis-grpc/src/main/java/org/apache/ratis/grpc/server/GrpcClientProtocolService.java b/ratis-grpc/src/main/java/org/apache/ratis/grpc/server/GrpcClientProtocolService.java index 97b0118909..d1ef99c74c 100644 --- a/ratis-grpc/src/main/java/org/apache/ratis/grpc/server/GrpcClientProtocolService.java +++ b/ratis-grpc/src/main/java/org/apache/ratis/grpc/server/GrpcClientProtocolService.java @@ -161,6 +161,7 @@ void closeAllExisting(RaftGroupId groupId) { this.executor = executor; this.zeroCopyRequestMarshaller = new ZeroCopyMessageMarshaller<>(RaftClientRequestProto.getDefaultInstance(), zeroCopyMetrics::onZeroCopyMessage, zeroCopyMetrics::onNonZeroCopyMessage, zeroCopyMetrics::onReleasedMessage); + zeroCopyMetrics.addUnreleased("client_protocol", zeroCopyRequestMarshaller::getUnclosedCount); } RaftPeerId getId() { diff --git a/ratis-grpc/src/main/java/org/apache/ratis/grpc/server/GrpcServerProtocolService.java b/ratis-grpc/src/main/java/org/apache/ratis/grpc/server/GrpcServerProtocolService.java index 867230572c..19ca6c3477 100644 --- a/ratis-grpc/src/main/java/org/apache/ratis/grpc/server/GrpcServerProtocolService.java +++ b/ratis-grpc/src/main/java/org/apache/ratis/grpc/server/GrpcServerProtocolService.java @@ -250,6 +250,7 @@ private void releaseLast() { this.server = server; this.zeroCopyRequestMarshaller = new ZeroCopyMessageMarshaller<>(AppendEntriesRequestProto.getDefaultInstance(), zeroCopyMetrics::onZeroCopyMessage, zeroCopyMetrics::onNonZeroCopyMessage, zeroCopyMetrics::onReleasedMessage); + zeroCopyMetrics.addUnreleased("server_protocol", zeroCopyRequestMarshaller::getUnclosedCount); } RaftPeerId getId() { diff --git a/ratis-grpc/src/main/java/org/apache/ratis/grpc/util/ZeroCopyMessageMarshaller.java b/ratis-grpc/src/main/java/org/apache/ratis/grpc/util/ZeroCopyMessageMarshaller.java index 385ac592fe..3cdbc07c71 100644 --- a/ratis-grpc/src/main/java/org/apache/ratis/grpc/util/ZeroCopyMessageMarshaller.java +++ b/ratis-grpc/src/main/java/org/apache/ratis/grpc/util/ZeroCopyMessageMarshaller.java @@ -227,26 +227,7 @@ public InputStream popStream(T message) { return unclosedStreams.remove(message); } - void assertNoUnclosedStreams() { - // Intended for tests/teardown to fail fast if callers forgot to release streams. - final int size = unclosedStreams.size(); - Preconditions.assertTrue(size == 0, () -> name + ": " + size + " unclosed stream(s)"); - } - - public void close() { - // Cleanup helper for tests/teardown; do not call while streams may still be in use. - synchronized (unclosedStreams) { - if (unclosedStreams.isEmpty()) { - return; - } - for (InputStream stream : unclosedStreams.values()) { - try { - stream.close(); - } catch (IOException e) { - LOG.warn("{}: Failed to close leaked stream.", name, e); - } - } - unclosedStreams.clear(); - } + public int getUnclosedCount() { + return unclosedStreams.size(); } } From 0193fbcbf3f29c3105c374d052a7743a09d6d554 Mon Sep 17 00:00:00 2001 From: Duong Nguyen Date: Fri, 24 May 2024 12:06:10 -0700 Subject: [PATCH 353/397] RATIS-2096. Add a conf to enable/disable zero copy. (#1099) (cherry picked from commit 26385f31a0bd5c70390dd99a9a2c6ad17ba76414) --- .../org/apache/ratis/grpc/GrpcConfigKeys.java | 31 ++++--------------- .../server/GrpcClientProtocolService.java | 8 ++++- .../server/GrpcServerProtocolService.java | 9 +++++- .../apache/ratis/grpc/server/GrpcService.java | 10 +++--- .../ratis/grpc/MiniRaftClusterWithGrpc.java | 2 ++ 5 files changed, 29 insertions(+), 31 deletions(-) diff --git a/ratis-grpc/src/main/java/org/apache/ratis/grpc/GrpcConfigKeys.java b/ratis-grpc/src/main/java/org/apache/ratis/grpc/GrpcConfigKeys.java index f31794ac36..e495c4c5c5 100644 --- a/ratis-grpc/src/main/java/org/apache/ratis/grpc/GrpcConfigKeys.java +++ b/ratis-grpc/src/main/java/org/apache/ratis/grpc/GrpcConfigKeys.java @@ -264,32 +264,13 @@ static void setLogMessageBatchDuration(RaftProperties properties, LOG_MESSAGE_BATCH_DURATION_KEY, logMessageBatchDuration); } - String SERVICES_CUSTOMIZER_PARAMETER = PREFIX + ".services.customizer"; - Class SERVICES_CUSTOMIZER_CLASS = GrpcServices.Customizer.class; - static GrpcServices.Customizer servicesCustomizer(Parameters parameters) { - return parameters == null ? null - : parameters.get(SERVICES_CUSTOMIZER_PARAMETER, SERVICES_CUSTOMIZER_CLASS); + String ZERO_COPY_ENABLED_KEY = PREFIX + ".zerocopy.enabled"; + boolean ZERO_COPY_ENABLED_DEFAULT = false; + static boolean zeroCopyEnabled(RaftProperties properties) { + return getBoolean(properties::getBoolean, ZERO_COPY_ENABLED_KEY, ZERO_COPY_ENABLED_DEFAULT, getDefaultLog()); } - static void setServicesCustomizer(Parameters parameters, GrpcServices.Customizer customizer) { - parameters.put(SERVICES_CUSTOMIZER_PARAMETER, customizer, SERVICES_CUSTOMIZER_CLASS); - } - - String TLS_CONF_PARAMETER = PREFIX + ".tls.conf"; - Class TLS_CONF_CLASS = TLS.CONF_CLASS; - static GrpcTlsConfig tlsConf(Parameters parameters) { - return parameters != null ? parameters.get(TLS_CONF_PARAMETER, TLS_CONF_CLASS): null; - } - static void setTlsConf(Parameters parameters, GrpcTlsConfig conf) { - parameters.put(TLS_CONF_PARAMETER, conf, TLS_CONF_CLASS); - } - - String STUB_POOL_SIZE_KEY = PREFIX + ".stub.pool.size"; - int STUB_POOL_SIZE_DEFAULT = 1; - static int stubPoolSize(RaftProperties properties) { - return get(properties::getInt, STUB_POOL_SIZE_KEY, STUB_POOL_SIZE_DEFAULT, getDefaultLog()); - } - static void setStubPoolSize(RaftProperties properties, int size) { - setInt(properties::setInt, STUB_POOL_SIZE_KEY, size); + static void setZeroCopyEnabled(RaftProperties properties, boolean enabled) { + setBoolean(properties::setBoolean, ZERO_COPY_ENABLED_KEY, enabled); } } diff --git a/ratis-grpc/src/main/java/org/apache/ratis/grpc/server/GrpcClientProtocolService.java b/ratis-grpc/src/main/java/org/apache/ratis/grpc/server/GrpcClientProtocolService.java index d1ef99c74c..80a9a439b9 100644 --- a/ratis-grpc/src/main/java/org/apache/ratis/grpc/server/GrpcClientProtocolService.java +++ b/ratis-grpc/src/main/java/org/apache/ratis/grpc/server/GrpcClientProtocolService.java @@ -152,13 +152,15 @@ void closeAllExisting(RaftGroupId groupId) { private final ExecutorService executor; private final OrderedStreamObservers orderedStreamObservers = new OrderedStreamObservers(); + private final boolean zeroCopyEnabled; private final ZeroCopyMessageMarshaller zeroCopyRequestMarshaller; GrpcClientProtocolService(Supplier idSupplier, RaftClientAsynchronousProtocol protocol, - ExecutorService executor, ZeroCopyMetrics zeroCopyMetrics) { + ExecutorService executor, boolean zeroCopyEnabled, ZeroCopyMetrics zeroCopyMetrics) { this.idSupplier = idSupplier; this.protocol = protocol; this.executor = executor; + this.zeroCopyEnabled = zeroCopyEnabled; this.zeroCopyRequestMarshaller = new ZeroCopyMessageMarshaller<>(RaftClientRequestProto.getDefaultInstance(), zeroCopyMetrics::onZeroCopyMessage, zeroCopyMetrics::onNonZeroCopyMessage, zeroCopyMetrics::onReleasedMessage); zeroCopyMetrics.addUnreleased("client_protocol", zeroCopyRequestMarshaller::getUnclosedCount); @@ -170,6 +172,10 @@ RaftPeerId getId() { ServerServiceDefinition bindServiceWithZeroCopy() { ServerServiceDefinition orig = super.bindService(); + if (!zeroCopyEnabled) { + LOG.info("{}: Zero copy is disabled.", getId()); + return orig; + } ServerServiceDefinition.Builder builder = ServerServiceDefinition.builder(orig.getServiceDescriptor().getName()); addMethodWithCustomMarshaller(orig, builder, getOrderedMethod(), zeroCopyRequestMarshaller); diff --git a/ratis-grpc/src/main/java/org/apache/ratis/grpc/server/GrpcServerProtocolService.java b/ratis-grpc/src/main/java/org/apache/ratis/grpc/server/GrpcServerProtocolService.java index 19ca6c3477..a02582ff4c 100644 --- a/ratis-grpc/src/main/java/org/apache/ratis/grpc/server/GrpcServerProtocolService.java +++ b/ratis-grpc/src/main/java/org/apache/ratis/grpc/server/GrpcServerProtocolService.java @@ -243,11 +243,14 @@ private void releaseLast() { private final Supplier idSupplier; private final RaftServer server; + private final boolean zeroCopyEnabled; private final ZeroCopyMessageMarshaller zeroCopyRequestMarshaller; - GrpcServerProtocolService(Supplier idSupplier, RaftServer server, ZeroCopyMetrics zeroCopyMetrics) { + GrpcServerProtocolService(Supplier idSupplier, RaftServer server, boolean zeroCopyEnabled, + ZeroCopyMetrics zeroCopyMetrics) { this.idSupplier = idSupplier; this.server = server; + this.zeroCopyEnabled = zeroCopyEnabled; this.zeroCopyRequestMarshaller = new ZeroCopyMessageMarshaller<>(AppendEntriesRequestProto.getDefaultInstance(), zeroCopyMetrics::onZeroCopyMessage, zeroCopyMetrics::onNonZeroCopyMessage, zeroCopyMetrics::onReleasedMessage); zeroCopyMetrics.addUnreleased("server_protocol", zeroCopyRequestMarshaller::getUnclosedCount); @@ -259,6 +262,10 @@ RaftPeerId getId() { ServerServiceDefinition bindServiceWithZeroCopy() { ServerServiceDefinition orig = super.bindService(); + if (!zeroCopyEnabled) { + LOG.info("{}: Zero copy is disabled.", getId()); + return orig; + } ServerServiceDefinition.Builder builder = ServerServiceDefinition.builder(orig.getServiceDescriptor().getName()); // Add appendEntries with zero copy marshaller. diff --git a/ratis-grpc/src/main/java/org/apache/ratis/grpc/server/GrpcService.java b/ratis-grpc/src/main/java/org/apache/ratis/grpc/server/GrpcService.java index fa93586040..e3c0a5eddb 100644 --- a/ratis-grpc/src/main/java/org/apache/ratis/grpc/server/GrpcService.java +++ b/ratis-grpc/src/main/java/org/apache/ratis/grpc/server/GrpcService.java @@ -177,7 +177,8 @@ private GrpcService(RaftServer server, RaftServerConfigKeys.Log.Appender.bufferByteLimit(server.getProperties()), GrpcConfigKeys.flowControlWindow(server.getProperties(), LOG::info), RaftServerConfigKeys.Rpc.requestTimeout(server.getProperties()), - GrpcConfigKeys.Server.heartbeatChannel(server.getProperties())); + GrpcConfigKeys.Server.heartbeatChannel(server.getProperties()), + GrpcConfigKeys.Server.zeroCopyEnabled(server.getProperties())); } @SuppressWarnings("checkstyle:ParameterNumber") // private constructor @@ -187,7 +188,7 @@ private GrpcService(RaftServer raftServer, Supplier idSupplier, String serverHost, int serverPort, GrpcTlsConfig serverTlsConfig, SizeInBytes grpcMessageSizeMax, SizeInBytes appenderBufferSize, SizeInBytes flowControlWindow,TimeDuration requestTimeoutDuration, - boolean useSeparateHBChannel) { + boolean useSeparateHBChannel, boolean zeroCopyEnabled) { super(idSupplier, id -> new PeerProxyMap<>(id.toString(), p -> new GrpcServerProtocolClient(p, flowControlWindow.getSizeInt(), requestTimeoutDuration, serverTlsConfig, useSeparateHBChannel))); @@ -203,7 +204,8 @@ private GrpcService(RaftServer raftServer, Supplier idSupplier, GrpcConfigKeys.Server.asyncRequestThreadPoolSize(properties), getId() + "-request-"); this.zeroCopyMetrics = new ZeroCopyMetrics(); - this.clientProtocolService = new GrpcClientProtocolService(idSupplier, raftServer, executor, zeroCopyMetrics); + this.clientProtocolService = new GrpcClientProtocolService(idSupplier, raftServer, executor, + zeroCopyEnabled, zeroCopyMetrics); this.serverInterceptor = new MetricServerInterceptor( idSupplier, @@ -216,7 +218,7 @@ private GrpcService(RaftServer raftServer, Supplier idSupplier, final NettyServerBuilder serverBuilder = startBuildingNettyServer(serverHost, serverPort, serverTlsConfig, grpcMessageSizeMax, flowControlWindow); GrpcServerProtocolService serverProtocolService = new GrpcServerProtocolService(idSupplier, raftServer, - zeroCopyMetrics); + zeroCopyEnabled, zeroCopyMetrics); serverBuilder.addService(ServerInterceptors.intercept( serverProtocolService.bindServiceWithZeroCopy(), serverInterceptor)); if (!separateAdminServer) { diff --git a/ratis-grpc/src/test/java/org/apache/ratis/grpc/MiniRaftClusterWithGrpc.java b/ratis-grpc/src/test/java/org/apache/ratis/grpc/MiniRaftClusterWithGrpc.java index aeee7c050c..bd3b137501 100644 --- a/ratis-grpc/src/test/java/org/apache/ratis/grpc/MiniRaftClusterWithGrpc.java +++ b/ratis-grpc/src/test/java/org/apache/ratis/grpc/MiniRaftClusterWithGrpc.java @@ -76,6 +76,8 @@ protected Parameters setPropertiesAndInitParameters(RaftPeerId id, RaftGroup gro GrpcConfigKeys.Client.setPort(properties, NetUtils.createSocketAddr(address).getPort())); Optional.ofNullable(getAddress(id, group, RaftPeer::getAdminAddress)).ifPresent(address -> GrpcConfigKeys.Admin.setPort(properties, NetUtils.createSocketAddr(address).getPort())); + // Always run grpc integration tests with zero-copy enabled because the path of nonzero-copy is not risky. + GrpcConfigKeys.Server.setZeroCopyEnabled(properties, true); return parameters; } From 76783195684480c82073ad541049c73cb0927c34 Mon Sep 17 00:00:00 2001 From: Duong Nguyen Date: Fri, 24 May 2024 13:08:55 -0700 Subject: [PATCH 354/397] RATIS-2093. Decouple metadata and configuration entries from appendEntries buffer for stateMachineCache' (#1096) (cherry picked from commit e0025875ecf81ce4b84986147b023822eb14a4ee) --- .../server/raftlog/segmented/SegmentedRaftLog.java | 14 +++++++++----- 1 file changed, 9 insertions(+), 5 deletions(-) diff --git a/ratis-server/src/main/java/org/apache/ratis/server/raftlog/segmented/SegmentedRaftLog.java b/ratis-server/src/main/java/org/apache/ratis/server/raftlog/segmented/SegmentedRaftLog.java index e3f2d96199..955ed851d1 100644 --- a/ratis-server/src/main/java/org/apache/ratis/server/raftlog/segmented/SegmentedRaftLog.java +++ b/ratis-server/src/main/java/org/apache/ratis/server/raftlog/segmented/SegmentedRaftLog.java @@ -461,13 +461,17 @@ protected CompletableFuture appendEntryImpl(ReferenceCountedObject appendEntryTimerContext.stop()); return writeFuture; From d0b88a8255665549a1902ae640377eb569bb68f7 Mon Sep 17 00:00:00 2001 From: Duong Nguyen Date: Mon, 3 Jun 2024 10:35:07 -0700 Subject: [PATCH 355/397] RATIS-2094. Avoid corruptions from TransactionContext's stateMachineLogEntry and stateMachineContext. (#1106) (cherry picked from commit a2bdd10fccb346d02f12d3961d6a5b2d281c5d56) --- .../ratis/util/ReferenceCountedObject.java | 32 +++++++++++++++++++ .../filestore/FileStoreStateMachine.java | 12 +++---- .../statemachine/TransactionContext.java | 3 ++ .../ratis/statemachine/TestStateMachine.java | 3 +- 4 files changed, 43 insertions(+), 7 deletions(-) diff --git a/ratis-common/src/main/java/org/apache/ratis/util/ReferenceCountedObject.java b/ratis-common/src/main/java/org/apache/ratis/util/ReferenceCountedObject.java index c3851a712d..b2c53182d3 100644 --- a/ratis-common/src/main/java/org/apache/ratis/util/ReferenceCountedObject.java +++ b/ratis-common/src/main/java/org/apache/ratis/util/ReferenceCountedObject.java @@ -23,6 +23,7 @@ import java.util.Objects; import java.util.concurrent.atomic.AtomicBoolean; import java.util.function.Consumer; +import java.util.function.Function; /** * A reference-counted object can be retained for later use @@ -128,6 +129,37 @@ public boolean release() { }; } + /** + * @return a {@link ReferenceCountedObject} of the given value by delegating to this object. + */ + default ReferenceCountedObject delegate(V value) { + final ReferenceCountedObject delegated = this; + return new ReferenceCountedObject() { + @Override + public V get() { + return value; + } + + @Override + public V retain() { + delegated.retain(); + return value; + } + + @Override + public boolean release() { + return delegated.release(); + } + }; + } + + /** + * @return a {@link ReferenceCountedObject} by apply the given function to this object. + */ + default ReferenceCountedObject apply(Function function) { + return delegate(function.apply(get())); + } + /** * Wrap the given value as a {@link ReferenceCountedObject}. * diff --git a/ratis-examples/src/main/java/org/apache/ratis/examples/filestore/FileStoreStateMachine.java b/ratis-examples/src/main/java/org/apache/ratis/examples/filestore/FileStoreStateMachine.java index 5ccebe8e12..345831dfb2 100644 --- a/ratis-examples/src/main/java/org/apache/ratis/examples/filestore/FileStoreStateMachine.java +++ b/ratis-examples/src/main/java/org/apache/ratis/examples/filestore/FileStoreStateMachine.java @@ -40,7 +40,6 @@ import org.apache.ratis.thirdparty.com.google.protobuf.ByteString; import org.apache.ratis.thirdparty.com.google.protobuf.InvalidProtocolBufferException; import org.apache.ratis.util.FileUtils; -import org.apache.ratis.util.JavaUtils; import org.apache.ratis.util.ReferenceCountedObject; import java.io.IOException; @@ -114,11 +113,12 @@ public TransactionContext startTransaction(RaftClientRequest request) throws IOE @Override public TransactionContext startTransaction(LogEntryProto entry, RaftProtos.RaftPeerRole role) { + ByteString copied = ByteString.copyFrom(entry.getStateMachineLogEntry().getLogData().asReadOnlyByteBuffer()); return TransactionContext.newBuilder() .setStateMachine(this) .setLogEntry(entry) .setServerRole(role) - .setStateMachineContext(getProto(entry)) + .setStateMachineContext(getProto(copied)) .build(); } @@ -146,14 +146,14 @@ static FileStoreRequestProto getProto(TransactionContext context, LogEntryProto return proto; } } - return getProto(entry); + return getProto(entry.getStateMachineLogEntry().getLogData()); } - static FileStoreRequestProto getProto(LogEntryProto entry) { + static FileStoreRequestProto getProto(ByteString bytes) { try { - return FileStoreRequestProto.parseFrom(entry.getStateMachineLogEntry().getLogData()); + return FileStoreRequestProto.parseFrom(bytes); } catch (InvalidProtocolBufferException e) { - throw new IllegalArgumentException("Failed to parse data, entry=" + entry, e); + throw new IllegalArgumentException("Failed to parse data", e); } } diff --git a/ratis-server-api/src/main/java/org/apache/ratis/statemachine/TransactionContext.java b/ratis-server-api/src/main/java/org/apache/ratis/statemachine/TransactionContext.java index b9e70e2a59..1a33bcaa59 100644 --- a/ratis-server-api/src/main/java/org/apache/ratis/statemachine/TransactionContext.java +++ b/ratis-server-api/src/main/java/org/apache/ratis/statemachine/TransactionContext.java @@ -59,7 +59,10 @@ public interface TransactionContext { /** * Returns the data from the {@link StateMachine} * @return the data from the {@link StateMachine} + * @deprecated access StateMachineLogEntry via {@link TransactionContext#getLogEntryRef()} or + * {@link TransactionContext#getLogEntryUnsafe()} */ + @Deprecated StateMachineLogEntryProto getStateMachineLogEntry(); /** Set exception in case of failure. */ diff --git a/ratis-test/src/test/java/org/apache/ratis/statemachine/TestStateMachine.java b/ratis-test/src/test/java/org/apache/ratis/statemachine/TestStateMachine.java index 07ea4edbcb..79c204e292 100644 --- a/ratis-test/src/test/java/org/apache/ratis/statemachine/TestStateMachine.java +++ b/ratis-test/src/test/java/org/apache/ratis/statemachine/TestStateMachine.java @@ -57,6 +57,7 @@ /** * Test StateMachine related functionality */ +@SuppressWarnings({"deprecation"}) public class TestStateMachine extends BaseTest implements MiniRaftClusterWithSimulatedRpc.FactoryGet { static { Slf4jUtils.setLogLevel(RaftServer.Division.LOG, Level.DEBUG); @@ -91,7 +92,7 @@ public TransactionContext startTransaction(RaftClientRequest request) { @Override public CompletableFuture applyTransaction(TransactionContext trx) { try { - assertNotNull(trx.getLogEntryUnsafe()); + assertNotNull(trx.getLogEntry()); assertNotNull(trx.getStateMachineLogEntry()); Object context = trx.getStateMachineContext(); if (isLeader.get()) { From 478639010f8a86d3edf81f451cb1e69f32081749 Mon Sep 17 00:00:00 2001 From: Duong Nguyen Date: Fri, 21 Jun 2024 09:46:36 -0700 Subject: [PATCH 356/397] RATIS-2114. Corruption due to SegmentedRaftLogWorker queue LogEntry without reference counter (#1113) (cherry picked from commit 34fa62ce7a59decf3fc8671f730f100e9bfe3ef6) --- .../raftlog/segmented/SegmentedRaftLog.java | 3 +++ .../raftlog/segmented/SegmentedRaftLogWorker.java | 15 +++++++++++++++ 2 files changed, 18 insertions(+) diff --git a/ratis-server/src/main/java/org/apache/ratis/server/raftlog/segmented/SegmentedRaftLog.java b/ratis-server/src/main/java/org/apache/ratis/server/raftlog/segmented/SegmentedRaftLog.java index 955ed851d1..05b4c78070 100644 --- a/ratis-server/src/main/java/org/apache/ratis/server/raftlog/segmented/SegmentedRaftLog.java +++ b/ratis-server/src/main/java/org/apache/ratis/server/raftlog/segmented/SegmentedRaftLog.java @@ -100,6 +100,9 @@ void done() { completeFuture(); } + void discard() { + } + final void completeFuture() { final boolean completed = future.complete(getEndIndex()); Preconditions.assertTrue(completed, diff --git a/ratis-server/src/main/java/org/apache/ratis/server/raftlog/segmented/SegmentedRaftLogWorker.java b/ratis-server/src/main/java/org/apache/ratis/server/raftlog/segmented/SegmentedRaftLogWorker.java index 6b09f2b1f8..183c0e381a 100644 --- a/ratis-server/src/main/java/org/apache/ratis/server/raftlog/segmented/SegmentedRaftLogWorker.java +++ b/ratis-server/src/main/java/org/apache/ratis/server/raftlog/segmented/SegmentedRaftLogWorker.java @@ -290,6 +290,7 @@ private Task addIOTask(Task task) { LOG.error("Failed to add IO task {}", task, e); Optional.ofNullable(server).ifPresent(RaftServer.Division::close); } + task.discard(); } task.startTimerOnEnqueue(raftLogMetrics.getEnqueuedTimer()); return task; @@ -498,6 +499,7 @@ private class WriteLog extends Task { private final LogEntryProto entry; private final CompletableFuture stateMachineFuture; private final CompletableFuture combined; + private final ReferenceCountedObject ref; WriteLog(ReferenceCountedObject entryRef, LogEntryProto removedStateMachineData, TransactionContext context) { @@ -514,6 +516,8 @@ private class WriteLog extends Task { } else { this.stateMachineFuture = null; } + entryRef.retain(); + this.ref = entryRef; } else { try { // this.entry != origEntry if it has state machine data @@ -523,6 +527,7 @@ private class WriteLog extends Task { + ", entry=" + LogProtoUtils.toLogEntryString(origEntry, stateMachine::toStateMachineLogEntryString), e); throw e; } + this.ref = null; } this.combined = stateMachineFuture == null? super.getFuture() : super.getFuture().thenCombine(stateMachineFuture, (index, stateMachineResult) -> index); @@ -547,6 +552,16 @@ CompletableFuture getFuture() { @Override void done() { writeTasks.offerOrCompleteFuture(this); + if (ref != null) { + ref.release(); + } + } + + @Override + void discard() { + if (ref != null) { + ref.release(); + } } @Override From 49caf365eaab54f65e24b965a21c87a7349145e3 Mon Sep 17 00:00:00 2001 From: "Doroszlai, Attila" <6454655+adoroszlai@users.noreply.github.com> Date: Sat, 29 Jun 2024 15:17:50 +0200 Subject: [PATCH 357/397] RATIS-2120. Bump version after 3.1.0 release (#1118) (cherry picked from commit c31806dbdcf2f288c57684958447a6ddfe6fc29c) --- pom.xml | 2 +- ratis-assembly/pom.xml | 2 +- ratis-client/pom.xml | 2 +- ratis-common/pom.xml | 2 +- ratis-docs/pom.xml | 2 +- ratis-examples/pom.xml | 2 +- ratis-experiments/pom.xml | 2 +- ratis-grpc/pom.xml | 2 +- ratis-metrics-api/pom.xml | 2 +- ratis-metrics-default/pom.xml | 2 +- ratis-metrics-dropwizard3/pom.xml | 2 +- ratis-netty/pom.xml | 2 +- ratis-proto/pom.xml | 2 +- ratis-replicated-map/pom.xml | 93 +++++++++++++++++++++++++++++++ ratis-resource-bundle/pom.xml | 2 +- ratis-server-api/pom.xml | 2 +- ratis-server/pom.xml | 2 +- ratis-shell/pom.xml | 2 +- ratis-test/pom.xml | 2 +- ratis-tools/pom.xml | 2 +- 20 files changed, 112 insertions(+), 19 deletions(-) create mode 100644 ratis-replicated-map/pom.xml diff --git a/pom.xml b/pom.xml index 4ffed3fdc2..972be8aaa8 100644 --- a/pom.xml +++ b/pom.xml @@ -24,7 +24,7 @@ ratis org.apache.ratis - 3.3.0-SNAPSHOT + 3.2.0-SNAPSHOT Apache Ratis pom diff --git a/ratis-assembly/pom.xml b/ratis-assembly/pom.xml index 0558693039..7cb758616c 100644 --- a/ratis-assembly/pom.xml +++ b/ratis-assembly/pom.xml @@ -17,7 +17,7 @@ ratis org.apache.ratis - 3.3.0-SNAPSHOT + 3.2.0-SNAPSHOT ratis-assembly diff --git a/ratis-client/pom.xml b/ratis-client/pom.xml index 26b2034983..f9de230b23 100644 --- a/ratis-client/pom.xml +++ b/ratis-client/pom.xml @@ -17,7 +17,7 @@ ratis org.apache.ratis - 3.3.0-SNAPSHOT + 3.2.0-SNAPSHOT ratis-client diff --git a/ratis-common/pom.xml b/ratis-common/pom.xml index ba19c73e33..bd7e75eddc 100644 --- a/ratis-common/pom.xml +++ b/ratis-common/pom.xml @@ -17,7 +17,7 @@ ratis org.apache.ratis - 3.3.0-SNAPSHOT + 3.2.0-SNAPSHOT ratis-common diff --git a/ratis-docs/pom.xml b/ratis-docs/pom.xml index 894b6a52e4..e0cbc5fcee 100644 --- a/ratis-docs/pom.xml +++ b/ratis-docs/pom.xml @@ -20,7 +20,7 @@ https://maven.apache.org/xsd/maven-4.0.0.xsd"> ratis org.apache.ratis - 3.3.0-SNAPSHOT + 3.2.0-SNAPSHOT ratis-docs diff --git a/ratis-examples/pom.xml b/ratis-examples/pom.xml index c52019334f..b763da7fd7 100644 --- a/ratis-examples/pom.xml +++ b/ratis-examples/pom.xml @@ -17,7 +17,7 @@ ratis org.apache.ratis - 3.3.0-SNAPSHOT + 3.2.0-SNAPSHOT ratis-examples diff --git a/ratis-experiments/pom.xml b/ratis-experiments/pom.xml index e4a8451db2..001d1755bf 100644 --- a/ratis-experiments/pom.xml +++ b/ratis-experiments/pom.xml @@ -17,7 +17,7 @@ ratis org.apache.ratis - 3.3.0-SNAPSHOT + 3.2.0-SNAPSHOT ratis-experiments diff --git a/ratis-grpc/pom.xml b/ratis-grpc/pom.xml index 360131d55b..e73b1f03dc 100644 --- a/ratis-grpc/pom.xml +++ b/ratis-grpc/pom.xml @@ -17,7 +17,7 @@ ratis org.apache.ratis - 3.3.0-SNAPSHOT + 3.2.0-SNAPSHOT ratis-grpc diff --git a/ratis-metrics-api/pom.xml b/ratis-metrics-api/pom.xml index db368269e2..6731d06602 100644 --- a/ratis-metrics-api/pom.xml +++ b/ratis-metrics-api/pom.xml @@ -18,7 +18,7 @@ ratis org.apache.ratis - 3.3.0-SNAPSHOT + 3.2.0-SNAPSHOT ratis-metrics-api diff --git a/ratis-metrics-default/pom.xml b/ratis-metrics-default/pom.xml index b561cbe924..bcd6020915 100644 --- a/ratis-metrics-default/pom.xml +++ b/ratis-metrics-default/pom.xml @@ -18,7 +18,7 @@ ratis org.apache.ratis - 3.3.0-SNAPSHOT + 3.2.0-SNAPSHOT ratis-metrics-default diff --git a/ratis-metrics-dropwizard3/pom.xml b/ratis-metrics-dropwizard3/pom.xml index 2dd329858e..c77bf2a290 100644 --- a/ratis-metrics-dropwizard3/pom.xml +++ b/ratis-metrics-dropwizard3/pom.xml @@ -18,7 +18,7 @@ ratis org.apache.ratis - 3.3.0-SNAPSHOT + 3.2.0-SNAPSHOT ratis-metrics-dropwizard3 diff --git a/ratis-netty/pom.xml b/ratis-netty/pom.xml index 5688a1fa4a..cb7430451c 100644 --- a/ratis-netty/pom.xml +++ b/ratis-netty/pom.xml @@ -17,7 +17,7 @@ ratis org.apache.ratis - 3.3.0-SNAPSHOT + 3.2.0-SNAPSHOT ratis-netty diff --git a/ratis-proto/pom.xml b/ratis-proto/pom.xml index 322bf56778..8211ff8189 100644 --- a/ratis-proto/pom.xml +++ b/ratis-proto/pom.xml @@ -17,7 +17,7 @@ ratis org.apache.ratis - 3.3.0-SNAPSHOT + 3.2.0-SNAPSHOT ratis-proto diff --git a/ratis-replicated-map/pom.xml b/ratis-replicated-map/pom.xml new file mode 100644 index 0000000000..bded7cd36d --- /dev/null +++ b/ratis-replicated-map/pom.xml @@ -0,0 +1,93 @@ + + + + 4.0.0 + + ratis + org.apache.ratis + 3.2.0-SNAPSHOT + + + ratis-replicated-map + Apache Ratis Replicated Map + + + + org.apache.ratis + ratis-thirdparty-misc + + + ratis-proto + org.apache.ratis + + + ratis-common + org.apache.ratis + + + ratis-common + org.apache.ratis + test + test-jar + + + ratis-client + org.apache.ratis + + + ratis-client + org.apache.ratis + test + test-jar + + + ratis-server + org.apache.ratis + + + ratis-server + org.apache.ratis + test + test-jar + + + ratis-grpc + org.apache.ratis + provided + + + ratis-grpc + org.apache.ratis + test + test-jar + + + ratis-netty + org.apache.ratis + provided + + + ratis-netty + org.apache.ratis + test + test-jar + + + + org.slf4j + slf4j-api + + + diff --git a/ratis-resource-bundle/pom.xml b/ratis-resource-bundle/pom.xml index edecf1b633..c000677c1e 100644 --- a/ratis-resource-bundle/pom.xml +++ b/ratis-resource-bundle/pom.xml @@ -23,7 +23,7 @@ ratis org.apache.ratis - 3.3.0-SNAPSHOT + 3.2.0-SNAPSHOT .. diff --git a/ratis-server-api/pom.xml b/ratis-server-api/pom.xml index feed49190c..36e77fa9a6 100644 --- a/ratis-server-api/pom.xml +++ b/ratis-server-api/pom.xml @@ -17,7 +17,7 @@ ratis org.apache.ratis - 3.3.0-SNAPSHOT + 3.2.0-SNAPSHOT ratis-server-api diff --git a/ratis-server/pom.xml b/ratis-server/pom.xml index 2c0bc93a25..c8f55eff73 100644 --- a/ratis-server/pom.xml +++ b/ratis-server/pom.xml @@ -17,7 +17,7 @@ ratis org.apache.ratis - 3.3.0-SNAPSHOT + 3.2.0-SNAPSHOT ratis-server diff --git a/ratis-shell/pom.xml b/ratis-shell/pom.xml index 495187b2a8..ddd1ad0737 100644 --- a/ratis-shell/pom.xml +++ b/ratis-shell/pom.xml @@ -17,7 +17,7 @@ ratis org.apache.ratis - 3.3.0-SNAPSHOT + 3.2.0-SNAPSHOT ratis-shell diff --git a/ratis-test/pom.xml b/ratis-test/pom.xml index 8780a862ca..ab34105c8b 100644 --- a/ratis-test/pom.xml +++ b/ratis-test/pom.xml @@ -17,7 +17,7 @@ ratis org.apache.ratis - 3.3.0-SNAPSHOT + 3.2.0-SNAPSHOT ratis-test diff --git a/ratis-tools/pom.xml b/ratis-tools/pom.xml index 2d81f2467a..2dccaf7a2e 100644 --- a/ratis-tools/pom.xml +++ b/ratis-tools/pom.xml @@ -17,7 +17,7 @@ ratis org.apache.ratis - 3.3.0-SNAPSHOT + 3.2.0-SNAPSHOT ratis-tools From 45ec1d108c46e59c99e690a999735704c4225f4a Mon Sep 17 00:00:00 2001 From: Haibo Sun <7675577+sunhaibotb@users.noreply.github.com> Date: Tue, 2 Jul 2024 14:42:21 +0800 Subject: [PATCH 358/397] RATIS-2116. Fix the issue where RaftServerImpl.appendEntries may be blocked indefinitely (#1116) (cherry picked from commit 32745c3c4ecb9777a6d44e703134ef1181ad2dd2) --- .../segmented/TestSegmentedRaftLog.java | 181 +++++++++++++++--- 1 file changed, 153 insertions(+), 28 deletions(-) diff --git a/ratis-test/src/test/java/org/apache/ratis/server/raftlog/segmented/TestSegmentedRaftLog.java b/ratis-test/src/test/java/org/apache/ratis/server/raftlog/segmented/TestSegmentedRaftLog.java index d22ed7f3b7..1f04982823 100644 --- a/ratis-test/src/test/java/org/apache/ratis/server/raftlog/segmented/TestSegmentedRaftLog.java +++ b/ratis-test/src/test/java/org/apache/ratis/server/raftlog/segmented/TestSegmentedRaftLog.java @@ -37,14 +37,12 @@ import org.apache.ratis.server.raftlog.RaftLog; import org.apache.ratis.server.storage.RaftStorage; import org.apache.ratis.server.storage.RaftStorageTestUtils; -import org.apache.ratis.statemachine.TransactionContext; import org.apache.ratis.statemachine.impl.SimpleStateMachine4Testing; import org.apache.ratis.statemachine.StateMachine; import org.apache.ratis.statemachine.impl.BaseStateMachine; import org.apache.ratis.util.CodeInjectionForTesting; import org.apache.ratis.util.DataBlockingQueue; import org.apache.ratis.util.LifeCycle; -import org.apache.ratis.util.ReferenceCountedObject; import org.apache.ratis.util.Slf4jUtils; import org.apache.ratis.util.FileUtils; import org.apache.ratis.util.JavaUtils; @@ -77,14 +75,19 @@ import org.junit.jupiter.params.ParameterizedTest; import org.junit.jupiter.params.provider.Arguments; import org.junit.jupiter.params.provider.MethodSource; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; import org.slf4j.event.Level; import static java.lang.Boolean.FALSE; import static java.lang.Boolean.TRUE; -import static org.apache.ratis.server.storage.RaftStorageTestUtils.getLogUnsafe; +import static org.apache.ratis.server.raftlog.segmented.SegmentedRaftLogWorker.RUN_WORKER; +import static org.junit.jupiter.api.Assertions.assertEquals; +import static org.junit.jupiter.api.Assertions.assertThrows; import static org.junit.jupiter.api.Assertions.assertTrue; import static org.junit.jupiter.params.provider.Arguments.arguments; +@SuppressWarnings({"deprecation"}) public class TestSegmentedRaftLog extends BaseTest { static { Slf4jUtils.setLogLevel(SegmentedRaftLogWorker.LOG, Level.INFO); @@ -212,7 +215,7 @@ static List prepareRanges(int startTerm, int endTerm, int segmentS private LogEntryProto getLastEntry(SegmentedRaftLog raftLog) throws IOException { - return getLogUnsafe(raftLog, raftLog.getLastEntryTermIndex().getIndex()); + return raftLog.get(raftLog.getLastEntryTermIndex().getIndex()); } @ParameterizedTest @@ -230,21 +233,21 @@ public void testLoadLogSegments(Boolean useAsyncFlush, Boolean smSyncFlush) thro // check if log entries are loaded correctly for (LogEntryProto e : entries) { LogEntryProto entry = raftLog.get(e.getIndex()); - Assertions.assertEquals(e, entry); + assertEquals(e, entry); } final LogEntryHeader[] termIndices = raftLog.getEntries(0, 500); LogEntryProto[] entriesFromLog = Arrays.stream(termIndices) .map(ti -> { try { - return getLogUnsafe(raftLog, ti.getIndex()); + return raftLog.get(ti.getIndex()); } catch (IOException e) { throw new RuntimeException(e); } }) .toArray(LogEntryProto[]::new); Assertions.assertArrayEquals(entries, entriesFromLog); - Assertions.assertEquals(entries[entries.length - 1], getLastEntry(raftLog)); + assertEquals(entries[entries.length - 1], getLastEntry(raftLog)); final RatisMetricRegistry metricRegistryForLogWorker = RaftLogMetricsBase.createRegistry(MEMBER_ID); @@ -399,7 +402,83 @@ public void testAppendAndRoll(Boolean useAsyncFlush, Boolean smSyncFlush) throws raftLog.open(RaftLog.INVALID_LOG_INDEX, null); // check if the raft log is correct checkEntries(raftLog, entries, 0, entries.size()); - Assertions.assertEquals(9, raftLog.getRaftLogCache().getNumOfSegments()); + assertEquals(9, raftLog.getRaftLogCache().getNumOfSegments()); + } + } + + @ParameterizedTest + @MethodSource("data") + public void testPurgeAfterAppendEntry(Boolean useAsyncFlush, Boolean smSyncFlush) throws Exception { + RaftServerConfigKeys.Log.setAsyncFlushEnabled(properties, useAsyncFlush); + RaftServerConfigKeys.Log.StateMachineData.setSync(properties, smSyncFlush); + RaftServerConfigKeys.Log.setPurgeGap(properties, 1); + RaftServerConfigKeys.Log.setForceSyncNum(properties, 128); + + int startTerm = 0; + int endTerm = 2; + int segmentSize = 10; + long endIndexOfClosedSegment = segmentSize * (endTerm - startTerm - 1); + long nextStartIndex = segmentSize * (endTerm - startTerm); + + // append entries and roll logSegment for later purge operation + List ranges0 = prepareRanges(startTerm, endTerm, segmentSize, 0); + List entries0 = prepareLogEntries(ranges0, null); + try (SegmentedRaftLog raftLog = newSegmentedRaftLog()) { + raftLog.open(RaftLog.INVALID_LOG_INDEX, null); + entries0.stream().map(raftLog::appendEntry).forEach(CompletableFuture::join); + } + + // test the pattern in the task queue of SegmentedRaftLogWorker: (WriteLog, ..., PurgeLog) + List ranges = prepareRanges(endTerm - 1, endTerm, 1, nextStartIndex); + List entries = prepareLogEntries(ranges, null); + + try (SegmentedRaftLog raftLog = newSegmentedRaftLog()) { + final CountDownLatch raftLogOpened = new CountDownLatch(1); + final CountDownLatch tasksAdded = new CountDownLatch(1); + + // inject test code to make the pattern (WriteLog, PurgeLog) + final ConcurrentLinkedQueue> appendFutures = new ConcurrentLinkedQueue<>(); + final AtomicReference> purgeFuture = new AtomicReference<>(); + final AtomicInteger tasksCount = new AtomicInteger(0); + CodeInjectionForTesting.put(RUN_WORKER, (localId, remoteId, args) -> { + // wait for raftLog to be opened + try { + if(!raftLogOpened.await(FIVE_SECONDS.getDuration(), FIVE_SECONDS.getUnit())) { + throw new TimeoutException(); + } + } catch (InterruptedException | TimeoutException e) { + LOG.error("an exception occurred", e); + throw new RuntimeException(e); + } + + // add WriteLog and PurgeLog tasks + entries.stream().map(raftLog::appendEntry).forEach(appendFutures::add); + purgeFuture.set(raftLog.purge(endIndexOfClosedSegment)); + + tasksCount.set(((DataBlockingQueue) args[0]).getNumElements()); + tasksAdded.countDown(); + return true; + }); + + // open raftLog + raftLog.open(RaftLog.INVALID_LOG_INDEX, null); + raftLogOpened.countDown(); + + // wait for all tasks to be added + if(!tasksAdded.await(FIVE_SECONDS.getDuration(), FIVE_SECONDS.getUnit())) { + throw new TimeoutException(); + } + assertEquals(entries.size() + 1, tasksCount.get()); + + // check if the purge task is executed + final Long purged = purgeFuture.get().get(); + LOG.info("purgeIndex = {}, purged = {}", endIndexOfClosedSegment, purged); + assertEquals(endIndexOfClosedSegment, raftLog.getRaftLogCache().getStartIndex()); + + // check if the appendEntry futures are done + JavaUtils.allOf(appendFutures).get(FIVE_SECONDS.getDuration(), FIVE_SECONDS.getUnit()); + } finally { + CodeInjectionForTesting.put(RUN_WORKER, (localId, remoteId, args) -> false); } } @@ -438,7 +517,7 @@ private void testTruncate(List entries, long fromIndex) raftLog.open(RaftLog.INVALID_LOG_INDEX, null); // check if the raft log is correct if (fromIndex > 0) { - Assertions.assertEquals(entries.get((int) (fromIndex - 1)), + assertEquals(entries.get((int) (fromIndex - 1)), getLastEntry(raftLog)); } else { Assertions.assertNull(raftLog.getLastEntryTermIndex()); @@ -452,7 +531,7 @@ private void checkEntries(RaftLog raftLog, List expected, if (size > 0) { for (int i = offset; i < size + offset; i++) { LogEntryProto entry = raftLog.get(expected.get(i).getIndex()); - Assertions.assertEquals(expected.get(i), entry); + assertEquals(expected.get(i), entry); } final LogEntryHeader[] termIndices = raftLog.getEntries( expected.get(offset).getIndex(), @@ -460,7 +539,7 @@ private void checkEntries(RaftLog raftLog, List expected, LogEntryProto[] entriesFromLog = Arrays.stream(termIndices) .map(ti -> { try { - return getLogUnsafe(raftLog, ti.getIndex()); + return raftLog.get(ti.getIndex()); } catch (IOException e) { throw new RuntimeException(e); } @@ -560,7 +639,7 @@ private void purgeAndVerify(int startTerm, int endTerm, int segmentSize, int pur final CompletableFuture f = raftLog.purge(purgeIndex); final Long purged = f.get(); LOG.info("purgeIndex = {}, purged = {}", purgeIndex, purged); - Assertions.assertEquals(expectedIndex, raftLog.getRaftLogCache().getStartIndex()); + assertEquals(expectedIndex, raftLog.getRaftLogCache().getStartIndex()); } } @@ -577,7 +656,8 @@ public void testAppendEntriesWithInconsistency(Boolean useAsyncFlush, Boolean sm List entries = prepareLogEntries(ranges, null); final RetryCache retryCache = RetryCacheTestUtil.createRetryCache(); - try (SegmentedRaftLog raftLog = RetryCacheTestUtil.newSegmentedRaftLog(MEMBER_ID, retryCache, storage, properties)) { + try (SegmentedRaftLog raftLog = + RetryCacheTestUtil.newSegmentedRaftLog(MEMBER_ID, retryCache, storage, properties)) { raftLog.open(RaftLog.INVALID_LOG_INDEX, null); entries.forEach(entry -> RetryCacheTestUtil.createEntry(retryCache, entry)); // append entries to the raftlog @@ -592,34 +672,80 @@ public void testAppendEntriesWithInconsistency(Boolean useAsyncFlush, Boolean sm List newEntries = prepareLogEntries( Arrays.asList(r1, r2, r3), null); - try (SegmentedRaftLog raftLog = RetryCacheTestUtil.newSegmentedRaftLog(MEMBER_ID, retryCache, storage, properties)) { + try (SegmentedRaftLog raftLog = + RetryCacheTestUtil.newSegmentedRaftLog(MEMBER_ID, retryCache, storage, properties)) { raftLog.open(RaftLog.INVALID_LOG_INDEX, null); LOG.info("newEntries[0] = {}", newEntries.get(0)); final int last = newEntries.size() - 1; LOG.info("newEntries[{}] = {}", last, newEntries.get(last)); - raftLog.append(ReferenceCountedObject.wrap(newEntries)).forEach(CompletableFuture::join); + raftLog.append(newEntries).forEach(CompletableFuture::join); checkFailedEntries(entries, 650, retryCache); checkEntries(raftLog, entries, 0, 650); checkEntries(raftLog, newEntries, 100, 100); - Assertions.assertEquals(newEntries.get(newEntries.size() - 1), + assertEquals(newEntries.get(newEntries.size() - 1), getLastEntry(raftLog)); - Assertions.assertEquals(newEntries.get(newEntries.size() - 1).getIndex(), + assertEquals(newEntries.get(newEntries.size() - 1).getIndex(), raftLog.getFlushIndex()); } // load the raftlog again and check - try (SegmentedRaftLog raftLog = RetryCacheTestUtil.newSegmentedRaftLog(MEMBER_ID, retryCache, storage, properties)) { + try (SegmentedRaftLog raftLog = + RetryCacheTestUtil.newSegmentedRaftLog(MEMBER_ID, retryCache, storage, properties)) { raftLog.open(RaftLog.INVALID_LOG_INDEX, null); checkEntries(raftLog, entries, 0, 650); checkEntries(raftLog, newEntries, 100, 100); - Assertions.assertEquals(newEntries.get(newEntries.size() - 1), + assertEquals(newEntries.get(newEntries.size() - 1), getLastEntry(raftLog)); - Assertions.assertEquals(newEntries.get(newEntries.size() - 1).getIndex(), + assertEquals(newEntries.get(newEntries.size() - 1).getIndex(), raftLog.getFlushIndex()); SegmentedRaftLogCache cache = raftLog.getRaftLogCache(); - Assertions.assertEquals(5, cache.getNumOfSegments()); + assertEquals(5, cache.getNumOfSegments()); + } + } + + @ParameterizedTest + @MethodSource("data") + public void testAppendEntriesWithGap(Boolean useAsyncFlush, Boolean smSyncFlush) throws Exception { + RaftServerConfigKeys.Log.setAsyncFlushEnabled(properties, useAsyncFlush); + RaftServerConfigKeys.Log.StateMachineData.setSync(properties, smSyncFlush); + // prepare the log for truncation + List ranges = prepareRanges(0, 5, 200, 0); + List entries = prepareLogEntries(ranges, null); + + final RetryCache retryCache = RetryCacheTestUtil.createRetryCache(); + try (SegmentedRaftLog raftLog = + RetryCacheTestUtil.newSegmentedRaftLog(MEMBER_ID, retryCache, storage, properties)) { + raftLog.open(RaftLog.INVALID_LOG_INDEX, null); + entries.forEach(entry -> RetryCacheTestUtil.createEntry(retryCache, entry)); + // append entries to the raftlog + entries.stream().map(raftLog::appendEntry).forEach(CompletableFuture::join); + } + + long lastIndex = ranges.get(ranges.size() - 1).end; + long snapshotIndex = lastIndex + 100; + LogEntryProto entryProto = prepareLogEntry(4, snapshotIndex + 1, null, false); + final LongSupplier getSnapshotIndexFromStateMachine = new LongSupplier() { + @Override + public long getAsLong() { + return snapshotIndex; + } + }; + try (SegmentedRaftLog raftLog = newSegmentedRaftLog(getSnapshotIndexFromStateMachine)) { + raftLog.open(RaftLog.INVALID_LOG_INDEX, null); + // Assert the wrapped exception + IllegalStateException exception = assertThrows(IllegalStateException.class, + () -> raftLog.appendEntry(entryProto)); + // Assert the original cause + assertTrue(exception.getMessage().contains("gap between entries")); + } + + // load the raftlog again and check + try (SegmentedRaftLog raftLog = + RetryCacheTestUtil.newSegmentedRaftLog(MEMBER_ID, retryCache, storage, properties)) { + raftLog.open(RaftLog.INVALID_LOG_INDEX, null); + Assertions.assertEquals(lastIndex, raftLog.getRaftLogCache().getEndIndex()); } } @@ -685,7 +811,7 @@ public void testServerShutdownOnTimeoutIOException(Boolean useAsyncFlush, Boolea final LogEntryProto entry = prepareLogEntry(0, 0, null, true); final StateMachine sm = new BaseStateMachine() { @Override - public CompletableFuture write(ReferenceCountedObject entry, TransactionContext context) { + public CompletableFuture write(LogEntryProto entry) { getLifeCycle().transition(LifeCycle.State.STARTING); getLifeCycle().transition(LifeCycle.State.RUNNING); @@ -715,7 +841,7 @@ public void notifyLogFailed(Throwable cause, LogEntryProto entry) { // SegmentedRaftLogWorker should catch TimeoutIOException CompletableFuture f = raftLog.appendEntry(entry); // Wait for async writeStateMachineData to finish - ex = Assertions.assertThrows(ExecutionException.class, f::get); + ex = assertThrows(ExecutionException.class, f::get); } Assertions.assertSame(LifeCycle.State.PAUSED, sm.getLifeCycleState()); Assertions.assertInstanceOf(TimeoutIOException.class, ex.getCause()); @@ -735,9 +861,9 @@ static Thread startAppendEntryThread(RaftLog raftLog, LogEntryProto entry) { void assertIndices(RaftLog raftLog, long expectedFlushIndex, long expectedNextIndex) { LOG.info("assert expectedFlushIndex={}", expectedFlushIndex); - Assertions.assertEquals(expectedFlushIndex, raftLog.getFlushIndex()); + assertEquals(expectedFlushIndex, raftLog.getFlushIndex()); LOG.info("assert expectedNextIndex={}", expectedNextIndex); - Assertions.assertEquals(expectedNextIndex, raftLog.getNextIndex()); + assertEquals(expectedNextIndex, raftLog.getNextIndex()); } void assertIndicesMultipleAttempts(RaftLog raftLog, long expectedFlushIndex, long expectedNextIndex) @@ -761,9 +887,8 @@ public void testAsyncFlushPerf1(Boolean useAsyncFlush, Boolean smSyncFlush) thro long start = System.nanoTime(); for (int i = 0; i < entries.size(); i += 5) { // call append API - List entries1 = Arrays.asList( - entries.get(i), entries.get(i + 1), entries.get(i + 2), entries.get(i + 3), entries.get(i + 4)); - futures.add(raftLog.append(ReferenceCountedObject.wrap(entries1))); + futures.add(raftLog.append(Arrays.asList( + entries.get(i), entries.get(i + 1), entries.get(i + 2), entries.get(i + 3), entries.get(i + 4)))); } for (List> futureList: futures) { futureList.forEach(CompletableFuture::join); From f09e5af32065b5456d5e063c3dd23a1698298749 Mon Sep 17 00:00:00 2001 From: Chung En Lee Date: Tue, 6 Aug 2024 22:01:31 +0800 Subject: [PATCH 359/397] RATIS-2134. `logMetadata` could miss appending the `metadataEntry` with the lastCommitIndex (#1130) (cherry picked from commit 4ba463789df7876837c728ce6477fd18b787fed9) --- .../ratis/server/raftlog/RaftLogBase.java | 18 +++--------------- 1 file changed, 3 insertions(+), 15 deletions(-) diff --git a/ratis-server/src/main/java/org/apache/ratis/server/raftlog/RaftLogBase.java b/ratis-server/src/main/java/org/apache/ratis/server/raftlog/RaftLogBase.java index 024845fac4..2fd301915c 100644 --- a/ratis-server/src/main/java/org/apache/ratis/server/raftlog/RaftLogBase.java +++ b/ratis-server/src/main/java/org/apache/ratis/server/raftlog/RaftLogBase.java @@ -244,21 +244,9 @@ private boolean shouldAppendMetadata(long newCommitIndex) { // do not log the first conf entry return false; } - ReferenceCountedObject ref = null; - try { - ref = retainLog(newCommitIndex); - if (ref.get().hasMetadataEntry()) { - // do not log the metadata entry - return false; - } - } catch(RaftLogIOException e) { - LOG.error("Failed to get log entry for index " + newCommitIndex, e); - } finally { - if (ref != null) { - ref.release(); - } - } - return true; + final LogEntryProto last = lastMetadataEntry.get(); + // do not log entries with a smaller commit index. + return last == null || newCommitIndex > last.getMetadataEntry().getCommitIndex(); } @Override From c028860797a67e8a6cb51611b78af5fd065cf55b Mon Sep 17 00:00:00 2001 From: Tsz-Wo Nicholas Sze Date: Wed, 14 Aug 2024 11:09:29 -0700 Subject: [PATCH 360/397] RATIS-2139. Fix checkstyle:ParameterNumber in GrpcService. (#1134) (cherry picked from commit 8d4c4ae3082f24e22aab454ec144f384ec6e1b1a) --- .../apache/ratis/grpc/server/GrpcService.java | 265 ++++++++++-------- 1 file changed, 152 insertions(+), 113 deletions(-) diff --git a/ratis-grpc/src/main/java/org/apache/ratis/grpc/server/GrpcService.java b/ratis-grpc/src/main/java/org/apache/ratis/grpc/server/GrpcService.java index e3c0a5eddb..e8d707fc51 100644 --- a/ratis-grpc/src/main/java/org/apache/ratis/grpc/server/GrpcService.java +++ b/ratis-grpc/src/main/java/org/apache/ratis/grpc/server/GrpcService.java @@ -23,7 +23,9 @@ import org.apache.ratis.grpc.GrpcUtil; import org.apache.ratis.grpc.metrics.ZeroCopyMetrics; import org.apache.ratis.grpc.metrics.intercept.server.MetricServerInterceptor; +import org.apache.ratis.protocol.AdminAsynchronousProtocol; import org.apache.ratis.protocol.RaftGroupId; +import org.apache.ratis.protocol.RaftPeer; import org.apache.ratis.protocol.RaftPeerId; import org.apache.ratis.rpc.SupportedRpcType; import org.apache.ratis.server.RaftServer; @@ -31,7 +33,9 @@ import org.apache.ratis.server.RaftServerRpcWithProxy; import org.apache.ratis.server.protocol.RaftServerAsynchronousProtocol; import org.apache.ratis.thirdparty.com.google.common.annotations.VisibleForTesting; +import org.apache.ratis.thirdparty.io.grpc.ServerInterceptor; import org.apache.ratis.thirdparty.io.grpc.ServerInterceptors; +import org.apache.ratis.thirdparty.io.grpc.ServerServiceDefinition; import org.apache.ratis.thirdparty.io.grpc.netty.GrpcSslContexts; import org.apache.ratis.thirdparty.io.grpc.netty.NettyServerBuilder; import org.apache.ratis.thirdparty.io.grpc.Server; @@ -99,20 +103,141 @@ public void onCompleted() { public static final class Builder { private RaftServer server; - private GrpcTlsConfig tlsConfig; + + private String adminHost; + private int adminPort; private GrpcTlsConfig adminTlsConfig; + private String clientHost; + private int clientPort; private GrpcTlsConfig clientTlsConfig; + private String serverHost; + private int serverPort; private GrpcTlsConfig serverTlsConfig; + private SizeInBytes messageSizeMax; + private SizeInBytes flowControlWindow; + private TimeDuration requestTimeoutDuration; + private boolean separateHeartbeatChannel; + private boolean zeroCopyEnabled; + private Builder() {} public Builder setServer(RaftServer raftServer) { this.server = raftServer; + + final RaftProperties properties = server.getProperties(); + this.adminHost = GrpcConfigKeys.Admin.host(properties); + this.adminPort = GrpcConfigKeys.Admin.port(properties); + this.clientHost = GrpcConfigKeys.Client.host(properties); + this.clientPort = GrpcConfigKeys.Client.port(properties); + this.serverHost = GrpcConfigKeys.Server.host(properties); + this.serverPort = GrpcConfigKeys.Server.port(properties); + this.messageSizeMax = GrpcConfigKeys.messageSizeMax(properties, LOG::info); + this.flowControlWindow = GrpcConfigKeys.flowControlWindow(properties, LOG::info); + this.requestTimeoutDuration = RaftServerConfigKeys.Rpc.requestTimeout(properties); + this.separateHeartbeatChannel = GrpcConfigKeys.Server.heartbeatChannel(properties); + this.zeroCopyEnabled = GrpcConfigKeys.Server.zeroCopyEnabled(properties); + + final SizeInBytes appenderBufferSize = RaftServerConfigKeys.Log.Appender.bufferByteLimit(properties); + final SizeInBytes gap = SizeInBytes.ONE_MB; + final long diff = messageSizeMax.getSize() - appenderBufferSize.getSize(); + if (diff < gap.getSize()) { + throw new IllegalArgumentException("Illegal configuration: " + + GrpcConfigKeys.MESSAGE_SIZE_MAX_KEY + "(= " + messageSizeMax + + ") must be " + gap + " larger than " + + RaftServerConfigKeys.Log.Appender.BUFFER_BYTE_LIMIT_KEY + "(= " + appenderBufferSize + ")."); + } + return this; } + private GrpcServerProtocolClient newGrpcServerProtocolClient(RaftPeer target) { + return new GrpcServerProtocolClient(target, flowControlWindow.getSizeInt(), + requestTimeoutDuration, serverTlsConfig, separateHeartbeatChannel); + } + + private ExecutorService newExecutor() { + final RaftProperties properties = server.getProperties(); + return ConcurrentUtils.newThreadPoolWithMax( + GrpcConfigKeys.Server.asyncRequestThreadPoolCached(properties), + GrpcConfigKeys.Server.asyncRequestThreadPoolSize(properties), + server.getId() + "-request-"); + } + + private GrpcClientProtocolService newGrpcClientProtocolService( + ExecutorService executor, ZeroCopyMetrics zeroCopyMetrics) { + return new GrpcClientProtocolService(server::getId, server, executor, zeroCopyEnabled, zeroCopyMetrics); + } + + private GrpcServerProtocolService newGrpcServerProtocolService(ZeroCopyMetrics zeroCopyMetrics) { + return new GrpcServerProtocolService(server::getId, server, zeroCopyEnabled, zeroCopyMetrics); + } + + private MetricServerInterceptor newMetricServerInterceptor() { + return new MetricServerInterceptor(server::getId, + JavaUtils.getClassSimpleName(getClass()) + "_" + serverPort); + } + + private NettyServerBuilder newNettyServerBuilderForServer() { + return newNettyServerBuilder(serverHost, serverPort, serverTlsConfig); + } + + private NettyServerBuilder newNettyServerBuilderForAdmin() { + return newNettyServerBuilder(adminHost, adminPort, adminTlsConfig); + } + + private NettyServerBuilder newNettyServerBuilderForClient() { + return newNettyServerBuilder(clientHost, clientPort, clientTlsConfig); + } + + private NettyServerBuilder newNettyServerBuilder(String hostname, int port, GrpcTlsConfig tlsConfig) { + final InetSocketAddress address = hostname == null || hostname.isEmpty() ? + new InetSocketAddress(port) : new InetSocketAddress(hostname, port); + final NettyServerBuilder nettyServerBuilder = NettyServerBuilder.forAddress(address) + .withChildOption(ChannelOption.SO_REUSEADDR, true) + .maxInboundMessageSize(messageSizeMax.getSizeInt()) + .flowControlWindow(flowControlWindow.getSizeInt()); + + if (tlsConfig != null) { + SslContextBuilder sslContextBuilder = GrpcUtil.initSslContextBuilderForServer(tlsConfig.getKeyManager()); + if (tlsConfig.getMtlsEnabled()) { + sslContextBuilder.clientAuth(ClientAuth.REQUIRE); + GrpcUtil.setTrustManager(sslContextBuilder, tlsConfig.getTrustManager()); + } + sslContextBuilder = GrpcSslContexts.configure(sslContextBuilder, OPENSSL); + try { + nettyServerBuilder.sslContext(sslContextBuilder.build()); + } catch (Exception ex) { + throw new IllegalArgumentException("Failed to build SslContext, tlsConfig=" + tlsConfig, ex); + } + } + return nettyServerBuilder; + } + + private boolean separateAdminServer() { + return adminPort > 0 && adminPort != serverPort; + } + + private boolean separateClientServer() { + return clientPort > 0 && clientPort != serverPort; + } + + Server newServer(GrpcClientProtocolService client, ZeroCopyMetrics zeroCopyMetrics, ServerInterceptor interceptor) { + final NettyServerBuilder serverBuilder = newNettyServerBuilderForServer(); + final ServerServiceDefinition service = newGrpcServerProtocolService(zeroCopyMetrics).bindServiceWithZeroCopy(); + serverBuilder.addService(ServerInterceptors.intercept(service, interceptor)); + + if (!separateAdminServer()) { + addAdminService(serverBuilder, server, interceptor); + } + if (!separateClientServer()) { + addClientService(serverBuilder, client, interceptor); + } + return serverBuilder.build(); + } + public GrpcService build() { - return new GrpcService(server, adminTlsConfig, clientTlsConfig, serverTlsConfig); + return new GrpcService(this); } public Builder setTlsConfig(GrpcTlsConfig tlsConfig) { @@ -155,144 +280,58 @@ public static Builder newBuilder() { private final GrpcClientProtocolService clientProtocolService; private final MetricServerInterceptor serverInterceptor; - private final ZeroCopyMetrics zeroCopyMetrics; + private final ZeroCopyMetrics zeroCopyMetrics = new ZeroCopyMetrics(); public MetricServerInterceptor getServerInterceptor() { return serverInterceptor; } - private GrpcService(RaftServer server, - GrpcTlsConfig adminTlsConfig, GrpcTlsConfig clientTlsConfig, GrpcTlsConfig serverTlsConfig) { - this(server, server::getId, - GrpcConfigKeys.Admin.host(server.getProperties()), - GrpcConfigKeys.Admin.port(server.getProperties()), - adminTlsConfig, - GrpcConfigKeys.Client.host(server.getProperties()), - GrpcConfigKeys.Client.port(server.getProperties()), - clientTlsConfig, - GrpcConfigKeys.Server.host(server.getProperties()), - GrpcConfigKeys.Server.port(server.getProperties()), - serverTlsConfig, - GrpcConfigKeys.messageSizeMax(server.getProperties(), LOG::info), - RaftServerConfigKeys.Log.Appender.bufferByteLimit(server.getProperties()), - GrpcConfigKeys.flowControlWindow(server.getProperties(), LOG::info), - RaftServerConfigKeys.Rpc.requestTimeout(server.getProperties()), - GrpcConfigKeys.Server.heartbeatChannel(server.getProperties()), - GrpcConfigKeys.Server.zeroCopyEnabled(server.getProperties())); - } - - @SuppressWarnings("checkstyle:ParameterNumber") // private constructor - private GrpcService(RaftServer raftServer, Supplier idSupplier, - String adminHost, int adminPort, GrpcTlsConfig adminTlsConfig, - String clientHost, int clientPort, GrpcTlsConfig clientTlsConfig, - String serverHost, int serverPort, GrpcTlsConfig serverTlsConfig, - SizeInBytes grpcMessageSizeMax, SizeInBytes appenderBufferSize, - SizeInBytes flowControlWindow,TimeDuration requestTimeoutDuration, - boolean useSeparateHBChannel, boolean zeroCopyEnabled) { - super(idSupplier, id -> new PeerProxyMap<>(id.toString(), - p -> new GrpcServerProtocolClient(p, flowControlWindow.getSizeInt(), - requestTimeoutDuration, serverTlsConfig, useSeparateHBChannel))); - if (appenderBufferSize.getSize() > grpcMessageSizeMax.getSize()) { - throw new IllegalArgumentException("Illegal configuration: " - + RaftServerConfigKeys.Log.Appender.BUFFER_BYTE_LIMIT_KEY + " = " + appenderBufferSize - + " > " + GrpcConfigKeys.MESSAGE_SIZE_MAX_KEY + " = " + grpcMessageSizeMax); - } + private GrpcService(Builder b) { + super(b.server::getId, id -> new PeerProxyMap<>(id.toString(), b::newGrpcServerProtocolClient)); - final RaftProperties properties = raftServer.getProperties(); - this.executor = ConcurrentUtils.newThreadPoolWithMax( - GrpcConfigKeys.Server.asyncRequestThreadPoolCached(properties), - GrpcConfigKeys.Server.asyncRequestThreadPoolSize(properties), - getId() + "-request-"); - this.zeroCopyMetrics = new ZeroCopyMetrics(); - this.clientProtocolService = new GrpcClientProtocolService(idSupplier, raftServer, executor, - zeroCopyEnabled, zeroCopyMetrics); - - this.serverInterceptor = new MetricServerInterceptor( - idSupplier, - JavaUtils.getClassSimpleName(getClass()) + "_" + serverPort - ); - - final boolean separateAdminServer = adminPort != serverPort && adminPort > 0; - final boolean separateClientServer = clientPort != serverPort && clientPort > 0; - - final NettyServerBuilder serverBuilder = - startBuildingNettyServer(serverHost, serverPort, serverTlsConfig, grpcMessageSizeMax, flowControlWindow); - GrpcServerProtocolService serverProtocolService = new GrpcServerProtocolService(idSupplier, raftServer, - zeroCopyEnabled, zeroCopyMetrics); - serverBuilder.addService(ServerInterceptors.intercept( - serverProtocolService.bindServiceWithZeroCopy(), serverInterceptor)); - if (!separateAdminServer) { - addAdminService(raftServer, serverBuilder); - } - if (!separateClientServer) { - addClientService(serverBuilder); - } + this.executor = b.newExecutor(); + this.clientProtocolService = b.newGrpcClientProtocolService(executor, zeroCopyMetrics); + this.serverInterceptor = b.newMetricServerInterceptor(); + final Server server = b.newServer(clientProtocolService, zeroCopyMetrics, serverInterceptor); - final Server server = serverBuilder.build(); servers.put(GrpcServerProtocolService.class.getSimpleName(), server); - addressSupplier = newAddressSupplier(serverPort, server); + addressSupplier = newAddressSupplier(b.serverPort, server); - if (separateAdminServer) { - final NettyServerBuilder builder = - startBuildingNettyServer(adminHost, adminPort, adminTlsConfig, grpcMessageSizeMax, flowControlWindow); - addAdminService(raftServer, builder); + if (b.separateAdminServer()) { + final NettyServerBuilder builder = b.newNettyServerBuilderForAdmin(); + addAdminService(builder, b.server, serverInterceptor); final Server adminServer = builder.build(); servers.put(GrpcAdminProtocolService.class.getName(), adminServer); - adminServerAddressSupplier = newAddressSupplier(adminPort, adminServer); + adminServerAddressSupplier = newAddressSupplier(b.adminPort, adminServer); } else { adminServerAddressSupplier = addressSupplier; } - if (separateClientServer) { - final NettyServerBuilder builder = - startBuildingNettyServer(clientHost, clientPort, clientTlsConfig, grpcMessageSizeMax, flowControlWindow); - addClientService(builder); + if (b.separateClientServer()) { + final NettyServerBuilder builder = b.newNettyServerBuilderForClient(); + addClientService(builder, clientProtocolService, serverInterceptor); final Server clientServer = builder.build(); servers.put(GrpcClientProtocolService.class.getName(), clientServer); - clientServerAddressSupplier = newAddressSupplier(clientPort, clientServer); + clientServerAddressSupplier = newAddressSupplier(b.clientPort, clientServer); } else { clientServerAddressSupplier = addressSupplier; } } - private MemoizedSupplier newAddressSupplier(int port, Server server) { + static MemoizedSupplier newAddressSupplier(int port, Server server) { return JavaUtils.memoize(() -> new InetSocketAddress(port != 0 ? port : server.getPort())); } - private void addClientService(NettyServerBuilder builder) { - builder.addService(ServerInterceptors.intercept(clientProtocolService.bindServiceWithZeroCopy(), - serverInterceptor)); + static void addClientService(NettyServerBuilder builder, GrpcClientProtocolService client, + ServerInterceptor interceptor) { + final ServerServiceDefinition service = client.bindServiceWithZeroCopy(); + builder.addService(ServerInterceptors.intercept(service, interceptor)); } - private void addAdminService(RaftServer raftServer, NettyServerBuilder nettyServerBuilder) { - nettyServerBuilder.addService(ServerInterceptors.intercept( - new GrpcAdminProtocolService(raftServer), - serverInterceptor)); - } - - private static NettyServerBuilder startBuildingNettyServer(String hostname, int port, GrpcTlsConfig tlsConfig, - SizeInBytes grpcMessageSizeMax, SizeInBytes flowControlWindow) { - InetSocketAddress address = hostname == null || hostname.isEmpty() ? - new InetSocketAddress(port) : new InetSocketAddress(hostname, port); - NettyServerBuilder nettyServerBuilder = NettyServerBuilder.forAddress(address) - .withChildOption(ChannelOption.SO_REUSEADDR, true) - .maxInboundMessageSize(grpcMessageSizeMax.getSizeInt()) - .flowControlWindow(flowControlWindow.getSizeInt()); - - if (tlsConfig != null) { - SslContextBuilder sslContextBuilder = GrpcUtil.initSslContextBuilderForServer(tlsConfig.getKeyManager()); - if (tlsConfig.getMtlsEnabled()) { - sslContextBuilder.clientAuth(ClientAuth.REQUIRE); - GrpcUtil.setTrustManager(sslContextBuilder, tlsConfig.getTrustManager()); - } - sslContextBuilder = GrpcSslContexts.configure(sslContextBuilder, OPENSSL); - try { - nettyServerBuilder.sslContext(sslContextBuilder.build()); - } catch (Exception ex) { - throw new IllegalArgumentException("Failed to build SslContext, tlsConfig=" + tlsConfig, ex); - } - } - return nettyServerBuilder; + static void addAdminService(NettyServerBuilder builder, AdminAsynchronousProtocol admin, + ServerInterceptor interceptor) { + final GrpcAdminProtocolService service = new GrpcAdminProtocolService(admin); + builder.addService(ServerInterceptors.intercept(service, interceptor)); } @Override From d1ebf2bb03a9c2429ec47df76349cd6a8fe59ede Mon Sep 17 00:00:00 2001 From: Flyangz Date: Thu, 22 Aug 2024 00:10:04 +0800 Subject: [PATCH 361/397] RATIS-2137. Fix LogAppenderDefault in handling of INCONSISTENCY. (#1136) (cherry picked from commit 287a4ac7c467c5a5132c0e85bc83923e27a0f064) --- .../org/apache/ratis/server/leader/LogAppenderDefault.java | 7 +++++-- 1 file changed, 5 insertions(+), 2 deletions(-) diff --git a/ratis-server/src/main/java/org/apache/ratis/server/leader/LogAppenderDefault.java b/ratis-server/src/main/java/org/apache/ratis/server/leader/LogAppenderDefault.java index be92423beb..3d1e6de30b 100644 --- a/ratis-server/src/main/java/org/apache/ratis/server/leader/LogAppenderDefault.java +++ b/ratis-server/src/main/java/org/apache/ratis/server/leader/LogAppenderDefault.java @@ -80,9 +80,12 @@ private AppendEntriesReplyProto sendAppendEntriesWithRetries(AtomicLong requestF return null; } - AppendEntriesReplyProto r = sendAppendEntries(request.get()); + final AppendEntriesRequestProto proto = request.get(); + final AppendEntriesReplyProto reply = sendAppendEntries(proto); + final long first = proto.getEntriesCount() > 0 ? proto.getEntries(0).getIndex() : RaftLog.INVALID_LOG_INDEX; + requestFirstIndex.set(first); request.release(); - return r; + return reply; } catch (InterruptedIOException | RaftLogIOException e) { throw e; } catch (IOException ioe) { From 4d31909a6523bf31b830a3324cbb20805242f6fd Mon Sep 17 00:00:00 2001 From: Tsz-Wo Nicholas Sze Date: Fri, 30 Aug 2024 10:37:27 -0700 Subject: [PATCH 362/397] RATIS-2129. Low replication performance because LogAppender is often blocked by RaftLog's readLock. (#1141) (cherry picked from commit 781d61d37411b374f104eb0806e1e2c4090fb35e) --- .../server/raftlog/segmented/LogSegment.java | 24 +++++------ .../raftlog/segmented/SegmentedRaftLog.java | 42 +++++++------------ .../segmented/TestSegmentedRaftLogCache.java | 2 +- 3 files changed, 28 insertions(+), 40 deletions(-) diff --git a/ratis-server/src/main/java/org/apache/ratis/server/raftlog/segmented/LogSegment.java b/ratis-server/src/main/java/org/apache/ratis/server/raftlog/segmented/LogSegment.java index 76a0dd35c1..76aef3c19c 100644 --- a/ratis-server/src/main/java/org/apache/ratis/server/raftlog/segmented/LogSegment.java +++ b/ratis-server/src/main/java/org/apache/ratis/server/raftlog/segmented/LogSegment.java @@ -36,7 +36,6 @@ import java.io.File; import java.io.IOException; -import java.nio.file.Path; import java.util.Comparator; import java.util.Map; import java.util.Objects; @@ -396,19 +395,18 @@ void appendToOpenSegment(LogEntryProto entry, Op op, boolean verifyEntryIndex) { public static final String APPEND_RECORD = LogSegment.class.getSimpleName() + ".append"; private void append(boolean keepEntryInCache, LogEntryProto entry, Op op, boolean verifyEntryIndex) { Objects.requireNonNull(entry, "entry == null"); - if (verifyEntryIndex) { - verifyEntryIndex(entry.getIndex()); + final LogRecord currentLast = records.getLast(); + if (currentLast == null) { + Preconditions.assertTrue(entry.getIndex() == startIndex, + "gap between start index %s and first entry to append %s", + startIndex, entry.getIndex()); + } else { + Preconditions.assertTrue(entry.getIndex() == currentLast.getTermIndex().getIndex() + 1, + "gap between entries %s and %s", entry.getIndex(), currentLast.getTermIndex().getIndex()); } + final LogRecord record = new LogRecord(totalFileSize, entry); - if (keepEntryInCache) { - // It is important to put the entry into the cache before appending the - // record to the record list. Otherwise, a reader thread may get the - // record from the list but not the entry from the cache. - putEntryCache(record.getTermIndex(), entry, op); - CodeInjectionForTesting.execute(APPEND_RECORD, this, record.getTermIndex()); - } records.append(record); - totalFileSize += getEntrySize(entry, op); endIndex = entry.getIndex(); } @@ -447,7 +445,7 @@ synchronized ReferenceCountedObject loadCache(LogRecord record) t } LogRecord getLogRecord(long index) { - if (index >= startIndex && index <= getEndIndex()) { + if (index >= startIndex && index <= endIndex) { return records.get(index); } return null; @@ -474,7 +472,7 @@ synchronized void truncate(long fromIndex) { for (long index = endIndex; index >= fromIndex; index--) { final LogRecord removed = records.removeLast(); Preconditions.assertSame(index, removed.getTermIndex().getIndex(), "removedIndex"); - removeEntryCache(removed.getTermIndex(), Op.REMOVE_CACHE); + removeEntryCache(removed.getTermIndex()); totalFileSize = removed.offset; } isOpen = false; diff --git a/ratis-server/src/main/java/org/apache/ratis/server/raftlog/segmented/SegmentedRaftLog.java b/ratis-server/src/main/java/org/apache/ratis/server/raftlog/segmented/SegmentedRaftLog.java index 05b4c78070..450450f13f 100644 --- a/ratis-server/src/main/java/org/apache/ratis/server/raftlog/segmented/SegmentedRaftLog.java +++ b/ratis-server/src/main/java/org/apache/ratis/server/raftlog/segmented/SegmentedRaftLog.java @@ -302,23 +302,19 @@ public LogEntryProto get(long index) throws RaftLogIOException { @Override public ReferenceCountedObject retainLog(long index) throws RaftLogIOException { checkLogState(); - final LogSegment segment; - final LogRecord record; - try (AutoCloseableLock readLock = readLock()) { - segment = cache.getSegment(index); - if (segment == null) { - return null; - } - record = segment.getLogRecord(index); - if (record == null) { - return null; - } - final ReferenceCountedObject entry = segment.getEntryFromCache(record.getTermIndex()); - if (entry != null) { - getRaftLogMetrics().onRaftLogCacheHit(); - entry.retain(); - return entry; - } + final LogSegment segment = cache.getSegment(index); + if (segment == null) { + return null; + } + final LogRecord record = segment.getLogRecord(index); + if (record == null) { + return null; + } + final ReferenceCountedObject entry = segment.getEntryFromCache(record.getTermIndex()); + if (entry != null) { + getRaftLogMetrics().onRaftLogCacheHit(); + entry.retain(); + return entry; } // the entry is not in the segment's cache. Load the cache without holding the lock. @@ -375,25 +371,19 @@ private void checkAndEvictCache() { @Override public TermIndex getTermIndex(long index) { checkLogState(); - try(AutoCloseableLock readLock = readLock()) { - return cache.getTermIndex(index); - } + return cache.getTermIndex(index); } @Override public LogEntryHeader[] getEntries(long startIndex, long endIndex) { checkLogState(); - try(AutoCloseableLock readLock = readLock()) { - return cache.getTermIndices(startIndex, endIndex); - } + return cache.getTermIndices(startIndex, endIndex); } @Override public TermIndex getLastEntryTermIndex() { checkLogState(); - try(AutoCloseableLock readLock = readLock()) { - return cache.getLastTermIndex(); - } + return cache.getLastTermIndex(); } @Override diff --git a/ratis-test/src/test/java/org/apache/ratis/server/raftlog/segmented/TestSegmentedRaftLogCache.java b/ratis-test/src/test/java/org/apache/ratis/server/raftlog/segmented/TestSegmentedRaftLogCache.java index da0c6c8a3f..9bf6116bcf 100644 --- a/ratis-test/src/test/java/org/apache/ratis/server/raftlog/segmented/TestSegmentedRaftLogCache.java +++ b/ratis-test/src/test/java/org/apache/ratis/server/raftlog/segmented/TestSegmentedRaftLogCache.java @@ -288,7 +288,7 @@ private void testIterator(long startIndex) { TermIndex prev = null; while (iterator.hasNext()) { TermIndex termIndex = iterator.next(); - Assertions.assertEquals(cache.getLogRecord(termIndex.getIndex()).getTermIndex(), termIndex); + Assertions.assertEquals(cache.getTermIndex(termIndex.getIndex()), termIndex); if (prev != null) { Assertions.assertEquals(prev.getIndex() + 1, termIndex.getIndex()); } From fd2b8cfc84dc60c0f189bc76a433ce0d6792b9c6 Mon Sep 17 00:00:00 2001 From: Tsz-Wo Nicholas Sze Date: Tue, 17 Sep 2024 22:13:38 -0700 Subject: [PATCH 363/397] RATIS-2155. Add a builder for RatisShell. (#1150) (cherry picked from commit 6451f9b3ee3784620f6fb8984bf305f4df11a185) --- .../grpc/client/GrpcClientProtocolClient.java | 14 ++++++++++++-- .../org/apache/ratis/grpc/server/GrpcService.java | 1 + .../apache/ratis/grpc/MiniRaftClusterWithGrpc.java | 7 +++++-- 3 files changed, 18 insertions(+), 4 deletions(-) diff --git a/ratis-grpc/src/main/java/org/apache/ratis/grpc/client/GrpcClientProtocolClient.java b/ratis-grpc/src/main/java/org/apache/ratis/grpc/client/GrpcClientProtocolClient.java index 0eaec6b962..8f089aa1c7 100644 --- a/ratis-grpc/src/main/java/org/apache/ratis/grpc/client/GrpcClientProtocolClient.java +++ b/ratis-grpc/src/main/java/org/apache/ratis/grpc/client/GrpcClientProtocolClient.java @@ -128,9 +128,19 @@ private ManagedChannel buildChannel(String address, SslContext sslContext, // ignore any http proxy for grpc channelBuilder.proxyDetector(uri -> null); - if (sslContext != null) { + if (tlsConf != null) { LOG.debug("Setting TLS for {}", address); - channelBuilder.useTransportSecurity().sslContext(sslContext); + SslContextBuilder sslContextBuilder = GrpcSslContexts.forClient(); + GrpcUtil.setTrustManager(sslContextBuilder, tlsConf.getTrustManager()); + if (tlsConf.getMtlsEnabled()) { + GrpcUtil.setKeyManager(sslContextBuilder, tlsConf.getKeyManager()); + } + try { + channelBuilder.useTransportSecurity().sslContext( + sslContextBuilder.build()); + } catch (Exception ex) { + throw new RuntimeException(ex); + } } else { channelBuilder.negotiationType(NegotiationType.PLAINTEXT); } diff --git a/ratis-grpc/src/main/java/org/apache/ratis/grpc/server/GrpcService.java b/ratis-grpc/src/main/java/org/apache/ratis/grpc/server/GrpcService.java index e8d707fc51..ac48fa1adf 100644 --- a/ratis-grpc/src/main/java/org/apache/ratis/grpc/server/GrpcService.java +++ b/ratis-grpc/src/main/java/org/apache/ratis/grpc/server/GrpcService.java @@ -199,6 +199,7 @@ private NettyServerBuilder newNettyServerBuilder(String hostname, int port, Grpc .flowControlWindow(flowControlWindow.getSizeInt()); if (tlsConfig != null) { + LOG.info("Setting TLS for {}", address); SslContextBuilder sslContextBuilder = GrpcUtil.initSslContextBuilderForServer(tlsConfig.getKeyManager()); if (tlsConfig.getMtlsEnabled()) { sslContextBuilder.clientAuth(ClientAuth.REQUIRE); diff --git a/ratis-grpc/src/test/java/org/apache/ratis/grpc/MiniRaftClusterWithGrpc.java b/ratis-grpc/src/test/java/org/apache/ratis/grpc/MiniRaftClusterWithGrpc.java index bd3b137501..47f9e1d4b6 100644 --- a/ratis-grpc/src/test/java/org/apache/ratis/grpc/MiniRaftClusterWithGrpc.java +++ b/ratis-grpc/src/test/java/org/apache/ratis/grpc/MiniRaftClusterWithGrpc.java @@ -64,8 +64,11 @@ default Factory getFactory() { public static final DelayLocalExecutionInjection SEND_SERVER_REQUEST_INJECTION = new DelayLocalExecutionInjection(GrpcService.GRPC_SEND_SERVER_REQUEST); - protected MiniRaftClusterWithGrpc(String[] ids, String[] listenerIds, RaftProperties properties, - Parameters parameters) { + public MiniRaftClusterWithGrpc(String[] ids, RaftProperties properties, Parameters parameters) { + this(ids, new String[0], properties, parameters); + } + + public MiniRaftClusterWithGrpc(String[] ids, String[] listenerIds, RaftProperties properties, Parameters parameters) { super(ids, listenerIds, properties, parameters); } From 8c92d2ca9aed718dcf0738f2bdbd0e763d85718f Mon Sep 17 00:00:00 2001 From: Tsz-Wo Nicholas Sze Date: Fri, 11 Oct 2024 08:00:06 -0700 Subject: [PATCH 364/397] RATIS-2151. TestRaftWithGrpc keeps failing with zero-copy. (#1164) (cherry picked from commit df1d38ac4ca06ad4891f9548a29ad437bef475fd) --- .../apache/ratis/util/DataBlockingQueue.java | 30 +++++ .../server/GrpcClientProtocolService.java | 26 ++--- .../ratis/grpc/server/GrpcLogAppender.java | 22 ++-- .../server/raftlog/segmented/LogSegment.java | 109 ++++++++++++++---- .../raftlog/segmented/SegmentedRaftLog.java | 14 ++- .../segmented/SegmentedRaftLogWorker.java | 29 ++--- .../impl/SimpleStateMachine4Testing.java | 21 ++-- 7 files changed, 176 insertions(+), 75 deletions(-) diff --git a/ratis-common/src/main/java/org/apache/ratis/util/DataBlockingQueue.java b/ratis-common/src/main/java/org/apache/ratis/util/DataBlockingQueue.java index e905893e5b..fb0f0715c5 100644 --- a/ratis-common/src/main/java/org/apache/ratis/util/DataBlockingQueue.java +++ b/ratis-common/src/main/java/org/apache/ratis/util/DataBlockingQueue.java @@ -29,6 +29,7 @@ import java.util.concurrent.locks.Condition; import java.util.concurrent.locks.Lock; import java.util.concurrent.locks.ReentrantLock; +import java.util.function.Consumer; import java.util.function.ToLongFunction; /** @@ -46,6 +47,8 @@ public class DataBlockingQueue extends DataQueue { private final Condition notFull = lock.newCondition(); private final Condition notEmpty = lock.newCondition(); + private boolean closed = false; + public DataBlockingQueue(Object name, SizeInBytes byteLimit, int elementLimit, ToLongFunction getNumBytes) { super(name, byteLimit, elementLimit, getNumBytes); } @@ -72,10 +75,34 @@ public void clear() { } } + /** Apply the given handler to each element and then {@link #clear()}. */ + public void clear(Consumer handler) { + try(AutoCloseableLock auto = AutoCloseableLock.acquire(lock)) { + for(E e : this) { + handler.accept(e); + } + super.clear(); + } + } + + /** + * Close this queue to stop accepting new elements, i.e. the offer(…) methods always return false. + * Note that closing the queue will not clear the existing elements. + * The existing elements can be peeked, polled or cleared after close. + */ + public void close() { + try(AutoCloseableLock ignored = AutoCloseableLock.acquire(lock)) { + closed = true; + } + } + @Override public boolean offer(E element) { Objects.requireNonNull(element, "element == null"); try(AutoCloseableLock auto = AutoCloseableLock.acquire(lock)) { + if (closed) { + return false; + } if (super.offer(element)) { notEmpty.signal(); return true; @@ -95,6 +122,9 @@ public boolean offer(E element, TimeDuration timeout) throws InterruptedExceptio long nanos = timeout.toLong(TimeUnit.NANOSECONDS); try(AutoCloseableLock auto = AutoCloseableLock.acquire(lock)) { for(;;) { + if (closed) { + return false; + } if (super.offer(element)) { notEmpty.signal(); return true; diff --git a/ratis-grpc/src/main/java/org/apache/ratis/grpc/server/GrpcClientProtocolService.java b/ratis-grpc/src/main/java/org/apache/ratis/grpc/server/GrpcClientProtocolService.java index 80a9a439b9..b7548780cd 100644 --- a/ratis-grpc/src/main/java/org/apache/ratis/grpc/server/GrpcClientProtocolService.java +++ b/ratis-grpc/src/main/java/org/apache/ratis/grpc/server/GrpcClientProtocolService.java @@ -29,7 +29,6 @@ import org.apache.ratis.thirdparty.io.grpc.stub.StreamObserver; import org.apache.ratis.proto.RaftProtos.RaftClientReplyProto; import org.apache.ratis.proto.RaftProtos.RaftClientRequestProto; -import org.apache.ratis.proto.RaftProtos.SlidingWindowEntry; import org.apache.ratis.proto.grpc.RaftClientProtocolServiceGrpc.RaftClientProtocolServiceImplBase; import org.apache.ratis.util.CollectionUtils; import org.apache.ratis.util.JavaUtils; @@ -323,18 +322,19 @@ private class UnorderedRequestStreamObserver extends RequestStreamObserver { @Override void processClientRequest(ReferenceCountedObject requestRef) { - final RaftClientRequest request = requestRef.retain(); - final long callId = request.getCallId(); - final SlidingWindowEntry slidingWindowEntry = request.getSlidingWindowEntry(); - final CompletableFuture f = processClientRequest(requestRef, reply -> { - if (!reply.isSuccess()) { - LOG.info("Failed request cid={}, {}, reply={}", callId, slidingWindowEntry, reply); - } - final RaftClientReplyProto proto = ClientProtoUtils.toRaftClientReplyProto(reply); - responseNext(proto); - }); - - requestRef.release(); + final long callId = requestRef.retain().getCallId(); + final CompletableFuture f; + try { + f = processClientRequest(requestRef, reply -> { + if (!reply.isSuccess()) { + LOG.info("Failed request cid={}, reply={}", callId, reply); + } + final RaftClientReplyProto proto = ClientProtoUtils.toRaftClientReplyProto(reply); + responseNext(proto); + }); + } finally { + requestRef.release(); + } put(callId, f); f.thenAccept(dummy -> remove(callId)); diff --git a/ratis-grpc/src/main/java/org/apache/ratis/grpc/server/GrpcLogAppender.java b/ratis-grpc/src/main/java/org/apache/ratis/grpc/server/GrpcLogAppender.java index e75da86236..dd5c365306 100644 --- a/ratis-grpc/src/main/java/org/apache/ratis/grpc/server/GrpcLogAppender.java +++ b/ratis-grpc/src/main/java/org/apache/ratis/grpc/server/GrpcLogAppender.java @@ -392,7 +392,7 @@ public Comparator getCallIdComparator() { } private void appendLog(boolean heartbeat) throws IOException { - ReferenceCountedObject pending = null; + final ReferenceCountedObject pending; final AppendEntriesRequest request; try (AutoCloseableLock writeLock = lock.writeLock(caller, LOG::trace)) { // Prepare and send the append request. @@ -401,18 +401,18 @@ private void appendLog(boolean heartbeat) throws IOException { if (pending == null) { return; } - request = new AppendEntriesRequest(pending.get(), getFollowerId(), grpcServerMetrics); - pendingRequests.put(request); - increaseNextIndex(pending.get()); - if (appendLogRequestObserver == null) { - appendLogRequestObserver = new StreamObservers( - getClient(), new AppendLogResponseHandler(), useSeparateHBChannel, getWaitTimeMin()); - } - } catch(Exception e) { - if (pending != null) { + try { + request = new AppendEntriesRequest(pending.get(), getFollowerId(), grpcServerMetrics); + pendingRequests.put(request); + increaseNextIndex(pending.get()); + if (appendLogRequestObserver == null) { + appendLogRequestObserver = new StreamObservers( + getClient(), new AppendLogResponseHandler(), useSeparateHBChannel, getWaitTimeMin()); + } + } catch (Exception e) { pending.release(); + throw e; } - throw e; } try { diff --git a/ratis-server/src/main/java/org/apache/ratis/server/raftlog/segmented/LogSegment.java b/ratis-server/src/main/java/org/apache/ratis/server/raftlog/segmented/LogSegment.java index 76aef3c19c..816e1b7bfb 100644 --- a/ratis-server/src/main/java/org/apache/ratis/server/raftlog/segmented/LogSegment.java +++ b/ratis-server/src/main/java/org/apache/ratis/server/raftlog/segmented/LogSegment.java @@ -29,6 +29,7 @@ import org.apache.ratis.thirdparty.com.google.common.cache.CacheLoader; import org.apache.ratis.thirdparty.com.google.protobuf.CodedOutputStream; import org.apache.ratis.util.FileUtils; +import org.apache.ratis.util.JavaUtils; import org.apache.ratis.util.Preconditions; import org.apache.ratis.util.SizeInBytes; import org.slf4j.Logger; @@ -37,9 +38,10 @@ import java.io.File; import java.io.IOException; import java.util.Comparator; +import java.util.HashMap; +import java.util.Iterator; import java.util.Map; import java.util.Objects; -import java.util.concurrent.ConcurrentHashMap; import java.util.concurrent.ConcurrentNavigableMap; import java.util.concurrent.ConcurrentSkipListMap; import java.util.concurrent.atomic.AtomicInteger; @@ -283,47 +285,103 @@ public ReferenceCountedObject load(LogRecord key) throws IOExcept final TermIndex ti = TermIndex.valueOf(entry); putEntryCache(ti, entry, Op.LOAD_SEGMENT_FILE); if (ti.equals(key.getTermIndex())) { - entryRef.retain(); toReturn.set(entryRef); + } else { + entryRef.release(); } - entryRef.release(); }); loadingTimes.incrementAndGet(); return Objects.requireNonNull(toReturn.get()); } } - static class EntryCache { - private final Map> map = new ConcurrentHashMap<>(); + private static class Item { + private final AtomicReference> ref; + private final long serializedSize; + + Item(ReferenceCountedObject obj, long serializedSize) { + this.ref = new AtomicReference<>(obj); + this.serializedSize = serializedSize; + } + + ReferenceCountedObject get() { + return ref.get(); + } + + long release() { + final ReferenceCountedObject entry = ref.getAndSet(null); + if (entry == null) { + return 0; + } + entry.release(); + return serializedSize; + } + } + + class EntryCache { + private Map map = new HashMap<>(); private final AtomicLong size = new AtomicLong(); + @Override + public String toString() { + return JavaUtils.getClassSimpleName(getClass()) + "-" + LogSegment.this; + } + long size() { return size.get(); } - ReferenceCountedObject get(TermIndex ti) { - return map.get(ti); + synchronized ReferenceCountedObject get(TermIndex ti) { + if (map == null) { + return null; + } + final Item ref = map.get(ti); + return ref == null? null: ref.get(); } - void clear() { - map.values().forEach(ReferenceCountedObject::release); - map.clear(); - size.set(0); + /** After close(), the cache CANNOT be used again. */ + synchronized void close() { + if (map == null) { + return; + } + evict(); + map = null; + LOG.info("Successfully closed {}", this); } - void put(TermIndex key, ReferenceCountedObject valueRef, Op op) { + /** After evict(), the cache can be used again. */ + synchronized void evict() { + if (map == null) { + return; + } + for (Iterator> i = map.entrySet().iterator(); i.hasNext(); i.remove()) { + release(i.next().getValue()); + } + } + + synchronized void put(TermIndex key, ReferenceCountedObject valueRef, Op op) { + if (map == null) { + return; + } valueRef.retain(); - Optional.ofNullable(map.put(key, valueRef)).ifPresent(this::release); - size.getAndAdd(getEntrySize(valueRef.get(), op)); + final long serializedSize = getEntrySize(valueRef.get(), op); + release(map.put(key, new Item(valueRef, serializedSize))); + size.getAndAdd(serializedSize); } - private void release(ReferenceCountedObject entry) { - size.getAndAdd(-getEntrySize(entry.get(), Op.REMOVE_CACHE)); - entry.release(); + private void release(Item ref) { + if (ref == null) { + return; + } + final long serializedSize = ref.release(); + size.getAndAdd(-serializedSize); } - void remove(TermIndex key) { - Optional.ofNullable(map.remove(key)).ifPresent(this::release); + synchronized void remove(TermIndex key) { + if (map == null) { + return; + } + release(map.remove(key)); } } @@ -433,7 +491,13 @@ ReferenceCountedObject getEntryFromCache(TermIndex ti) { synchronized ReferenceCountedObject loadCache(LogRecord record) throws RaftLogIOException { ReferenceCountedObject entry = entryCache.get(record.getTermIndex()); if (entry != null) { - return entry; + try { + entry.retain(); + return entry; + } catch (IllegalStateException ignored) { + // The entry could be removed from the cache and released. + // The exception can be safely ignored since it is the same as cache miss. + } } try { return cacheLoader.load(record); @@ -507,7 +571,7 @@ private int compareTo(Long l) { synchronized void clear() { records.clear(); - evictCache(); + entryCache.close(); endIndex = startIndex - 1; } @@ -516,8 +580,7 @@ int getLoadingTimes() { } void evictCache() { - entryCache.clear(); - totalCacheSize.set(0); + entryCache.evict(); } void putEntryCache(TermIndex key, LogEntryProto value, Op op) { diff --git a/ratis-server/src/main/java/org/apache/ratis/server/raftlog/segmented/SegmentedRaftLog.java b/ratis-server/src/main/java/org/apache/ratis/server/raftlog/segmented/SegmentedRaftLog.java index 450450f13f..6e9a43aa5d 100644 --- a/ratis-server/src/main/java/org/apache/ratis/server/raftlog/segmented/SegmentedRaftLog.java +++ b/ratis-server/src/main/java/org/apache/ratis/server/raftlog/segmented/SegmentedRaftLog.java @@ -312,9 +312,14 @@ public ReferenceCountedObject retainLog(long index) throws RaftLo } final ReferenceCountedObject entry = segment.getEntryFromCache(record.getTermIndex()); if (entry != null) { - getRaftLogMetrics().onRaftLogCacheHit(); - entry.retain(); - return entry; + try { + entry.retain(); + getRaftLogMetrics().onRaftLogCacheHit(); + return entry; + } catch (IllegalStateException ignored) { + // The entry could be removed from the cache and released. + // The exception can be safely ignored since it is the same as cache miss. + } } // the entry is not in the segment's cache. Load the cache without holding the lock. @@ -352,6 +357,7 @@ public ReferenceCountedObject retainEntryWithData(long index) thr } catch (Exception e) { final String err = getName() + ": Failed readStateMachineData for " + toLogEntryString(entry); LOG.error(err, e); + entryRef.release(); throw new RaftLogIOException(err, JavaUtils.unwrapCompletionException(e)); } } @@ -565,6 +571,7 @@ public CompletableFuture onSnapshotInstalled(long lastSnapshotIndex) { @Override public void close() throws IOException { try(AutoCloseableLock writeLock = writeLock()) { + LOG.info("Start closing {}", this); super.close(); cacheEviction.close(); cache.close(); @@ -572,6 +579,7 @@ public void close() throws IOException { fileLogWorker.close(); storage.close(); getRaftLogMetrics().unregister(); + LOG.info("Successfully closed {}", this); } SegmentedRaftLogCache getRaftLogCache() { diff --git a/ratis-server/src/main/java/org/apache/ratis/server/raftlog/segmented/SegmentedRaftLogWorker.java b/ratis-server/src/main/java/org/apache/ratis/server/raftlog/segmented/SegmentedRaftLogWorker.java index 183c0e381a..98b20bade3 100644 --- a/ratis-server/src/main/java/org/apache/ratis/server/raftlog/segmented/SegmentedRaftLogWorker.java +++ b/ratis-server/src/main/java/org/apache/ratis/server/raftlog/segmented/SegmentedRaftLogWorker.java @@ -51,9 +51,8 @@ import java.util.Objects; import java.util.Optional; import java.util.Queue; -import java.util.concurrent.CompletableFuture; -import java.util.concurrent.ExecutorService; -import java.util.concurrent.TimeUnit; +import java.util.concurrent.*; +import java.util.concurrent.atomic.AtomicReference; import java.util.function.Consumer; import java.util.function.Supplier; @@ -245,10 +244,11 @@ void start(long latestIndex, long evictIndex, File openSegmentFile) throws IOExc } void close() { + queue.close(); this.running = false; + ConcurrentUtils.shutdownAndWait(TimeDuration.ONE_MINUTE, workerThreadExecutor, + timeout -> LOG.warn("{}: shutdown timeout in {}", name, timeout)); Optional.ofNullable(flushExecutor).ifPresent(ExecutorService::shutdown); - ConcurrentUtils.shutdownAndWait(TimeDuration.ONE_SECOND.multiply(3), - workerThreadExecutor, timeout -> LOG.warn("{}: shutdown timeout in " + timeout, name)); IOUtils.cleanup(LOG, out); PlatformDependent.freeDirectBuffer(writeBuffer); LOG.info("{} close()", name); @@ -344,7 +344,7 @@ private void run() { LOG.info(Thread.currentThread().getName() + " was interrupted, exiting. There are " + queue.getNumElements() + " tasks remaining in the queue."); - return; + break; } catch (Exception e) { if (!running) { LOG.info("{} got closed and hit exception", @@ -355,6 +355,8 @@ private void run() { } } } + + queue.clear(Task::discard); } private boolean shouldFlush() { @@ -499,7 +501,7 @@ private class WriteLog extends Task { private final LogEntryProto entry; private final CompletableFuture stateMachineFuture; private final CompletableFuture combined; - private final ReferenceCountedObject ref; + private final AtomicReference> ref = new AtomicReference<>(); WriteLog(ReferenceCountedObject entryRef, LogEntryProto removedStateMachineData, TransactionContext context) { @@ -517,7 +519,7 @@ private class WriteLog extends Task { this.stateMachineFuture = null; } entryRef.retain(); - this.ref = entryRef; + this.ref.set(entryRef); } else { try { // this.entry != origEntry if it has state machine data @@ -527,7 +529,6 @@ private class WriteLog extends Task { + ", entry=" + LogProtoUtils.toLogEntryString(origEntry, stateMachine::toStateMachineLogEntryString), e); throw e; } - this.ref = null; } this.combined = stateMachineFuture == null? super.getFuture() : super.getFuture().thenCombine(stateMachineFuture, (index, stateMachineResult) -> index); @@ -537,6 +538,7 @@ private class WriteLog extends Task { void failed(IOException e) { stateMachine.event().notifyLogFailed(e, entry); super.failed(e); + discard(); } @Override @@ -552,15 +554,14 @@ CompletableFuture getFuture() { @Override void done() { writeTasks.offerOrCompleteFuture(this); - if (ref != null) { - ref.release(); - } + discard(); } @Override void discard() { - if (ref != null) { - ref.release(); + final ReferenceCountedObject entryRef = ref.getAndSet(null); + if (entryRef != null) { + entryRef.release(); } } diff --git a/ratis-server/src/test/java/org/apache/ratis/statemachine/impl/SimpleStateMachine4Testing.java b/ratis-server/src/test/java/org/apache/ratis/statemachine/impl/SimpleStateMachine4Testing.java index 71fbe216ca..3a7c8aa774 100644 --- a/ratis-server/src/test/java/org/apache/ratis/statemachine/impl/SimpleStateMachine4Testing.java +++ b/ratis-server/src/test/java/org/apache/ratis/statemachine/impl/SimpleStateMachine4Testing.java @@ -83,7 +83,7 @@ public static SimpleStateMachine4Testing get(RaftServer.Division s) { return (SimpleStateMachine4Testing)s.getStateMachine(); } - private final SortedMap> indexMap = Collections.synchronizedSortedMap(new TreeMap<>()); + private final SortedMap indexMap = Collections.synchronizedSortedMap(new TreeMap<>()); private final SortedMap dataMap = Collections.synchronizedSortedMap(new TreeMap<>()); private final Daemon checkpointer; private final SimpleStateMachineStorage storage = new SimpleStateMachineStorage(); @@ -198,9 +198,8 @@ public RoleInfoProto getLeaderElectionTimeoutInfo() { return leaderElectionTimeoutInfo; } - private void put(ReferenceCountedObject entryRef) { - LogEntryProto entry = entryRef.retain(); - final ReferenceCountedObject previous = indexMap.put(entry.getIndex(), entryRef); + private void put(LogEntryProto entry) { + final LogEntryProto previous = indexMap.put(entry.getIndex(), entry); Preconditions.assertNull(previous, "previous"); final String s = entry.getStateMachineLogEntry().getLogData().toStringUtf8(); dataMap.put(s, entry); @@ -253,7 +252,7 @@ public CompletableFuture applyTransaction(TransactionContext trx) { LogEntryProto entry = entryRef.get(); LOG.info("applyTransaction for log index {}", entry.getIndex()); - put(entryRef); + put(LogProtoUtils.copy(entry)); updateLastAppliedTermIndex(entry.getTerm(), entry.getIndex()); final SimpleMessage m = new SimpleMessage(entry.getIndex() + " OK"); @@ -273,8 +272,7 @@ public long takeSnapshot() { LOG.debug("Taking a snapshot with {}, file:{}", termIndex, snapshotFile); try (SegmentedRaftLogOutputStream out = new SegmentedRaftLogOutputStream(snapshotFile, false, segmentMaxSize, preallocatedSize, ByteBuffer.allocateDirect(bufferSize))) { - for (final ReferenceCountedObject entryRef : indexMap.values()) { - LogEntryProto entry = entryRef.get(); + for (final LogEntryProto entry : indexMap.values()) { if (entry.getIndex() > endIndex) { break; } else { @@ -309,7 +307,7 @@ private synchronized void loadSnapshot(SingleFileSnapshotInfo snapshot) throws I snapshot.getFile().getPath().toFile(), 0, endIndex, false)) { LogEntryProto entry; while ((entry = in.nextEntry()) != null) { - put(ReferenceCountedObject.wrap(entry)); + put(entry); updateLastAppliedTermIndex(entry.getTerm(), entry.getIndex()); } } @@ -338,7 +336,7 @@ public CompletableFuture query(Message request) { LOG.info("query {}, all available: {}", string, dataMap.keySet()); final LogEntryProto entry = dataMap.get(string); if (entry != null) { - return CompletableFuture.completedFuture(Message.valueOf(entry.toByteString())); + return CompletableFuture.completedFuture(Message.valueOf(entry)); } exception = new IndexOutOfBoundsException(getId() + ": LogEntry not found for query " + string); } catch (Exception e) { @@ -384,11 +382,12 @@ public void close() { running = false; checkpointer.interrupt(); }); - indexMap.values().forEach(ReferenceCountedObject::release); + indexMap.clear(); + dataMap.clear(); } public LogEntryProto[] getContent() { - return indexMap.values().stream().map(ReferenceCountedObject::get).toArray(LogEntryProto[]::new); + return indexMap.values().toArray(new LogEntryProto[0]); } public void blockStartTransaction() { From 2c5ba675e23271d65b2734c782c5f844dd094246 Mon Sep 17 00:00:00 2001 From: Tsz-Wo Nicholas Sze Date: Wed, 16 Oct 2024 16:49:24 -0700 Subject: [PATCH 365/397] RATIS-2164. LeakDetector has a race condition. (#1163) (cherry picked from commit a15bde173573eb3ec2dda90f0d99204bf7056eba) --- .../org/apache/ratis/util/LeakDetector.java | 122 ++++++--- .../util/ReferenceCountedLeakDetector.java | 248 ++++++++++++++---- .../ratis/util/ReferenceCountedObject.java | 9 - .../ratis/grpc/MiniRaftClusterWithGrpc.java | 5 +- .../ratis/server/impl/MiniRaftCluster.java | 40 ++- 5 files changed, 321 insertions(+), 103 deletions(-) diff --git a/ratis-common/src/main/java/org/apache/ratis/util/LeakDetector.java b/ratis-common/src/main/java/org/apache/ratis/util/LeakDetector.java index 82202f2884..a6b2ec28bd 100644 --- a/ratis-common/src/main/java/org/apache/ratis/util/LeakDetector.java +++ b/ratis-common/src/main/java/org/apache/ratis/util/LeakDetector.java @@ -22,10 +22,14 @@ import java.lang.ref.ReferenceQueue; import java.lang.ref.WeakReference; +import java.util.ArrayList; import java.util.Collections; +import java.util.HashMap; +import java.util.List; import java.util.Set; -import java.util.concurrent.ConcurrentHashMap; import java.util.concurrent.atomic.AtomicLong; +import java.util.function.Consumer; +import java.util.function.Supplier; /** * Simple general resource leak detector using {@link ReferenceQueue} and {@link java.lang.ref.WeakReference} to @@ -55,13 +59,61 @@ */ public class LeakDetector { private static final Logger LOG = LoggerFactory.getLogger(LeakDetector.class); + + private static class LeakTrackerSet { + private final Set set = Collections.newSetFromMap(new HashMap<>()); + + synchronized boolean remove(LeakTracker tracker) { + return set.remove(tracker); + } + + synchronized void removeExisting(LeakTracker tracker) { + final boolean removed = set.remove(tracker); + Preconditions.assertTrue(removed, () -> "Failed to remove existing " + tracker); + } + + synchronized LeakTracker add(Object referent, ReferenceQueue queue, Supplier leakReporter) { + final LeakTracker tracker = new LeakTracker(referent, queue, this::removeExisting, leakReporter); + final boolean added = set.add(tracker); + Preconditions.assertTrue(added, () -> "Failed to add " + tracker + " for " + referent); + return tracker; + } + + synchronized int getNumLeaks(boolean throwException) { + if (set.isEmpty()) { + return 0; + } + + int n = 0; + for (LeakTracker tracker : set) { + if (tracker.reportLeak() != null) { + n++; + } + } + if (throwException) { + assertNoLeaks(n); + } + return n; + } + + synchronized void assertNoLeaks(int leaks) { + Preconditions.assertTrue(leaks == 0, () -> { + final int size = set.size(); + return "#leaks = " + leaks + " > 0, #leaks " + (leaks == size? "==" : "!=") + " set.size = " + size; + }); + } + } + private static final AtomicLong COUNTER = new AtomicLong(); private final ReferenceQueue queue = new ReferenceQueue<>(); - private final Set allLeaks = Collections.newSetFromMap(new ConcurrentHashMap<>()); + /** All the {@link LeakTracker}s. */ + private final LeakTrackerSet trackers = new LeakTrackerSet(); + /** When a leak is discovered, a message is printed and added to this list. */ + private final List leakMessages = Collections.synchronizedList(new ArrayList<>()); private final String name; - public LeakDetector(String name) { + LeakDetector(String name) { this.name = name + COUNTER.getAndIncrement(); } @@ -80,8 +132,11 @@ private void run() { LeakTracker tracker = (LeakTracker) queue.remove(); // Original resource already been GCed, if tracker is not closed yet, // report a leak. - if (allLeaks.remove(tracker)) { - tracker.reportLeak(); + if (trackers.remove(tracker)) { + final String leak = tracker.reportLeak(); + if (leak != null) { + leakMessages.add(leak); + } } } catch (InterruptedException e) { Thread.currentThread().interrupt(); @@ -93,48 +148,51 @@ private void run() { LOG.warn("Exiting leak detector {}.", name); } - public UncheckedAutoCloseable track(Object leakable, Runnable reportLeak) { - // A rate filter can be put here to only track a subset of all objects, e.g. 5%, 10%, + Runnable track(Object leakable, Supplier reportLeak) { + // TODO: A rate filter can be put here to only track a subset of all objects, e.g. 5%, 10%, // if we have proofs that leak tracking impacts performance, or a single LeakDetector // thread can't keep up with the pace of object allocation. // For now, it looks effective enough and let keep it simple. - LeakTracker tracker = new LeakTracker(leakable, queue, allLeaks, reportLeak); - allLeaks.add(tracker); - return tracker; + return trackers.add(leakable, queue, reportLeak)::remove; } - public void assertNoLeaks() { - Preconditions.assertTrue(allLeaks.isEmpty(), this::allLeaksString); - } + public void assertNoLeaks(int maxRetries, TimeDuration retrySleep) throws InterruptedException { + synchronized (leakMessages) { + // leakMessages are all the leaks discovered so far. + Preconditions.assertTrue(leakMessages.isEmpty(), + () -> "#leaks = " + leakMessages.size() + "\n" + leakMessages); + } - String allLeaksString() { - if (allLeaks.isEmpty()) { - return "allLeaks = "; + for(int i = 0; i < maxRetries; i++) { + final int numLeaks = trackers.getNumLeaks(false); + if (numLeaks == 0) { + return; + } + LOG.warn("{}/{}) numLeaks == {} > 0, will wait and retry ...", i, maxRetries, numLeaks); + retrySleep.sleep(); } - allLeaks.forEach(LeakTracker::reportLeak); - return "allLeaks.size = " + allLeaks.size(); + trackers.getNumLeaks(true); } - private static final class LeakTracker extends WeakReference implements UncheckedAutoCloseable { - private final Set allLeaks; - private final Runnable leakReporter; + private static final class LeakTracker extends WeakReference { + private final Consumer removeMethod; + private final Supplier getLeakMessage; + LeakTracker(Object referent, ReferenceQueue referenceQueue, - Set allLeaks, Runnable leakReporter) { + Consumer removeMethod, Supplier getLeakMessage) { super(referent, referenceQueue); - this.allLeaks = allLeaks; - this.leakReporter = leakReporter; + this.removeMethod = removeMethod; + this.getLeakMessage = getLeakMessage; } - /** - * Called by the tracked resource when closing. - */ - @Override - public void close() { - allLeaks.remove(this); + /** Called by the tracked resource when the object is completely released. */ + void remove() { + removeMethod.accept(this); } - void reportLeak() { - leakReporter.run(); + /** @return the leak message if there is a leak; return null if there is no leak. */ + String reportLeak() { + return getLeakMessage.get(); } } } diff --git a/ratis-common/src/main/java/org/apache/ratis/util/ReferenceCountedLeakDetector.java b/ratis-common/src/main/java/org/apache/ratis/util/ReferenceCountedLeakDetector.java index 32abe805f1..acf6fb8cfc 100644 --- a/ratis-common/src/main/java/org/apache/ratis/util/ReferenceCountedLeakDetector.java +++ b/ratis-common/src/main/java/org/apache/ratis/util/ReferenceCountedLeakDetector.java @@ -20,7 +20,7 @@ import org.slf4j.Logger; import org.slf4j.LoggerFactory; -import java.util.LinkedList; +import java.util.ArrayList; import java.util.List; import java.util.concurrent.atomic.AtomicInteger; import java.util.concurrent.atomic.AtomicReference; @@ -49,7 +49,7 @@ public static LeakDetector getLeakDetector() { private ReferenceCountedLeakDetector() { } - static synchronized void enable(boolean advanced) { + public static synchronized void enable(boolean advanced) { FACTORY.set(advanced ? Mode.ADVANCED : Mode.SIMPLE); } @@ -108,6 +108,10 @@ public V get() { return value; } + final int getCount() { + return count.get(); + } + @Override public V retain() { // n < 0: exception @@ -138,85 +142,221 @@ public boolean release() { } private static class SimpleTracing extends Impl { - private final UncheckedAutoCloseable leakTracker; + private final LeakDetector leakDetector; + private final Class valueClass; + private String valueString = null; + private Runnable removeMethod = null; SimpleTracing(T value, Runnable retainMethod, Consumer releaseMethod, LeakDetector leakDetector) { super(value, retainMethod, releaseMethod); - final Class clazz = value.getClass(); - this.leakTracker = leakDetector.track(this, - () -> LOG.warn("LEAK: A {} is not released properly", clazz.getName())); + this.valueClass = value.getClass(); + this.leakDetector = leakDetector; + } + + String getTraceString(int count) { + return "(" + valueClass + ", count=" + count + ", value=" + valueString + ")"; + } + + /** @return the leak message if there is a leak; return null if there is no leak. */ + String logLeakMessage() { + final int count = getCount(); + if (count == 0) { // never retain + return null; + } + final String message = "LEAK: " + getTraceString(count); + LOG.warn(message); + return message; } @Override - public boolean release() { - boolean released = super.release(); + public synchronized T get() { + try { + return super.get(); + } catch (Exception e) { + throw new IllegalStateException("Failed to get: " + getTraceString(getCount()), e); + } + } + + @Override + public synchronized T retain() { + final T value; + try { + value = super.retain(); + } catch (Exception e) { + throw new IllegalStateException("Failed to retain: " + getTraceString(getCount()), e); + } + if (getCount() == 1) { // this is the first retain + this.removeMethod = leakDetector.track(this, this::logLeakMessage); + this.valueString = value.toString(); + } + return value; + } + + @Override + public synchronized boolean release() { + final boolean released; + try { + released = super.release(); + } catch (Exception e) { + throw new IllegalStateException("Failed to release: " + getTraceString(getCount()), e); + } + if (released) { - leakTracker.close(); + Preconditions.assertNotNull(removeMethod, () -> "Not yet retained (removeMethod == null): " + valueClass); + removeMethod.run(); } return released; } } - private static class AdvancedTracing extends Impl { - private final UncheckedAutoCloseable leakTracker; - private final List retainsTraces; - private final List releaseTraces; + private static class AdvancedTracing extends SimpleTracing { + enum Op {CREATION, RETAIN, RELEASE, CURRENT} - AdvancedTracing(T value, Runnable retainMethod, Consumer releaseMethod, LeakDetector leakDetector) { - super(value, retainMethod, releaseMethod); + static class Counts { + private final int refCount; + private final int retainCount; + private final int releaseCount; + + Counts() { + this.refCount = 0; + this.retainCount = 0; + this.releaseCount = 0; + } + + Counts(Op op, Counts previous) { + if (op == Op.RETAIN) { + this.refCount = previous.refCount + 1; + this.retainCount = previous.retainCount + 1; + this.releaseCount = previous.releaseCount; + } else if (op == Op.RELEASE) { + this.refCount = previous.refCount - 1; + this.retainCount = previous.retainCount; + this.releaseCount = previous.releaseCount + 1; + } else { + throw new IllegalStateException("Unexpected op: " + op); + } + } + + @Override + public String toString() { + return "refCount=" + refCount + + ", retainCount=" + retainCount + + ", releaseCount=" + releaseCount; + } + } + + static class TraceInfo { + private final int id; + private final Op op; + private final int previousRefCount; + private final Counts counts; + + private final String threadInfo; + private final StackTraceElement[] stackTraces; + private final int newTraceElementIndex; + + TraceInfo(int id, Op op, TraceInfo previous, int previousRefCount) { + this.id = id; + this.op = op; + this.previousRefCount = previousRefCount; + this.counts = previous == null? new Counts() + : op == Op.CURRENT ? previous.counts + : new Counts(op, previous.counts); + + final Thread thread = Thread.currentThread(); + this.threadInfo = "Thread_" + thread.getId() + ":" + thread.getName(); + this.stackTraces = thread.getStackTrace(); + this.newTraceElementIndex = previous == null? stackTraces.length - 1 + : findFirstUnequalFromTail(this.stackTraces, previous.stackTraces); + } + + static int findFirstUnequalFromTail(T[] current, T[] previous) { + int c = current.length - 1; + for(int p = previous.length - 1; p >= 0; p--, c--) { + if (!previous[p].equals(current[c])) { + return c; + } + } + return -1; + } + + private StringBuilder appendTo(StringBuilder b) { + b.append(op).append("_").append(id) + .append(": previousRefCount=").append(previousRefCount) + .append(", ").append(counts) + .append(", ").append(threadInfo).append("\n"); + final int n = newTraceElementIndex + 1; + int line = 3; + for (; line <= n && line < stackTraces.length; line++) { + b.append(" ").append(stackTraces[line]).append("\n"); + } + if (line < stackTraces.length) { + b.append(" ...\n"); + } + return b; + } + + @Override + public String toString() { + return appendTo(new StringBuilder()).toString(); + } + } - StackTraceElement[] createStrace = Thread.currentThread().getStackTrace(); - final Class clazz = value.getClass(); - final List localRetainsTraces = new LinkedList<>(); - final List localReleaseTraces = new LinkedList<>(); + private final List traceInfos = new ArrayList<>(); + private TraceInfo previous; - this.leakTracker = leakDetector.track(this, () -> - LOG.warn("LEAK: A {} is not released properly.\nCreation trace:\n{}\n" + - "Retain traces({}):\n{}\nRelease traces({}):\n{}", - clazz.getName(), formatStackTrace(createStrace, 3), - localRetainsTraces.size(), formatStackTraces(localRetainsTraces, 2), - localReleaseTraces.size(), formatStackTraces(localReleaseTraces, 2))); + AdvancedTracing(T value, Runnable retainMethod, Consumer releaseMethod, LeakDetector leakDetector) { + super(value, retainMethod, releaseMethod, leakDetector); + addTraceInfo(Op.CREATION, -1); + } - this.retainsTraces = localRetainsTraces; - this.releaseTraces = localReleaseTraces; + private synchronized TraceInfo addTraceInfo(Op op, int previousRefCount) { + final TraceInfo current = new TraceInfo(traceInfos.size(), op, previous, previousRefCount); + traceInfos.add(current); + previous = current; + return current; } + @Override - public T retain() { - T retain = super.retain(); - retainsTraces.add(Thread.currentThread().getStackTrace()); - return retain; + public synchronized T retain() { + final int previousRefCount = getCount(); + final T retained = super.retain(); + final TraceInfo info = addTraceInfo(Op.RETAIN, previousRefCount); + Preconditions.assertSame(getCount(), info.counts.refCount, "refCount"); + return retained; } @Override - public boolean release() { - boolean released = super.release(); - if (released) { - leakTracker.close(); - } - releaseTraces.add(Thread.currentThread().getStackTrace()); + public synchronized boolean release() { + final int previousRefCount = getCount(); + final boolean released = super.release(); + final TraceInfo info = addTraceInfo(Op.RELEASE, previousRefCount); + final int count = getCount(); + final int expected = count == -1? 0 : count; + Preconditions.assertSame(expected, info.counts.refCount, "refCount"); return released; } - } - private static String formatStackTrace(StackTraceElement[] stackTrace, int startIdx) { - final StringBuilder sb = new StringBuilder(); - for (int line = startIdx; line < stackTrace.length; line++) { - sb.append(stackTrace[line]).append("\n"); + @Override + synchronized String getTraceString(int count) { + return super.getTraceString(count) + getTraceInfosString(); } - return sb.toString(); - } - private static String formatStackTraces(List stackTraces, int startIdx) { - final StringBuilder sb = new StringBuilder(); - stackTraces.forEach(stackTrace -> { - if (sb.length() > 0) { - sb.append("\n"); - } - for (int line = startIdx; line < stackTrace.length; line++) { - sb.append(stackTrace[line]).append("\n"); + private String getTraceInfosString() { + final int n = traceInfos.size(); + final StringBuilder b = new StringBuilder(n << 10).append(" #TraceInfos=").append(n); + TraceInfo last = null; + for (TraceInfo info : traceInfos) { + info.appendTo(b.append("\n")); + last = info; } - }); - return sb.toString(); + + // append current track info + final TraceInfo current = new TraceInfo(n, Op.CURRENT, last, getCount()); + current.appendTo(b.append("\n")); + + return b.toString(); + } } } diff --git a/ratis-common/src/main/java/org/apache/ratis/util/ReferenceCountedObject.java b/ratis-common/src/main/java/org/apache/ratis/util/ReferenceCountedObject.java index b2c53182d3..1fc72c3445 100644 --- a/ratis-common/src/main/java/org/apache/ratis/util/ReferenceCountedObject.java +++ b/ratis-common/src/main/java/org/apache/ratis/util/ReferenceCountedObject.java @@ -182,13 +182,4 @@ static ReferenceCountedObject wrap(V value, Runnable retainMethod, Consum static ReferenceCountedObject wrap(V value, Runnable retainMethod, Runnable releaseMethod) { return wrap(value, retainMethod, ignored -> releaseMethod.run()); } - - static void enableLeakDetection() { - ReferenceCountedLeakDetector.enable(false); - } - - static void enableAdvancedLeakDetection() { - ReferenceCountedLeakDetector.enable(true); - } - } diff --git a/ratis-grpc/src/test/java/org/apache/ratis/grpc/MiniRaftClusterWithGrpc.java b/ratis-grpc/src/test/java/org/apache/ratis/grpc/MiniRaftClusterWithGrpc.java index 47f9e1d4b6..fe12e29f11 100644 --- a/ratis-grpc/src/test/java/org/apache/ratis/grpc/MiniRaftClusterWithGrpc.java +++ b/ratis-grpc/src/test/java/org/apache/ratis/grpc/MiniRaftClusterWithGrpc.java @@ -32,7 +32,7 @@ import org.apache.ratis.server.impl.MiniRaftCluster; import org.apache.ratis.server.impl.RaftServerTestUtil; import org.apache.ratis.util.NetUtils; -import org.apache.ratis.util.ReferenceCountedObject; +import org.apache.ratis.util.ReferenceCountedLeakDetector; import org.junit.Assert; import java.util.Optional; @@ -51,7 +51,8 @@ public MiniRaftClusterWithGrpc newCluster(String[] ids, String[] listenerIds, Ra }; static { - ReferenceCountedObject.enableLeakDetection(); + // TODO move it to MiniRaftCluster for detecting non-gRPC cases + ReferenceCountedLeakDetector.enable(false); } public interface FactoryGet extends Factory.Get { diff --git a/ratis-server/src/test/java/org/apache/ratis/server/impl/MiniRaftCluster.java b/ratis-server/src/test/java/org/apache/ratis/server/impl/MiniRaftCluster.java index bc432c10f4..4f83549b91 100644 --- a/ratis-server/src/test/java/org/apache/ratis/server/impl/MiniRaftCluster.java +++ b/ratis-server/src/test/java/org/apache/ratis/server/impl/MiniRaftCluster.java @@ -138,17 +138,27 @@ default void runWithNewCluster(int numServers, int numListeners, boolean startCl final StackTraceElement caller = JavaUtils.getCallerStackTraceElement(); LOG.info("Running {}", caller.getMethodName()); final CLUSTER cluster = newCluster(numServers, numListeners); + Throwable failed = null; try { if (startCluster) { cluster.start(); } testCase.accept(cluster); - } catch(Exception t) { + } catch(Throwable t) { LOG.info(cluster.printServers()); - LOG.error("Failed {}", caller, t); + LOG.error("Failed " + caller, t); + failed = t; throw t; } finally { - cluster.shutdown(); + try { + cluster.shutdown(); + } catch (Exception e) { + if (failed == null) { + throw e; + } else { + failed.addSuppressed(e); + } + } } } @@ -809,10 +819,24 @@ public void shutdown() { final ExecutorService executor = Executors.newFixedThreadPool(servers.size(), (t) -> Daemon.newBuilder().setName("MiniRaftCluster-" + THREAD_COUNT.incrementAndGet()).setRunnable(t).build()); getServers().forEach(proxy -> executor.submit(() -> JavaUtils.runAsUnchecked(proxy::close))); + final int maxRetries = 30; + final TimeDuration retrySleep = TimeDuration.ONE_SECOND; try { executor.shutdown(); // just wait for a few seconds - executor.awaitTermination(5, TimeUnit.SECONDS); + boolean terminated = false; + + for(int i = 0; i < maxRetries && !terminated; ) { + terminated = executor.awaitTermination(retrySleep.getDuration(), retrySleep.getUnit()); + if (!terminated) { + i++; + if (i < maxRetries) { + LOG.warn("{}/{}: Not yet able to shutdown executor, will wait again ...", i, maxRetries); + } else { + LOG.error("Failed to shutdown executor, some servers may be still running:\n{}", printServers()); + } + } + } } catch (InterruptedException e) { LOG.warn("shutdown interrupted", e); Thread.currentThread().interrupt(); @@ -826,9 +850,13 @@ public void shutdown() { try { RaftTestUtil.gc(); } catch (InterruptedException e) { - LOG.info("gc interrupted."); + LOG.warn("gc interrupted.", e); + } + try { + ReferenceCountedLeakDetector.getLeakDetector().assertNoLeaks(maxRetries, retrySleep); + } catch (InterruptedException e) { + LOG.warn("LeakDetector interrupted.", e); } - ReferenceCountedLeakDetector.getLeakDetector().assertNoLeaks(); } /** From f2decccab45c48db64b6ad93908e5c5a8bb10ff3 Mon Sep 17 00:00:00 2001 From: Tsz-Wo Nicholas Sze Date: Wed, 16 Oct 2024 18:06:59 -0700 Subject: [PATCH 366/397] RATIS-2168. Support custom gRPC services. (#1169) (cherry picked from commit e96ed1a33840385446f4e647864a169467da5ab7) --- .../org/apache/ratis/grpc/GrpcConfigKeys.java | 19 + .../org/apache/ratis/grpc/GrpcFactory.java | 28 +- .../apache/ratis/grpc/server/GrpcService.java | 436 ------------------ .../ratis/grpc/server/GrpcServicesImpl.java | 135 +++--- .../ratis/grpc/MiniRaftClusterWithGrpc.java | 6 +- .../grpc/server/TestGrpcMessageMetrics.java | 1 - 6 files changed, 116 insertions(+), 509 deletions(-) delete mode 100644 ratis-grpc/src/main/java/org/apache/ratis/grpc/server/GrpcService.java diff --git a/ratis-grpc/src/main/java/org/apache/ratis/grpc/GrpcConfigKeys.java b/ratis-grpc/src/main/java/org/apache/ratis/grpc/GrpcConfigKeys.java index e495c4c5c5..4ed9b5f5b8 100644 --- a/ratis-grpc/src/main/java/org/apache/ratis/grpc/GrpcConfigKeys.java +++ b/ratis-grpc/src/main/java/org/apache/ratis/grpc/GrpcConfigKeys.java @@ -272,6 +272,25 @@ static boolean zeroCopyEnabled(RaftProperties properties) { static void setZeroCopyEnabled(RaftProperties properties, boolean enabled) { setBoolean(properties::setBoolean, ZERO_COPY_ENABLED_KEY, enabled); } + + String SERVICES_CUSTOMIZER_PARAMETER = PREFIX + ".services.customizer"; + Class SERVICES_CUSTOMIZER_CLASS = GrpcServices.Customizer.class; + static GrpcServices.Customizer servicesCustomizer(Parameters parameters) { + return parameters == null ? null + : parameters.get(SERVICES_CUSTOMIZER_PARAMETER, SERVICES_CUSTOMIZER_CLASS); + } + static void setServicesCustomizer(Parameters parameters, GrpcServices.Customizer customizer) { + parameters.put(SERVICES_CUSTOMIZER_PARAMETER, customizer, SERVICES_CUSTOMIZER_CLASS); + } + + String TLS_CONF_PARAMETER = PREFIX + ".tls.conf"; + Class TLS_CONF_CLASS = TLS.CONF_CLASS; + static GrpcTlsConfig tlsConf(Parameters parameters) { + return parameters != null ? parameters.get(TLS_CONF_PARAMETER, TLS_CONF_CLASS): null; + } + static void setTlsConf(Parameters parameters, GrpcTlsConfig conf) { + parameters.put(TLS_CONF_PARAMETER, conf, TLS_CONF_CLASS); + } } String MESSAGE_SIZE_MAX_KEY = PREFIX + ".message.size.max"; diff --git a/ratis-grpc/src/main/java/org/apache/ratis/grpc/GrpcFactory.java b/ratis-grpc/src/main/java/org/apache/ratis/grpc/GrpcFactory.java index 1053cab80e..8af523d55b 100644 --- a/ratis-grpc/src/main/java/org/apache/ratis/grpc/GrpcFactory.java +++ b/ratis-grpc/src/main/java/org/apache/ratis/grpc/GrpcFactory.java @@ -69,8 +69,12 @@ static boolean checkPooledByteBufAllocatorUseCacheForAllThreads(Consumer return value; } - static final BiFunction BUILD_SSL_CONTEXT_FOR_SERVER - = (tlsConf, defaultContext) -> tlsConf == null ? defaultContext : GrpcUtil.buildSslContextForServer(tlsConf); + private final GrpcServices.Customizer servicesCustomizer; + + private final GrpcTlsConfig tlsConfig; + private final GrpcTlsConfig adminTlsConfig; + private final GrpcTlsConfig clientTlsConfig; + private final GrpcTlsConfig serverTlsConfig; static final BiFunction BUILD_SSL_CONTEXT_FOR_CLIENT = (tlsConf, defaultContext) -> tlsConf == null ? defaultContext : GrpcUtil.buildSslContextForClient(tlsConf); @@ -104,11 +108,21 @@ public GrpcFactory(Parameters parameters) { ); } + public GrpcFactory(GrpcTlsConfig tlsConfig) { + this(null, tlsConfig, null, null, null); + } + private GrpcFactory(GrpcServices.Customizer servicesCustomizer, GrpcTlsConfig tlsConfig, GrpcTlsConfig adminTlsConfig, GrpcTlsConfig clientTlsConfig, GrpcTlsConfig serverTlsConfig) { this.servicesCustomizer = servicesCustomizer; + this.tlsConfig = tlsConfig; + this.adminTlsConfig = adminTlsConfig; + this.clientTlsConfig = clientTlsConfig; + this.serverTlsConfig = serverTlsConfig; + } + this.forServerSupplier = MemoizedSupplier.valueOf(() -> new SslContexts( tlsConfig, adminTlsConfig, clientTlsConfig, serverTlsConfig, BUILD_SSL_CONTEXT_FOR_SERVER)); this.forClientSupplier = MemoizedSupplier.valueOf(() -> new SslContexts( @@ -128,16 +142,12 @@ public LogAppender newLogAppender(RaftServer.Division server, LeaderState state, @Override public GrpcServices newRaftServerRpc(RaftServer server) { checkPooledByteBufAllocatorUseCacheForAllThreads(LOG::info); - - final SslContexts forServer = forServerSupplier.get(); - final SslContexts forClient = forClientSupplier.get(); return GrpcServicesImpl.newBuilder() .setServer(server) .setCustomizer(servicesCustomizer) - .setAdminSslContext(forServer.adminSslContext) - .setServerSslContextForServer(forServer.serverSslContext) - .setServerSslContextForClient(forClient.serverSslContext) - .setClientSslContext(forServer.clientSslContext) + .setAdminTlsConfig(getAdminTlsConfig()) + .setServerTlsConfig(getServerTlsConfig()) + .setClientTlsConfig(getClientTlsConfig()) .build(); } diff --git a/ratis-grpc/src/main/java/org/apache/ratis/grpc/server/GrpcService.java b/ratis-grpc/src/main/java/org/apache/ratis/grpc/server/GrpcService.java deleted file mode 100644 index ac48fa1adf..0000000000 --- a/ratis-grpc/src/main/java/org/apache/ratis/grpc/server/GrpcService.java +++ /dev/null @@ -1,436 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ -package org.apache.ratis.grpc.server; - -import org.apache.ratis.conf.RaftProperties; -import org.apache.ratis.grpc.GrpcConfigKeys; -import org.apache.ratis.grpc.GrpcTlsConfig; -import org.apache.ratis.grpc.GrpcUtil; -import org.apache.ratis.grpc.metrics.ZeroCopyMetrics; -import org.apache.ratis.grpc.metrics.intercept.server.MetricServerInterceptor; -import org.apache.ratis.protocol.AdminAsynchronousProtocol; -import org.apache.ratis.protocol.RaftGroupId; -import org.apache.ratis.protocol.RaftPeer; -import org.apache.ratis.protocol.RaftPeerId; -import org.apache.ratis.rpc.SupportedRpcType; -import org.apache.ratis.server.RaftServer; -import org.apache.ratis.server.RaftServerConfigKeys; -import org.apache.ratis.server.RaftServerRpcWithProxy; -import org.apache.ratis.server.protocol.RaftServerAsynchronousProtocol; -import org.apache.ratis.thirdparty.com.google.common.annotations.VisibleForTesting; -import org.apache.ratis.thirdparty.io.grpc.ServerInterceptor; -import org.apache.ratis.thirdparty.io.grpc.ServerInterceptors; -import org.apache.ratis.thirdparty.io.grpc.ServerServiceDefinition; -import org.apache.ratis.thirdparty.io.grpc.netty.GrpcSslContexts; -import org.apache.ratis.thirdparty.io.grpc.netty.NettyServerBuilder; -import org.apache.ratis.thirdparty.io.grpc.Server; -import org.apache.ratis.thirdparty.io.grpc.stub.StreamObserver; -import org.apache.ratis.thirdparty.io.netty.channel.ChannelOption; -import org.apache.ratis.thirdparty.io.netty.handler.ssl.ClientAuth; -import org.apache.ratis.thirdparty.io.netty.handler.ssl.SslContextBuilder; - -import org.apache.ratis.proto.RaftProtos.*; -import org.apache.ratis.util.*; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -import java.io.IOException; -import java.net.InetSocketAddress; -import java.util.HashMap; -import java.util.Map; -import java.util.concurrent.CompletableFuture; -import java.util.concurrent.ExecutorService; -import java.util.function.Supplier; - -import static org.apache.ratis.thirdparty.io.netty.handler.ssl.SslProvider.OPENSSL; - -/** A grpc implementation of {@link org.apache.ratis.server.RaftServerRpc}. */ -public final class GrpcService extends RaftServerRpcWithProxy> { - static final Logger LOG = LoggerFactory.getLogger(GrpcService.class); - public static final String GRPC_SEND_SERVER_REQUEST = - JavaUtils.getClassSimpleName(GrpcService.class) + ".sendRequest"; - - class AsyncService implements RaftServerAsynchronousProtocol { - - @Override - public CompletableFuture appendEntriesAsync(AppendEntriesRequestProto request) - throws IOException { - throw new UnsupportedOperationException("This method is not supported"); - } - - @Override - public CompletableFuture readIndexAsync(ReadIndexRequestProto request) throws IOException { - CodeInjectionForTesting.execute(GRPC_SEND_SERVER_REQUEST, getId(), null, request); - - final CompletableFuture f = new CompletableFuture<>(); - final StreamObserver s = new StreamObserver() { - @Override - public void onNext(ReadIndexReplyProto reply) { - f.complete(reply); - } - - @Override - public void onError(Throwable throwable) { - f.completeExceptionally(throwable); - } - - @Override - public void onCompleted() { - } - }; - - final RaftPeerId target = RaftPeerId.valueOf(request.getServerRequest().getReplyId()); - getProxies().getProxy(target).readIndex(request, s); - return f; - } - } - - public static final class Builder { - private RaftServer server; - - private String adminHost; - private int adminPort; - private GrpcTlsConfig adminTlsConfig; - private String clientHost; - private int clientPort; - private GrpcTlsConfig clientTlsConfig; - private String serverHost; - private int serverPort; - private GrpcTlsConfig serverTlsConfig; - - private SizeInBytes messageSizeMax; - private SizeInBytes flowControlWindow; - private TimeDuration requestTimeoutDuration; - private boolean separateHeartbeatChannel; - private boolean zeroCopyEnabled; - - private Builder() {} - - public Builder setServer(RaftServer raftServer) { - this.server = raftServer; - - final RaftProperties properties = server.getProperties(); - this.adminHost = GrpcConfigKeys.Admin.host(properties); - this.adminPort = GrpcConfigKeys.Admin.port(properties); - this.clientHost = GrpcConfigKeys.Client.host(properties); - this.clientPort = GrpcConfigKeys.Client.port(properties); - this.serverHost = GrpcConfigKeys.Server.host(properties); - this.serverPort = GrpcConfigKeys.Server.port(properties); - this.messageSizeMax = GrpcConfigKeys.messageSizeMax(properties, LOG::info); - this.flowControlWindow = GrpcConfigKeys.flowControlWindow(properties, LOG::info); - this.requestTimeoutDuration = RaftServerConfigKeys.Rpc.requestTimeout(properties); - this.separateHeartbeatChannel = GrpcConfigKeys.Server.heartbeatChannel(properties); - this.zeroCopyEnabled = GrpcConfigKeys.Server.zeroCopyEnabled(properties); - - final SizeInBytes appenderBufferSize = RaftServerConfigKeys.Log.Appender.bufferByteLimit(properties); - final SizeInBytes gap = SizeInBytes.ONE_MB; - final long diff = messageSizeMax.getSize() - appenderBufferSize.getSize(); - if (diff < gap.getSize()) { - throw new IllegalArgumentException("Illegal configuration: " - + GrpcConfigKeys.MESSAGE_SIZE_MAX_KEY + "(= " + messageSizeMax - + ") must be " + gap + " larger than " - + RaftServerConfigKeys.Log.Appender.BUFFER_BYTE_LIMIT_KEY + "(= " + appenderBufferSize + ")."); - } - - return this; - } - - private GrpcServerProtocolClient newGrpcServerProtocolClient(RaftPeer target) { - return new GrpcServerProtocolClient(target, flowControlWindow.getSizeInt(), - requestTimeoutDuration, serverTlsConfig, separateHeartbeatChannel); - } - - private ExecutorService newExecutor() { - final RaftProperties properties = server.getProperties(); - return ConcurrentUtils.newThreadPoolWithMax( - GrpcConfigKeys.Server.asyncRequestThreadPoolCached(properties), - GrpcConfigKeys.Server.asyncRequestThreadPoolSize(properties), - server.getId() + "-request-"); - } - - private GrpcClientProtocolService newGrpcClientProtocolService( - ExecutorService executor, ZeroCopyMetrics zeroCopyMetrics) { - return new GrpcClientProtocolService(server::getId, server, executor, zeroCopyEnabled, zeroCopyMetrics); - } - - private GrpcServerProtocolService newGrpcServerProtocolService(ZeroCopyMetrics zeroCopyMetrics) { - return new GrpcServerProtocolService(server::getId, server, zeroCopyEnabled, zeroCopyMetrics); - } - - private MetricServerInterceptor newMetricServerInterceptor() { - return new MetricServerInterceptor(server::getId, - JavaUtils.getClassSimpleName(getClass()) + "_" + serverPort); - } - - private NettyServerBuilder newNettyServerBuilderForServer() { - return newNettyServerBuilder(serverHost, serverPort, serverTlsConfig); - } - - private NettyServerBuilder newNettyServerBuilderForAdmin() { - return newNettyServerBuilder(adminHost, adminPort, adminTlsConfig); - } - - private NettyServerBuilder newNettyServerBuilderForClient() { - return newNettyServerBuilder(clientHost, clientPort, clientTlsConfig); - } - - private NettyServerBuilder newNettyServerBuilder(String hostname, int port, GrpcTlsConfig tlsConfig) { - final InetSocketAddress address = hostname == null || hostname.isEmpty() ? - new InetSocketAddress(port) : new InetSocketAddress(hostname, port); - final NettyServerBuilder nettyServerBuilder = NettyServerBuilder.forAddress(address) - .withChildOption(ChannelOption.SO_REUSEADDR, true) - .maxInboundMessageSize(messageSizeMax.getSizeInt()) - .flowControlWindow(flowControlWindow.getSizeInt()); - - if (tlsConfig != null) { - LOG.info("Setting TLS for {}", address); - SslContextBuilder sslContextBuilder = GrpcUtil.initSslContextBuilderForServer(tlsConfig.getKeyManager()); - if (tlsConfig.getMtlsEnabled()) { - sslContextBuilder.clientAuth(ClientAuth.REQUIRE); - GrpcUtil.setTrustManager(sslContextBuilder, tlsConfig.getTrustManager()); - } - sslContextBuilder = GrpcSslContexts.configure(sslContextBuilder, OPENSSL); - try { - nettyServerBuilder.sslContext(sslContextBuilder.build()); - } catch (Exception ex) { - throw new IllegalArgumentException("Failed to build SslContext, tlsConfig=" + tlsConfig, ex); - } - } - return nettyServerBuilder; - } - - private boolean separateAdminServer() { - return adminPort > 0 && adminPort != serverPort; - } - - private boolean separateClientServer() { - return clientPort > 0 && clientPort != serverPort; - } - - Server newServer(GrpcClientProtocolService client, ZeroCopyMetrics zeroCopyMetrics, ServerInterceptor interceptor) { - final NettyServerBuilder serverBuilder = newNettyServerBuilderForServer(); - final ServerServiceDefinition service = newGrpcServerProtocolService(zeroCopyMetrics).bindServiceWithZeroCopy(); - serverBuilder.addService(ServerInterceptors.intercept(service, interceptor)); - - if (!separateAdminServer()) { - addAdminService(serverBuilder, server, interceptor); - } - if (!separateClientServer()) { - addClientService(serverBuilder, client, interceptor); - } - return serverBuilder.build(); - } - - public GrpcService build() { - return new GrpcService(this); - } - - public Builder setTlsConfig(GrpcTlsConfig tlsConfig) { - this.tlsConfig = tlsConfig; - return this; - } - - public Builder setAdminTlsConfig(GrpcTlsConfig config) { - this.adminTlsConfig = config; - return this; - } - - public Builder setClientTlsConfig(GrpcTlsConfig config) { - this.clientTlsConfig = config; - return this; - } - - public Builder setServerTlsConfig(GrpcTlsConfig config) { - this.serverTlsConfig = config; - return this; - } - - public GrpcTlsConfig getTlsConfig() { - return tlsConfig; - } - } - - public static Builder newBuilder() { - return new Builder(); - } - - private final Map servers = new HashMap<>(); - private final Supplier addressSupplier; - private final Supplier clientServerAddressSupplier; - private final Supplier adminServerAddressSupplier; - - private final AsyncService asyncService = new AsyncService(); - - private final ExecutorService executor; - private final GrpcClientProtocolService clientProtocolService; - - private final MetricServerInterceptor serverInterceptor; - private final ZeroCopyMetrics zeroCopyMetrics = new ZeroCopyMetrics(); - - public MetricServerInterceptor getServerInterceptor() { - return serverInterceptor; - } - - private GrpcService(Builder b) { - super(b.server::getId, id -> new PeerProxyMap<>(id.toString(), b::newGrpcServerProtocolClient)); - - this.executor = b.newExecutor(); - this.clientProtocolService = b.newGrpcClientProtocolService(executor, zeroCopyMetrics); - this.serverInterceptor = b.newMetricServerInterceptor(); - final Server server = b.newServer(clientProtocolService, zeroCopyMetrics, serverInterceptor); - - servers.put(GrpcServerProtocolService.class.getSimpleName(), server); - addressSupplier = newAddressSupplier(b.serverPort, server); - - if (b.separateAdminServer()) { - final NettyServerBuilder builder = b.newNettyServerBuilderForAdmin(); - addAdminService(builder, b.server, serverInterceptor); - final Server adminServer = builder.build(); - servers.put(GrpcAdminProtocolService.class.getName(), adminServer); - adminServerAddressSupplier = newAddressSupplier(b.adminPort, adminServer); - } else { - adminServerAddressSupplier = addressSupplier; - } - - if (b.separateClientServer()) { - final NettyServerBuilder builder = b.newNettyServerBuilderForClient(); - addClientService(builder, clientProtocolService, serverInterceptor); - final Server clientServer = builder.build(); - servers.put(GrpcClientProtocolService.class.getName(), clientServer); - clientServerAddressSupplier = newAddressSupplier(b.clientPort, clientServer); - } else { - clientServerAddressSupplier = addressSupplier; - } - } - - static MemoizedSupplier newAddressSupplier(int port, Server server) { - return JavaUtils.memoize(() -> new InetSocketAddress(port != 0 ? port : server.getPort())); - } - - static void addClientService(NettyServerBuilder builder, GrpcClientProtocolService client, - ServerInterceptor interceptor) { - final ServerServiceDefinition service = client.bindServiceWithZeroCopy(); - builder.addService(ServerInterceptors.intercept(service, interceptor)); - } - - static void addAdminService(NettyServerBuilder builder, AdminAsynchronousProtocol admin, - ServerInterceptor interceptor) { - final GrpcAdminProtocolService service = new GrpcAdminProtocolService(admin); - builder.addService(ServerInterceptors.intercept(service, interceptor)); - } - - @Override - public SupportedRpcType getRpcType() { - return SupportedRpcType.GRPC; - } - - @Override - public void startImpl() { - for (Server server : servers.values()) { - try { - server.start(); - } catch (IOException e) { - ExitUtils.terminate(1, "Failed to start Grpc server", e, LOG); - } - LOG.info("{}: {} started, listening on {}", - getId(), JavaUtils.getClassSimpleName(getClass()), server.getPort()); - } - } - - @Override - public void closeImpl() throws IOException { - for (Map.Entry server : servers.entrySet()) { - final String name = getId() + ": shutdown server " + server.getKey(); - LOG.info("{} now", name); - final Server s = server.getValue().shutdownNow(); - super.closeImpl(); - try { - s.awaitTermination(); - } catch (InterruptedException e) { - Thread.currentThread().interrupt(); - throw IOUtils.toInterruptedIOException(name + " failed", e); - } - LOG.info("{} successfully", name); - } - - serverInterceptor.close(); - ConcurrentUtils.shutdownAndWait(executor); - zeroCopyMetrics.unregister(); - } - - @Override - public void notifyNotLeader(RaftGroupId groupId) { - clientProtocolService.closeAllOrderedRequestStreamObservers(groupId); - } - - @Override - public InetSocketAddress getInetSocketAddress() { - return addressSupplier.get(); - } - - @Override - public InetSocketAddress getClientServerAddress() { - return clientServerAddressSupplier.get(); - } - - @Override - public InetSocketAddress getAdminServerAddress() { - return adminServerAddressSupplier.get(); - } - - @Override - public RaftServerAsynchronousProtocol async() { - return asyncService; - } - - @Override - public AppendEntriesReplyProto appendEntries(AppendEntriesRequestProto request) { - throw new UnsupportedOperationException( - "Blocking " + JavaUtils.getCurrentStackTraceElement().getMethodName() + " call is not supported"); - } - - @Override - public InstallSnapshotReplyProto installSnapshot(InstallSnapshotRequestProto request) { - throw new UnsupportedOperationException( - "Blocking " + JavaUtils.getCurrentStackTraceElement().getMethodName() + " call is not supported"); - } - - @Override - public RequestVoteReplyProto requestVote(RequestVoteRequestProto request) - throws IOException { - CodeInjectionForTesting.execute(GRPC_SEND_SERVER_REQUEST, getId(), - null, request); - - final RaftPeerId target = RaftPeerId.valueOf(request.getServerRequest().getReplyId()); - return getProxies().getProxy(target).requestVote(request); - } - - @Override - public StartLeaderElectionReplyProto startLeaderElection(StartLeaderElectionRequestProto request) throws IOException { - CodeInjectionForTesting.execute(GRPC_SEND_SERVER_REQUEST, getId(), null, request); - - final RaftPeerId target = RaftPeerId.valueOf(request.getServerRequest().getReplyId()); - return getProxies().getProxy(target).startLeaderElection(request); - } - - @VisibleForTesting - public ZeroCopyMetrics getZeroCopyMetrics() { - return zeroCopyMetrics; - } -} diff --git a/ratis-grpc/src/main/java/org/apache/ratis/grpc/server/GrpcServicesImpl.java b/ratis-grpc/src/main/java/org/apache/ratis/grpc/server/GrpcServicesImpl.java index d554ca583a..f864e31308 100644 --- a/ratis-grpc/src/main/java/org/apache/ratis/grpc/server/GrpcServicesImpl.java +++ b/ratis-grpc/src/main/java/org/apache/ratis/grpc/server/GrpcServicesImpl.java @@ -19,7 +19,10 @@ import org.apache.ratis.conf.RaftProperties; import org.apache.ratis.grpc.GrpcConfigKeys; +import org.apache.ratis.grpc.GrpcTlsConfig; +import org.apache.ratis.grpc.GrpcUtil; import org.apache.ratis.grpc.metrics.MessageMetrics; +import org.apache.ratis.grpc.metrics.ZeroCopyMetrics; import org.apache.ratis.grpc.metrics.intercept.server.MetricServerInterceptor; import org.apache.ratis.protocol.AdminAsynchronousProtocol; import org.apache.ratis.protocol.RaftGroupId; @@ -30,13 +33,17 @@ import org.apache.ratis.server.RaftServerConfigKeys; import org.apache.ratis.server.RaftServerRpcWithProxy; import org.apache.ratis.server.protocol.RaftServerAsynchronousProtocol; +import org.apache.ratis.thirdparty.com.google.common.annotations.VisibleForTesting; import org.apache.ratis.thirdparty.io.grpc.ServerInterceptor; import org.apache.ratis.thirdparty.io.grpc.ServerInterceptors; +import org.apache.ratis.thirdparty.io.grpc.ServerServiceDefinition; +import org.apache.ratis.thirdparty.io.grpc.netty.GrpcSslContexts; import org.apache.ratis.thirdparty.io.grpc.netty.NettyServerBuilder; import org.apache.ratis.thirdparty.io.grpc.Server; import org.apache.ratis.thirdparty.io.grpc.stub.StreamObserver; import org.apache.ratis.thirdparty.io.netty.channel.ChannelOption; -import org.apache.ratis.thirdparty.io.netty.handler.ssl.SslContext; +import org.apache.ratis.thirdparty.io.netty.handler.ssl.ClientAuth; +import org.apache.ratis.thirdparty.io.netty.handler.ssl.SslContextBuilder; import org.apache.ratis.proto.RaftProtos.*; import org.apache.ratis.util.*; @@ -52,6 +59,8 @@ import java.util.concurrent.ExecutorService; import java.util.function.Supplier; +import static org.apache.ratis.thirdparty.io.netty.handler.ssl.SslProvider.OPENSSL; + /** A grpc implementation of {@link org.apache.ratis.server.RaftServerRpc}. */ public final class GrpcServicesImpl extends RaftServerRpcWithProxy> @@ -63,7 +72,8 @@ public final class GrpcServicesImpl class AsyncService implements RaftServerAsynchronousProtocol { @Override - public CompletableFuture appendEntriesAsync(AppendEntriesRequestProto request) { + public CompletableFuture appendEntriesAsync(AppendEntriesRequestProto request) + throws IOException { throw new UnsupportedOperationException("This method is not supported"); } @@ -100,20 +110,19 @@ public static final class Builder { private String adminHost; private int adminPort; - private SslContext adminSslContext; + private GrpcTlsConfig adminTlsConfig; private String clientHost; private int clientPort; - private SslContext clientSslContext; + private GrpcTlsConfig clientTlsConfig; private String serverHost; private int serverPort; - private SslContext serverSslContextForServer; - private SslContext serverSslContextForClient; - private int serverStubPoolSize; + private GrpcTlsConfig serverTlsConfig; private SizeInBytes messageSizeMax; private SizeInBytes flowControlWindow; private TimeDuration requestTimeoutDuration; private boolean separateHeartbeatChannel; + private boolean zeroCopyEnabled; private Builder() {} @@ -131,7 +140,7 @@ public Builder setServer(RaftServer raftServer) { this.flowControlWindow = GrpcConfigKeys.flowControlWindow(properties, LOG::info); this.requestTimeoutDuration = RaftServerConfigKeys.Rpc.requestTimeout(properties); this.separateHeartbeatChannel = GrpcConfigKeys.Server.heartbeatChannel(properties); - this.serverStubPoolSize = GrpcConfigKeys.Server.stubPoolSize(properties); + this.zeroCopyEnabled = GrpcConfigKeys.Server.zeroCopyEnabled(properties); final SizeInBytes appenderBufferSize = RaftServerConfigKeys.Log.Appender.bufferByteLimit(properties); final SizeInBytes gap = SizeInBytes.ONE_MB; @@ -152,8 +161,8 @@ public Builder setCustomizer(Customizer customizer) { } private GrpcServerProtocolClient newGrpcServerProtocolClient(RaftPeer target) { - return new GrpcServerProtocolClient(target, serverStubPoolSize, flowControlWindow.getSizeInt(), - requestTimeoutDuration, serverSslContextForClient, separateHeartbeatChannel); + return new GrpcServerProtocolClient(target, flowControlWindow.getSizeInt(), + requestTimeoutDuration, serverTlsConfig, separateHeartbeatChannel); } private ExecutorService newExecutor() { @@ -165,12 +174,12 @@ private ExecutorService newExecutor() { } private GrpcClientProtocolService newGrpcClientProtocolService( - ExecutorService executor) { - return new GrpcClientProtocolService(server::getId, server, executor); + ExecutorService executor, ZeroCopyMetrics zeroCopyMetrics) { + return new GrpcClientProtocolService(server::getId, server, executor, zeroCopyEnabled, zeroCopyMetrics); } - private GrpcServerProtocolService newGrpcServerProtocolService() { - return new GrpcServerProtocolService(server::getId, server); + private GrpcServerProtocolService newGrpcServerProtocolService(ZeroCopyMetrics zeroCopyMetrics) { + return new GrpcServerProtocolService(server::getId, server, zeroCopyEnabled, zeroCopyMetrics); } private MetricServerInterceptor newMetricServerInterceptor() { @@ -183,18 +192,18 @@ Server buildServer(NettyServerBuilder builder, EnumSet types) } private NettyServerBuilder newNettyServerBuilderForServer() { - return newNettyServerBuilder(serverHost, serverPort, serverSslContextForServer); + return newNettyServerBuilder(serverHost, serverPort, serverTlsConfig); } private NettyServerBuilder newNettyServerBuilderForAdmin() { - return newNettyServerBuilder(adminHost, adminPort, adminSslContext); + return newNettyServerBuilder(adminHost, adminPort, adminTlsConfig); } private NettyServerBuilder newNettyServerBuilderForClient() { - return newNettyServerBuilder(clientHost, clientPort, clientSslContext); + return newNettyServerBuilder(clientHost, clientPort, clientTlsConfig); } - private NettyServerBuilder newNettyServerBuilder(String hostname, int port, SslContext sslContext) { + private NettyServerBuilder newNettyServerBuilder(String hostname, int port, GrpcTlsConfig tlsConfig) { final InetSocketAddress address = hostname == null || hostname.isEmpty() ? new InetSocketAddress(port) : new InetSocketAddress(hostname, port); final NettyServerBuilder nettyServerBuilder = NettyServerBuilder.forAddress(address) @@ -202,9 +211,19 @@ private NettyServerBuilder newNettyServerBuilder(String hostname, int port, SslC .maxInboundMessageSize(messageSizeMax.getSizeInt()) .flowControlWindow(flowControlWindow.getSizeInt()); - if (sslContext != null) { + if (tlsConfig != null) { LOG.info("Setting TLS for {}", address); - nettyServerBuilder.sslContext(sslContext); + SslContextBuilder sslContextBuilder = GrpcUtil.initSslContextBuilderForServer(tlsConfig.getKeyManager()); + if (tlsConfig.getMtlsEnabled()) { + sslContextBuilder.clientAuth(ClientAuth.REQUIRE); + GrpcUtil.setTrustManager(sslContextBuilder, tlsConfig.getTrustManager()); + } + sslContextBuilder = GrpcSslContexts.configure(sslContextBuilder, OPENSSL); + try { + nettyServerBuilder.sslContext(sslContextBuilder.build()); + } catch (Exception ex) { + throw new IllegalArgumentException("Failed to build SslContext, tlsConfig=" + tlsConfig, ex); + } } return nettyServerBuilder; } @@ -217,10 +236,10 @@ private boolean separateClientServer() { return clientPort > 0 && clientPort != serverPort; } - Server newServer(GrpcClientProtocolService client, ServerInterceptor interceptor) { + Server newServer(GrpcClientProtocolService client, ZeroCopyMetrics zeroCopyMetrics, ServerInterceptor interceptor) { final EnumSet types = EnumSet.of(GrpcServices.Type.SERVER); final NettyServerBuilder serverBuilder = newNettyServerBuilderForServer(); - final GrpcServerProtocolService service = newGrpcServerProtocolService(); + final ServerServiceDefinition service = newGrpcServerProtocolService(zeroCopyMetrics).bindServiceWithZeroCopy(); serverBuilder.addService(ServerInterceptors.intercept(service, interceptor)); if (!separateAdminServer()) { @@ -238,25 +257,29 @@ public GrpcServicesImpl build() { return new GrpcServicesImpl(this); } - public Builder setAdminSslContext(SslContext adminSslContext) { - this.adminSslContext = adminSslContext; + public Builder setTlsConfig(GrpcTlsConfig tlsConfig) { + this.tlsConfig = tlsConfig; return this; } - public Builder setClientSslContext(SslContext clientSslContext) { - this.clientSslContext = clientSslContext; + public Builder setAdminTlsConfig(GrpcTlsConfig config) { + this.adminTlsConfig = config; return this; } - public Builder setServerSslContextForServer(SslContext serverSslContextForServer) { - this.serverSslContextForServer = serverSslContextForServer; + public Builder setClientTlsConfig(GrpcTlsConfig config) { + this.clientTlsConfig = config; return this; } - public Builder setServerSslContextForClient(SslContext serverSslContextForClient) { - this.serverSslContextForClient = serverSslContextForClient; + public Builder setServerTlsConfig(GrpcTlsConfig config) { + this.serverTlsConfig = config; return this; } + + public GrpcTlsConfig getTlsConfig() { + return tlsConfig; + } } public static Builder newBuilder() { @@ -274,14 +297,15 @@ public static Builder newBuilder() { private final GrpcClientProtocolService clientProtocolService; private final MetricServerInterceptor serverInterceptor; + private final ZeroCopyMetrics zeroCopyMetrics = new ZeroCopyMetrics(); private GrpcServicesImpl(Builder b) { super(b.server::getId, id -> new PeerProxyMap<>(id.toString(), b::newGrpcServerProtocolClient)); this.executor = b.newExecutor(); - this.clientProtocolService = b.newGrpcClientProtocolService(executor); + this.clientProtocolService = b.newGrpcClientProtocolService(executor, zeroCopyMetrics); this.serverInterceptor = b.newMetricServerInterceptor(); - final Server server = b.newServer(clientProtocolService, serverInterceptor); + final Server server = b.newServer(clientProtocolService, zeroCopyMetrics, serverInterceptor); servers.put(GrpcServerProtocolService.class.getSimpleName(), server); addressSupplier = newAddressSupplier(b.serverPort, server); @@ -313,7 +337,8 @@ static MemoizedSupplier newAddressSupplier(int port, Server s static void addClientService(NettyServerBuilder builder, GrpcClientProtocolService client, ServerInterceptor interceptor) { - builder.addService(ServerInterceptors.intercept(client, interceptor)); + final ServerServiceDefinition service = client.bindServiceWithZeroCopy(); + builder.addService(ServerInterceptors.intercept(service, interceptor)); } static void addAdminService(NettyServerBuilder builder, AdminAsynchronousProtocol admin, @@ -341,40 +366,24 @@ public void startImpl() { } @Override - public void closeImpl() { - for (Server server : servers.values()) { - server.shutdownNow(); - } - boolean interrupted = false; + public void closeImpl() throws IOException { for (Map.Entry server : servers.entrySet()) { + final String name = getId() + ": shutdown server " + server.getKey(); + LOG.info("{} now", name); + final Server s = server.getValue().shutdownNow(); + super.closeImpl(); try { - server.getValue().awaitTermination(); - LOG.info("{}: Shutdown {} successfully", getId(), server.getKey()); + s.awaitTermination(); } catch (InterruptedException e) { Thread.currentThread().interrupt(); - LOG.warn("{}: Interrupted shutdown {}", getId(), server.getKey()); - interrupted = true; - break; + throw IOUtils.toInterruptedIOException(name + " failed", e); } + LOG.info("{} successfully", name); } - try { - serverInterceptor.close(); - } catch (Exception e) { - LOG.warn("{}: Failed to unregister metrics", getId(), e); - } - - if (interrupted) { - executor.shutdown(); // shutdown but not wait - } else { - ConcurrentUtils.shutdownAndWait(executor); - } - - try { - super.closeImpl(); - } catch (IOException e) { - LOG.warn("{}: Failed to close proxies", getId(), e); - } + serverInterceptor.close(); + ConcurrentUtils.shutdownAndWait(executor); + zeroCopyMetrics.unregister(); } @Override @@ -432,7 +441,13 @@ public StartLeaderElectionReplyProto startLeaderElection(StartLeaderElectionRequ return getProxies().getProxy(target).startLeaderElection(request); } + @VisibleForTesting MessageMetrics getMessageMetrics() { return serverInterceptor.getMetrics(); } + + @VisibleForTesting + public ZeroCopyMetrics getZeroCopyMetrics() { + return zeroCopyMetrics; + } } diff --git a/ratis-grpc/src/test/java/org/apache/ratis/grpc/MiniRaftClusterWithGrpc.java b/ratis-grpc/src/test/java/org/apache/ratis/grpc/MiniRaftClusterWithGrpc.java index fe12e29f11..1519298f38 100644 --- a/ratis-grpc/src/test/java/org/apache/ratis/grpc/MiniRaftClusterWithGrpc.java +++ b/ratis-grpc/src/test/java/org/apache/ratis/grpc/MiniRaftClusterWithGrpc.java @@ -22,7 +22,7 @@ import org.apache.ratis.conf.Parameters; import org.apache.ratis.conf.RaftProperties; import org.apache.ratis.grpc.metrics.ZeroCopyMetrics; -import org.apache.ratis.grpc.server.GrpcService; +import org.apache.ratis.grpc.server.GrpcServicesImpl; import org.apache.ratis.protocol.RaftGroup; import org.apache.ratis.protocol.RaftPeer; import org.apache.ratis.protocol.RaftPeerId; @@ -63,7 +63,7 @@ default Factory getFactory() { } public static final DelayLocalExecutionInjection SEND_SERVER_REQUEST_INJECTION = - new DelayLocalExecutionInjection(GrpcService.GRPC_SEND_SERVER_REQUEST); + new DelayLocalExecutionInjection(GrpcServicesImpl.GRPC_SEND_SERVER_REQUEST); public MiniRaftClusterWithGrpc(String[] ids, RaftProperties properties, Parameters parameters) { this(ids, new String[0], properties, parameters); @@ -102,7 +102,7 @@ public void assertZeroCopyMetrics() { getServers().forEach(server -> server.getGroupIds().forEach(id -> { LOG.info("Checking {}-{}", server.getId(), id); RaftServer.Division division = RaftServerTestUtil.getDivision(server, id); - GrpcService service = (GrpcService) RaftServerTestUtil.getServerRpc(division); + final GrpcServicesImpl service = (GrpcServicesImpl) RaftServerTestUtil.getServerRpc(division); ZeroCopyMetrics zeroCopyMetrics = service.getZeroCopyMetrics(); Assert.assertEquals(0, zeroCopyMetrics.nonZeroCopyMessages()); Assert.assertEquals("Zero copy messages are not released, please check logs to find leaks. ", diff --git a/ratis-test/src/test/java/org/apache/ratis/grpc/server/TestGrpcMessageMetrics.java b/ratis-test/src/test/java/org/apache/ratis/grpc/server/TestGrpcMessageMetrics.java index 39b745a561..737325d72f 100644 --- a/ratis-test/src/test/java/org/apache/ratis/grpc/server/TestGrpcMessageMetrics.java +++ b/ratis-test/src/test/java/org/apache/ratis/grpc/server/TestGrpcMessageMetrics.java @@ -19,7 +19,6 @@ import org.apache.ratis.BaseTest; import org.apache.ratis.grpc.MiniRaftClusterWithGrpc; -import org.apache.ratis.grpc.metrics.MessageMetrics; import org.apache.ratis.server.impl.MiniRaftCluster; import org.apache.ratis.RaftTestUtil; import org.apache.ratis.client.RaftClient; From ab8b482a3151d06b273abfb66f2b48c46e6eba5b Mon Sep 17 00:00:00 2001 From: Tsz-Wo Nicholas Sze Date: Sun, 20 Oct 2024 03:31:08 -0700 Subject: [PATCH 367/397] RATIS-2173. Fix zero-copy bugs for non-gRPC cases. (#1167) (cherry picked from commit fc39c38eab115bd8349d753b2b9087c7f9398e0e) --- .../org/apache/ratis/TestMultiRaftGroup.java | 11 +-- .../server/leader/LogAppenderDefault.java | 30 +++---- .../server/raftlog/memory/MemoryRaftLog.java | 23 ++---- .../java/org/apache/ratis/RaftBasicTests.java | 6 -- .../java/org/apache/ratis/RaftTestUtil.java | 7 +- .../server/impl/LeaderElectionTests.java | 79 ++++++++----------- 6 files changed, 59 insertions(+), 97 deletions(-) diff --git a/ratis-examples/src/test/java/org/apache/ratis/TestMultiRaftGroup.java b/ratis-examples/src/test/java/org/apache/ratis/TestMultiRaftGroup.java index 7822b694df..ea3962c088 100644 --- a/ratis-examples/src/test/java/org/apache/ratis/TestMultiRaftGroup.java +++ b/ratis-examples/src/test/java/org/apache/ratis/TestMultiRaftGroup.java @@ -22,29 +22,20 @@ import org.apache.ratis.examples.arithmetic.ArithmeticStateMachine; import org.apache.ratis.examples.arithmetic.TestArithmetic; import org.apache.ratis.protocol.RaftGroup; -import org.apache.ratis.server.RaftServer; import org.apache.ratis.server.impl.GroupManagementBaseTest; import org.apache.ratis.server.impl.MiniRaftCluster; -import org.apache.ratis.util.Slf4jUtils; -import org.apache.ratis.test.tag.Flaky; import org.apache.ratis.util.function.CheckedBiConsumer; import org.junit.jupiter.api.Timeout; import org.junit.jupiter.params.ParameterizedTest; import org.junit.jupiter.params.provider.MethodSource; -import org.slf4j.event.Level; import java.io.IOException; import java.util.Collection; import java.util.concurrent.atomic.AtomicInteger; -@Flaky("RATIS-2218") @Timeout(value = 300) public class TestMultiRaftGroup extends BaseTest { - static { - Slf4jUtils.setLogLevel(RaftServer.Division.LOG, Level.DEBUG); - } - - public static Collection data() throws IOException { + public static Collection data() { return ParameterizedBaseTest.getMiniRaftClusters(ArithmeticStateMachine.class, 0); } diff --git a/ratis-server/src/main/java/org/apache/ratis/server/leader/LogAppenderDefault.java b/ratis-server/src/main/java/org/apache/ratis/server/leader/LogAppenderDefault.java index 3d1e6de30b..8c1675c7c3 100644 --- a/ratis-server/src/main/java/org/apache/ratis/server/leader/LogAppenderDefault.java +++ b/ratis-server/src/main/java/org/apache/ratis/server/leader/LogAppenderDefault.java @@ -59,23 +59,15 @@ public Comparator getCallIdComparator() { /** Send an appendEntries RPC; retry indefinitely. */ private AppendEntriesReplyProto sendAppendEntriesWithRetries(AtomicLong requestFirstIndex) throws InterruptedException, InterruptedIOException, RaftLogIOException { - int retry = 0; - - ReferenceCountedObject request = nextAppendEntriesRequest( - CallId.getAndIncrement(), false); - while (isRunning()) { // keep retrying for IOException + for(int retry = 0; isRunning(); retry++) { + final ReferenceCountedObject request = nextAppendEntriesRequest( + CallId.getAndIncrement(), false); + if (request == null) { + LOG.trace("{} no entries to send now, wait ...", this); + return null; + } try { - if (request == null || request.get().getEntriesCount() == 0) { - if (request != null) { - request.release(); - } - request = nextAppendEntriesRequest(CallId.getAndIncrement(), false); - } - - if (request == null) { - LOG.trace("{} no entries to send now, wait ...", this); - return null; - } else if (!isRunning()) { + if (!isRunning()) { LOG.info("{} is stopped. Skip appendEntries.", this); return null; } @@ -84,17 +76,19 @@ private AppendEntriesReplyProto sendAppendEntriesWithRetries(AtomicLong requestF final AppendEntriesReplyProto reply = sendAppendEntries(proto); final long first = proto.getEntriesCount() > 0 ? proto.getEntries(0).getIndex() : RaftLog.INVALID_LOG_INDEX; requestFirstIndex.set(first); - request.release(); return reply; } catch (InterruptedIOException | RaftLogIOException e) { throw e; } catch (IOException ioe) { // TODO should have more detailed retry policy here. - if (retry++ % 10 == 0) { // to reduce the number of messages + if (retry % 10 == 0) { // to reduce the number of messages LOG.warn("{}: Failed to appendEntries (retry={})", this, retry, ioe); } handleException(ioe); + } finally { + request.release(); } + if (isRunning()) { getServer().properties().rpcSleepTime().sleep(); } diff --git a/ratis-server/src/main/java/org/apache/ratis/server/raftlog/memory/MemoryRaftLog.java b/ratis-server/src/main/java/org/apache/ratis/server/raftlog/memory/MemoryRaftLog.java index 2aac6c1b1f..f4b6dc452e 100644 --- a/ratis-server/src/main/java/org/apache/ratis/server/raftlog/memory/MemoryRaftLog.java +++ b/ratis-server/src/main/java/org/apache/ratis/server/raftlog/memory/MemoryRaftLog.java @@ -45,15 +45,10 @@ */ public class MemoryRaftLog extends RaftLogBase { static class EntryList { - private final List> entries = new ArrayList<>(); - - ReferenceCountedObject getRef(int i) { - return i >= 0 && i < entries.size() ? entries.get(i) : null; - } + private final List entries = new ArrayList<>(); LogEntryProto get(int i) { - final ReferenceCountedObject ref = getRef(i); - return ref != null ? ref.get() : null; + return i >= 0 && i < entries.size() ? entries.get(i) : null; } TermIndex getTermIndex(int i) { @@ -81,13 +76,10 @@ void purge(int index) { } void clear(int from, int to) { - List> subList = entries.subList(from, to); - subList.forEach(ReferenceCountedObject::release); - subList.clear(); + entries.subList(from, to).clear(); } - void add(ReferenceCountedObject entryRef) { - entryRef.retain(); + void add(LogEntryProto entryRef) { entries.add(entryRef); } } @@ -128,7 +120,8 @@ public LogEntryProto get(long index) throws RaftLogIOException { public ReferenceCountedObject retainLog(long index) { checkLogState(); try (AutoCloseableLock readLock = readLock()) { - ReferenceCountedObject ref = entries.getRef(Math.toIntExact(index)); + final LogEntryProto entry = entries.get(Math.toIntExact(index)); + final ReferenceCountedObject ref = ReferenceCountedObject.wrap(entry); ref.retain(); return ref; } @@ -205,7 +198,7 @@ protected CompletableFuture appendEntryImpl(ReferenceCountedObject> appendImpl(ReferenceCountedObject extends BaseTest implements MiniRaftCluster.Factory.Get { { - Slf4jUtils.setLogLevel(RaftServer.Division.LOG, Level.DEBUG); - RaftServerTestUtil.setStateMachineUpdaterLogLevel(Level.DEBUG); - RaftServerConfigKeys.RetryCache.setExpiryTime(getProperties(), TimeDuration.valueOf(5, TimeUnit.SECONDS)); } diff --git a/ratis-server/src/test/java/org/apache/ratis/RaftTestUtil.java b/ratis-server/src/test/java/org/apache/ratis/RaftTestUtil.java index b1a68b8816..360cb922c6 100644 --- a/ratis-server/src/test/java/org/apache/ratis/RaftTestUtil.java +++ b/ratis-server/src/test/java/org/apache/ratis/RaftTestUtil.java @@ -189,8 +189,11 @@ static boolean logEntriesContains(RaftLog log, long startIndex, long endIndex, S log.get(termIndices[idxEntries].getIndex()).getStateMachineLogEntry().getLogData().toByteArray())) { ++idxExpected; } - } catch (IOException e) { - throw new RuntimeException(e); + } catch (Exception e) { + throw new IllegalStateException("Failed logEntriesContains: startIndex=" + startIndex + + ", endIndex=" + endIndex + + ", #expectedMessages=" + expectedMessages.length + + ", log=" + log, e); } ++idxEntries; } diff --git a/ratis-server/src/test/java/org/apache/ratis/server/impl/LeaderElectionTests.java b/ratis-server/src/test/java/org/apache/ratis/server/impl/LeaderElectionTests.java index 8f2da0bba3..ff604f5631 100644 --- a/ratis-server/src/test/java/org/apache/ratis/server/impl/LeaderElectionTests.java +++ b/ratis-server/src/test/java/org/apache/ratis/server/impl/LeaderElectionTests.java @@ -21,8 +21,8 @@ import org.apache.ratis.RaftTestUtil; import org.apache.ratis.client.RaftClient; import org.apache.ratis.conf.RaftProperties; -import org.apache.ratis.metrics.impl.RatisMetricRegistryImpl; import org.apache.ratis.metrics.impl.DefaultTimekeeperImpl; +import org.apache.ratis.metrics.impl.RatisMetricRegistryImpl; import org.apache.ratis.proto.RaftProtos; import org.apache.ratis.protocol.RaftClientReply; import org.apache.ratis.protocol.RaftGroupId; @@ -37,8 +37,6 @@ import org.apache.ratis.server.RaftServerConfigKeys; import org.apache.ratis.server.metrics.LeaderElectionMetrics; import org.apache.ratis.server.protocol.TermIndex; -import org.apache.ratis.server.raftlog.segmented.SegmentedRaftLogTestUtils; -import org.apache.ratis.test.tag.Flaky; import org.apache.ratis.thirdparty.com.codahale.metrics.Timer; import org.apache.ratis.util.CodeInjectionForTesting; import org.apache.ratis.util.JavaUtils; @@ -50,6 +48,7 @@ import org.junit.jupiter.api.Assertions; import org.junit.jupiter.api.Test; import org.slf4j.Logger; +import org.slf4j.event.Level; import java.io.IOException; import java.util.ArrayList; @@ -66,8 +65,8 @@ import static org.apache.ratis.RaftTestUtil.waitForLeader; import static org.apache.ratis.server.metrics.LeaderElectionMetrics.LAST_LEADER_ELECTION_ELAPSED_TIME; import static org.apache.ratis.server.metrics.LeaderElectionMetrics.LEADER_ELECTION_COUNT_METRIC; -import static org.apache.ratis.server.metrics.LeaderElectionMetrics.LEADER_ELECTION_TIME_TAKEN; import static org.apache.ratis.server.metrics.LeaderElectionMetrics.LEADER_ELECTION_TIMEOUT_COUNT_METRIC; +import static org.apache.ratis.server.metrics.LeaderElectionMetrics.LEADER_ELECTION_TIME_TAKEN; import static org.junit.jupiter.api.Assertions.assertEquals; import static org.junit.jupiter.api.Assertions.assertNotNull; import static org.junit.jupiter.api.Assertions.assertTrue; @@ -75,8 +74,6 @@ import static org.mockito.Mockito.mock; import static org.mockito.Mockito.when; -import org.slf4j.event.Level; - public abstract class LeaderElectionTests extends BaseTest implements MiniRaftCluster.Factory.Get { @@ -88,15 +85,16 @@ public abstract class LeaderElectionTests @Test public void testBasicLeaderElection() throws Exception { LOG.info("Running testBasicLeaderElection"); - final MiniRaftCluster cluster = newCluster(5); - cluster.start(); + runWithNewCluster(5, this::runTestBasicLeaderElection); + } + + void runTestBasicLeaderElection(MiniRaftCluster cluster) throws Exception { RaftTestUtil.waitAndKillLeader(cluster); RaftTestUtil.waitAndKillLeader(cluster); RaftTestUtil.waitAndKillLeader(cluster); testFailureCase("waitForLeader after killed a majority of servers", () -> RaftTestUtil.waitForLeader(cluster, null, false), IllegalStateException.class); - cluster.shutdown(); } static class SleepCode implements CodeInjectionForTesting.Code { @@ -123,9 +121,11 @@ public void testWaitServerReady() throws Exception { final int sleepMs = 1000 + ThreadLocalRandom.current().nextInt(1000); LOG.info("Running testWaitServerReady, sleep = {}ms", sleepMs); CodeInjectionForTesting.put(RaftServerImpl.START_COMPLETE, new SleepCode(sleepMs)); - final MiniRaftCluster cluster = newCluster(1); final Timestamp startTime = Timestamp.currentTime(); - cluster.start(); + runWithNewCluster(1, c -> runTestWaitServerReady(c, sleepMs, startTime)); + } + + void runTestWaitServerReady(MiniRaftCluster cluster, int sleepMs, Timestamp startTime) throws Exception { LOG.info("Cluster started at {}ms", startTime.elapsedTimeMs()); final RaftGroupId groupId = cluster.getGroupId(); final RaftServerImpl server = (RaftServerImpl) cluster.getServers().iterator().next().getDivision(groupId); @@ -136,17 +136,18 @@ public void testWaitServerReady() throws Exception { assertNotNull(waitForLeader(cluster), "No leader is elected."); final long elapsedMs = startTime.elapsedTimeMs(); // allow a small difference to tolerate system timer inaccuracy - assertTrue(elapsedMs > sleepMs - 10, () -> "elapseMs = " + elapsedMs + " but sleepMs = " + sleepMs); - cluster.shutdown(); + Assertions.assertTrue(elapsedMs > sleepMs - 10, () -> "elapseMs = " + elapsedMs + " but sleepMs = " + sleepMs); CodeInjectionForTesting.remove(RaftServerImpl.START_COMPLETE); } @Test - public void testAddServerForWaitReady() throws IOException, InterruptedException { + public void testAddServerForWaitReady() throws Exception { LOG.info("Running testAddServerForWaitReady"); // normal startup cluster with 3 server - final MiniRaftCluster cluster = newCluster(3); - cluster.start(); + runWithNewCluster(3, this::runTestAddServerForWaitReady); + } + + void runTestAddServerForWaitReady(MiniRaftCluster cluster) throws Exception { RaftTestUtil.waitForLeader(cluster); try (RaftClient client = cluster.createClient()) { for (int i = 0; i < 10; ++i) { @@ -169,23 +170,20 @@ public void testAddServerForWaitReady() throws IOException, InterruptedException }); } } - cluster.shutdown();; CodeInjectionForTesting.remove(RaftServerImpl.START_COMPLETE); } @Test public void testChangeLeader() throws Exception { - SegmentedRaftLogTestUtils.setRaftLogWorkerLogLevel(Level.TRACE); LOG.info("Running testChangeLeader"); - final MiniRaftCluster cluster = newCluster(3); - cluster.start(); + runWithNewCluster(3, this::runTestChangeLeader); + } + void runTestChangeLeader(MiniRaftCluster cluster) throws Exception { RaftPeerId leader = RaftTestUtil.waitForLeader(cluster).getId(); for(int i = 0; i < 10; i++) { - leader = RaftTestUtil.changeLeader(cluster, leader, Assertions::fail); + leader = RaftTestUtil.changeLeader(cluster, leader, IllegalStateException::new); } - SegmentedRaftLogTestUtils.setRaftLogWorkerLogLevel(Level.INFO); - cluster.shutdown(); } @Test @@ -271,8 +269,6 @@ public void testTransferLeader() throws Exception { Assertions.assertEquals(newLeader.getId().toString(), reply.getReplierId()); Assertions.assertTrue(reply.isSuccess()); } - - cluster.shutdown(); } } @@ -305,8 +301,6 @@ public void testYieldLeaderToHigherPriority() throws Exception { Assertions.assertEquals(newLeader.getId().toString(), reply.getReplierId()); Assertions.assertTrue(reply.isSuccess()); } - - cluster.shutdown(); } } @@ -364,8 +358,6 @@ public void testTransferLeaderTimeout() throws Exception { RaftTestUtil.deIsolate(cluster, newLeader.getId()); } - - cluster.shutdown(); } } @@ -403,14 +395,18 @@ static void enforceLeader(MiniRaftCluster cluster, final String newLeader, Logge @Test public void testLateServerStart() throws Exception { - final int numServer = 3; LOG.info("Running testLateServerStart"); - final MiniRaftCluster cluster = newCluster(numServer); + try (final MiniRaftCluster cluster = newCluster(3)) { + runTestLateServerStart(cluster); + } + } + + void runTestLateServerStart(MiniRaftCluster cluster) throws Exception { cluster.initServers(); // start all except one servers final Iterator i = cluster.getServers().iterator(); - for(int j = 1; j < numServer; j++) { + for(int j = 1; j < cluster.getNumServers(); j++) { i.next().start(); } @@ -428,7 +424,6 @@ public void testLateServerStart() throws Exception { 10, ONE_SECOND, "getLeaderId", LOG); LOG.info(cluster.printServers()); Assertions.assertEquals(leader.getId(), lastServerLeaderId); - cluster.shutdown(); } protected void testDisconnectLeader() throws Exception { @@ -446,8 +441,6 @@ protected void testDisconnectLeader() throws Exception { } finally { RaftTestUtil.deIsolate(cluster, leader.getId()); } - - cluster.shutdown(); } } @@ -469,7 +462,6 @@ public void testAddListener() throws Exception { Assertions.assertEquals(1, listener.size()); Assertions.assertEquals(changes.newPeers[0].getId(), new ArrayList<>(listener).get(0).getId()); } - cluster.shutdown(); } } @@ -495,7 +487,6 @@ public void testAddFollowerWhenExistsListener() throws Exception { Assertions.assertEquals(1, leader.getRaftConf().getAllPeers(RaftProtos.RaftPeerRole.LISTENER).size()); } - cluster.shutdown(); } } @@ -514,7 +505,6 @@ public void testRemoveListener() throws Exception { Assertions.assertTrue(reply.isSuccess()); Assertions.assertEquals(0, leader.getRaftConf().getAllPeers(RaftProtos.RaftPeerRole.LISTENER).size()); } - cluster.shutdown(); } } @@ -537,7 +527,6 @@ public void testChangeFollowerToListener() throws Exception { Assertions.assertEquals(1, peer.size()); Assertions.assertEquals(listeners.get(0).getId(), new ArrayList<>(peer).get(0).getId()); } - cluster.shutdown(); } } @@ -556,15 +545,16 @@ public void testChangeListenerToFollower() throws Exception { Collection peer = leader.getRaftConf().getAllPeers(RaftProtos.RaftPeerRole.LISTENER); Assertions.assertEquals(0, peer.size()); } - cluster.shutdown(); } } @Test - public void testLeaderElectionMetrics() throws IOException, InterruptedException { + public void testLeaderElectionMetrics() throws Exception { + runWithNewCluster(3, this::runTestLeaderElectionMetrics); + } + + void runTestLeaderElectionMetrics(MiniRaftCluster cluster) throws Exception { Timestamp timestamp = Timestamp.currentTime(); - final MiniRaftCluster cluster = newCluster(3); - cluster.start(); final RaftServer.Division leaderServer = waitForLeader(cluster); final RatisMetricRegistryImpl ratisMetricRegistry = (RatisMetricRegistryImpl) @@ -586,7 +576,6 @@ public void testLeaderElectionMetrics() throws IOException, InterruptedException Long leaderElectionLatency = (Long) ratisMetricRegistry.getGauges((s, metric) -> s.contains(LAST_LEADER_ELECTION_ELAPSED_TIME)).values().iterator().next().getValue(); assertTrue(leaderElectionLatency > 0L); - cluster.shutdown(); } @Test @@ -652,8 +641,6 @@ public void testPreVote() { reply = client.io().send(new RaftTestUtil.SimpleMessage("message")); Assertions.assertTrue(reply.isSuccess()); } - - cluster.shutdown(); } catch (Exception e) { fail(e.getMessage()); } From ebfb53d6c7445c53c0316ac2260d894b0a040854 Mon Sep 17 00:00:00 2001 From: 133tosakarin <97331129+133tosakarin@users.noreply.github.com> Date: Tue, 22 Oct 2024 02:26:53 +0800 Subject: [PATCH 368/397] RATIS-2174. Move future.join outside the lock (#1168) (cherry picked from commit d08ba81c26bd5e3ab862be3649c7d5ec541a9b04) --- .../ratis/server/impl/LeaderStateImpl.java | 3 +- .../ratis/server/impl/RaftServerImpl.java | 16 ++--------- .../impl/SnapshotInstallationHandler.java | 28 +++++++++---------- 3 files changed, 17 insertions(+), 30 deletions(-) diff --git a/ratis-server/src/main/java/org/apache/ratis/server/impl/LeaderStateImpl.java b/ratis-server/src/main/java/org/apache/ratis/server/impl/LeaderStateImpl.java index 5dcb3251c9..622db7bf69 100644 --- a/ratis-server/src/main/java/org/apache/ratis/server/impl/LeaderStateImpl.java +++ b/ratis-server/src/main/java/org/apache/ratis/server/impl/LeaderStateImpl.java @@ -737,8 +737,7 @@ void submitStepDownEvent(long term, StepDownReason reason) { private void stepDown(long term, StepDownReason reason) { try { lease.getAndSetEnabled(false); - server.changeToFollowerAndPersistMetadata(term, false, reason) - .get(5, TimeUnit.SECONDS); + server.changeToFollowerAndPersistMetadata(term, false, reason).join(); pendingStepDown.complete(server::newSuccessReply); } catch(Exception e) { pendingStepDown.completeExceptionally(e); diff --git a/ratis-server/src/main/java/org/apache/ratis/server/impl/RaftServerImpl.java b/ratis-server/src/main/java/org/apache/ratis/server/impl/RaftServerImpl.java index 7ce3f5feb0..069ed82d4c 100644 --- a/ratis-server/src/main/java/org/apache/ratis/server/impl/RaftServerImpl.java +++ b/ratis-server/src/main/java/org/apache/ratis/server/impl/RaftServerImpl.java @@ -610,14 +610,6 @@ private synchronized CompletableFuture changeToFollower( return future; } - private boolean shouldSetFollower(RaftPeerRole old, boolean force) { - if (old == RaftPeerRole.LISTENER) { - final RaftConfigurationImpl conf = state.getRaftConf(); - return conf.isStable() && conf.containsInConf(getId()); - } - return old != RaftPeerRole.FOLLOWER || force; - } - synchronized CompletableFuture changeToFollowerAndPersistMetadata( long newTerm, boolean allowListener, @@ -1619,7 +1611,7 @@ leaderId, getMemberId(), currentTerm, followerCommit, state.getNextIndex(), AppendResult.NOT_LEADER, callId, RaftLog.INVALID_LOG_INDEX, isHeartbeat)); } try { - future = changeToFollowerAndPersistMetadata(leaderTerm, true, Op.APPEND_ENTRIES); + future = changeToFollowerAndPersistMetadata(leaderTerm, true, "appendEntries"); } catch (IOException e) { return JavaUtils.completeExceptionally(e); } @@ -1644,16 +1636,12 @@ leaderId, getMemberId(), currentTerm, followerCommit, inconsistencyReplyNextInde AppendResult.INCONSISTENCY, callId, RaftLog.INVALID_LOG_INDEX, isHeartbeat); LOG.info("{}: appendEntries* reply {}", getMemberId(), toAppendEntriesReplyString(reply)); followerState.ifPresent(fs -> fs.updateLastRpcTime(FollowerState.UpdateType.APPEND_COMPLETE)); - return CompletableFuture.completedFuture(reply); + return future.thenApply(dummy -> reply); } state.updateConfiguration(entries); } future.join(); - final CompletableFuture appendFuture = entries.isEmpty()? CompletableFuture.completedFuture(null) - : appendLogTermIndices != null ? appendLogTermIndices.append(entries, this::appendLog) - : JavaUtils.allOf(state.getLog().append(entries)); - final List> futures = entries.isEmpty() ? Collections.emptyList() : state.getLog().append(requestRef.delegate(entries)); diff --git a/ratis-server/src/main/java/org/apache/ratis/server/impl/SnapshotInstallationHandler.java b/ratis-server/src/main/java/org/apache/ratis/server/impl/SnapshotInstallationHandler.java index f8ba31a395..7f191e54dc 100644 --- a/ratis-server/src/main/java/org/apache/ratis/server/impl/SnapshotInstallationHandler.java +++ b/ratis-server/src/main/java/org/apache/ratis/server/impl/SnapshotInstallationHandler.java @@ -175,11 +175,11 @@ private CompletableFuture checkAndInstallSnapshot(Ins final boolean recognized = state.recognizeLeader(RaftServerProtocol.Op.INSTALL_SNAPSHOT, leaderId, leaderTerm); currentTerm = state.getCurrentTerm(); if (!recognized) { - return toInstallSnapshotReplyProto(leaderId, getMemberId(), - currentTerm, snapshotChunkRequest.getRequestIndex(), InstallSnapshotResult.NOT_LEADER); + return CompletableFuture.completedFuture(toInstallSnapshotReplyProto(leaderId, getMemberId(), + currentTerm, snapshotChunkRequest.getRequestIndex(), InstallSnapshotResult.NOT_LEADER)); } - future = server.changeToFollowerAndPersistMetadata(leaderTerm, true, Op.INSTALL_SNAPSHOT); - state.setLeader(leaderId, Op.INSTALL_SNAPSHOT); + future = server.changeToFollowerAndPersistMetadata(leaderTerm, true, "installSnapshot"); + state.setLeader(leaderId, "installSnapshot"); server.updateLastRpcTime(UpdateType.INSTALL_SNAPSHOT_START); long callId = chunk0CallId.get(); @@ -234,7 +234,7 @@ private CompletableFuture checkAndInstallSnapshot(Ins if (snapshotChunkRequest.getDone()) { LOG.info("{}: successfully install the entire snapshot-{}", getMemberId(), lastIncludedIndex); } - return toInstallSnapshotReplyProto(leaderId, getMemberId(), + final InstallSnapshotReplyProto reply = toInstallSnapshotReplyProto(leaderId, getMemberId(), currentTerm, snapshotChunkRequest.getRequestIndex(), InstallSnapshotResult.SUCCESS); return future.thenApply(dummy -> reply); } @@ -251,12 +251,12 @@ private CompletableFuture notifyStateMachineToInstall final boolean recognized = state.recognizeLeader("notifyInstallSnapshot", leaderId, leaderTerm); currentTerm = state.getCurrentTerm(); if (!recognized) { - return toInstallSnapshotReplyProto(leaderId, getMemberId(), - currentTerm, InstallSnapshotResult.NOT_LEADER); + return CompletableFuture.completedFuture(toInstallSnapshotReplyProto(leaderId, getMemberId(), + currentTerm, InstallSnapshotResult.NOT_LEADER)); } - future = server.changeToFollowerAndPersistMetadata(leaderTerm, true, UpdateType.INSTALL_SNAPSHOT_NOTIFICATION); - state.setLeader(leaderId, UpdateType.INSTALL_SNAPSHOT_NOTIFICATION); - server.updateLastRpcTime(UpdateType.INSTALL_SNAPSHOT_NOTIFICATION); + future = server.changeToFollowerAndPersistMetadata(leaderTerm, true, "installSnapshot"); + state.setLeader(leaderId, "installSnapshot"); + server.updateLastRpcTime(FollowerState.UpdateType.INSTALL_SNAPSHOT_NOTIFICATION); if (inProgressInstallSnapshotIndex.compareAndSet(INVALID_LOG_INDEX, firstAvailableLogIndex)) { LOG.info("{}: Received notification to install snapshot at index {}", getMemberId(), firstAvailableLogIndex); @@ -271,7 +271,7 @@ private CompletableFuture notifyStateMachineToInstall inProgressInstallSnapshotIndex.compareAndSet(firstAvailableLogIndex, INVALID_LOG_INDEX); LOG.info("{}: InstallSnapshot notification result: {}, current snapshot index: {}", getMemberId(), InstallSnapshotResult.ALREADY_INSTALLED, snapshotIndex); - return toInstallSnapshotReplyProto(leaderId, getMemberId(), currentTerm, + final InstallSnapshotReplyProto reply = toInstallSnapshotReplyProto(leaderId, getMemberId(), currentTerm, InstallSnapshotResult.ALREADY_INSTALLED, snapshotIndex); return future.thenApply(dummy -> reply); } @@ -350,7 +350,7 @@ private CompletableFuture notifyStateMachineToInstall inProgressInstallSnapshotIndex.set(INVALID_LOG_INDEX); server.getStateMachine().event().notifySnapshotInstalled( InstallSnapshotResult.SNAPSHOT_UNAVAILABLE, INVALID_LOG_INDEX, server.getPeer()); - return toInstallSnapshotReplyProto(leaderId, getMemberId(), + final InstallSnapshotReplyProto reply = toInstallSnapshotReplyProto(leaderId, getMemberId(), currentTerm, InstallSnapshotResult.SNAPSHOT_UNAVAILABLE); return future.thenApply(dummy -> reply); } @@ -369,7 +369,7 @@ private CompletableFuture notifyStateMachineToInstall server.getStateMachine().event().notifySnapshotInstalled( InstallSnapshotResult.SNAPSHOT_INSTALLED, latestInstalledIndex, server.getPeer()); installedIndex.set(latestInstalledIndex); - return toInstallSnapshotReplyProto(leaderId, getMemberId(), + final InstallSnapshotReplyProto reply = toInstallSnapshotReplyProto(leaderId, getMemberId(), currentTerm, InstallSnapshotResult.SNAPSHOT_INSTALLED, latestInstalledSnapshotTermIndex.getIndex()); return future.thenApply(dummy -> reply); } @@ -379,7 +379,7 @@ private CompletableFuture notifyStateMachineToInstall LOG.debug("{}: InstallSnapshot notification result: {}", getMemberId(), InstallSnapshotResult.IN_PROGRESS); } - return toInstallSnapshotReplyProto(leaderId, getMemberId(), + final InstallSnapshotReplyProto reply = toInstallSnapshotReplyProto(leaderId, getMemberId(), currentTerm, InstallSnapshotResult.IN_PROGRESS); return future.thenApply(dummy -> reply); } From fb497efe088872cf19cdd29db2d549f558647449 Mon Sep 17 00:00:00 2001 From: Tsz-Wo Nicholas Sze Date: Sun, 27 Oct 2024 20:56:18 -0700 Subject: [PATCH 369/397] RATIS-2178. Add a log message whenever deleting a RaftLog segment. (#1170) (cherry picked from commit bf15765a58ac1476a13239d812df0c0b172acb22) --- .../ratis/server/raftlog/segmented/LogSegment.java | 1 + .../raftlog/segmented/SegmentedRaftLogCache.java | 2 +- .../raftlog/segmented/SegmentedRaftLogWorker.java | 7 +++---- .../apache/ratis/server/ServerRestartTests.java | 14 ++++++-------- 4 files changed, 11 insertions(+), 13 deletions(-) diff --git a/ratis-server/src/main/java/org/apache/ratis/server/raftlog/segmented/LogSegment.java b/ratis-server/src/main/java/org/apache/ratis/server/raftlog/segmented/LogSegment.java index 816e1b7bfb..f2bb5a63aa 100644 --- a/ratis-server/src/main/java/org/apache/ratis/server/raftlog/segmented/LogSegment.java +++ b/ratis-server/src/main/java/org/apache/ratis/server/raftlog/segmented/LogSegment.java @@ -37,6 +37,7 @@ import java.io.File; import java.io.IOException; +import java.nio.file.Path; import java.util.Comparator; import java.util.HashMap; import java.util.Iterator; diff --git a/ratis-server/src/main/java/org/apache/ratis/server/raftlog/segmented/SegmentedRaftLogCache.java b/ratis-server/src/main/java/org/apache/ratis/server/raftlog/segmented/SegmentedRaftLogCache.java index 86fb483eee..7429477e3d 100644 --- a/ratis-server/src/main/java/org/apache/ratis/server/raftlog/segmented/SegmentedRaftLogCache.java +++ b/ratis-server/src/main/java/org/apache/ratis/server/raftlog/segmented/SegmentedRaftLogCache.java @@ -390,7 +390,7 @@ TruncationSegments purge(long index) { list.forEach(LogSegment::evictCache); List toDelete = list.stream().map(SegmentFileInfo::newClosedSegmentFileInfo) .collect(Collectors.toList()); - return list.isEmpty() ? null : new TruncationSegments(null, toDelete); + return list.isEmpty() ? null : new TruncationSegments("purge(" + index + ")", null, toDelete); } } diff --git a/ratis-server/src/main/java/org/apache/ratis/server/raftlog/segmented/SegmentedRaftLogWorker.java b/ratis-server/src/main/java/org/apache/ratis/server/raftlog/segmented/SegmentedRaftLogWorker.java index 98b20bade3..a44be6ee55 100644 --- a/ratis-server/src/main/java/org/apache/ratis/server/raftlog/segmented/SegmentedRaftLogWorker.java +++ b/ratis-server/src/main/java/org/apache/ratis/server/raftlog/segmented/SegmentedRaftLogWorker.java @@ -482,10 +482,9 @@ private PurgeLog(TruncationSegments segments) { void execute() throws IOException { if (segments.getToDelete() != null) { try(UncheckedAutoCloseable ignored = raftLogMetrics.startPurgeTimer()) { - SegmentFileInfo[] toDeletes = segments.getToDelete(); - for (int i = toDeletes.length - 1; i >= 0; i--) { - final Path deleted = FileUtils.deleteFile(toDeletes[i].getFile(storage)); - LOG.info("{}: Purged RaftLog segment: info={}, path={}", name, toDeletes[i], deleted); + for (SegmentFileInfo fileInfo : segments.getToDelete()) { + final Path deleted = FileUtils.deleteFile(fileInfo.getFile(storage)); + LOG.info("{}: Purged RaftLog segment: info={}, path={}", name, fileInfo, deleted); } } } diff --git a/ratis-test/src/test/java/org/apache/ratis/server/ServerRestartTests.java b/ratis-test/src/test/java/org/apache/ratis/server/ServerRestartTests.java index f7a8bbaeca..c9495fdf79 100644 --- a/ratis-test/src/test/java/org/apache/ratis/server/ServerRestartTests.java +++ b/ratis-test/src/test/java/org/apache/ratis/server/ServerRestartTests.java @@ -64,8 +64,6 @@ import java.util.concurrent.atomic.AtomicInteger; import java.util.function.Supplier; -import static org.apache.ratis.server.storage.RaftStorageTestUtils.getLogUnsafe; - /** * Test restarting raft peers. */ @@ -178,7 +176,7 @@ static void assertTruncatedLog(RaftPeerId id, File openLogFile, long lastIndex, } static File getOpenLogFile(RaftServer.Division server) throws Exception { - final List openLogs = getOpenLogFiles(server); + final List openLogs = SegmentedRaftLogTestUtils.getOpenLogFiles(server); Assertions.assertEquals(1, openLogs.size()); return openLogs.get(0).toFile(); } @@ -205,7 +203,7 @@ void runTestRestartWithCorruptedLogHeader(MiniRaftCluster cluster) throws Except 10, HUNDRED_MILLIS, impl.getId() + "-getOpenLogFile", LOG); for(int i = 0; i < SegmentedRaftLogFormat.getHeaderLength(); i++) { assertCorruptedLogHeader(impl.getId(), openLogFile, i, cluster, LOG); - Assertions.assertTrue(getOpenLogFiles(impl).isEmpty()); + Assertions.assertTrue(SegmentedRaftLogTestUtils.getOpenLogFiles(impl).isEmpty()); } } } @@ -263,10 +261,10 @@ void runTestRestartCommitIndex(MiniRaftCluster cluster) throws Exception { final long lastIndex = leaderLog.getLastEntryTermIndex().getIndex(); LOG.info("{}: leader lastIndex={}", leaderId, lastIndex); - final LogEntryProto lastEntry = getLogUnsafe(leaderLog, lastIndex); + final LogEntryProto lastEntry = leaderLog.get(lastIndex); LOG.info("{}: leader lastEntry entry[{}] = {}", leaderId, lastIndex, LogProtoUtils.toLogEntryString(lastEntry)); final long loggedCommitIndex = lastEntry.getMetadataEntry().getCommitIndex(); - final LogEntryProto lastCommittedEntry = getLogUnsafe(leaderLog, loggedCommitIndex); + final LogEntryProto lastCommittedEntry = leaderLog.get(loggedCommitIndex); LOG.info("{}: leader lastCommittedEntry = entry[{}] = {}", leaderId, loggedCommitIndex, LogProtoUtils.toLogEntryString(lastCommittedEntry)); @@ -312,11 +310,11 @@ void runTestRestartCommitIndex(MiniRaftCluster cluster) throws Exception { static void assertLastLogEntry(RaftServer.Division server) throws RaftLogIOException { final RaftLog raftLog = server.getRaftLog(); final long lastIndex = raftLog.getLastEntryTermIndex().getIndex(); - final LogEntryProto lastEntry = getLogUnsafe(raftLog, lastIndex); + final LogEntryProto lastEntry = raftLog.get(lastIndex); Assertions.assertTrue(lastEntry.hasMetadataEntry()); final long loggedCommitIndex = lastEntry.getMetadataEntry().getCommitIndex(); - final LogEntryProto lastCommittedEntry = getLogUnsafe(raftLog, loggedCommitIndex); + final LogEntryProto lastCommittedEntry = raftLog.get(loggedCommitIndex); Assertions.assertTrue(lastCommittedEntry.hasStateMachineLogEntry()); final SimpleStateMachine4Testing leaderStateMachine = SimpleStateMachine4Testing.get(server); From a0173339e64d94d4be8fcb6865dc0d40252ecb57 Mon Sep 17 00:00:00 2001 From: Ivan Andika Date: Wed, 13 Nov 2024 00:23:04 +0800 Subject: [PATCH 370/397] RATIS-2186. Raft log should not purge index lower than the log start index (#1175) (cherry picked from commit 13b8cdd0ccdd9ae2578600fbc454315b339da4a1) --- .../ratis/server/raftlog/segmented/SegmentedRaftLogCache.java | 4 ++++ 1 file changed, 4 insertions(+) diff --git a/ratis-server/src/main/java/org/apache/ratis/server/raftlog/segmented/SegmentedRaftLogCache.java b/ratis-server/src/main/java/org/apache/ratis/server/raftlog/segmented/SegmentedRaftLogCache.java index 7429477e3d..66a7c8e511 100644 --- a/ratis-server/src/main/java/org/apache/ratis/server/raftlog/segmented/SegmentedRaftLogCache.java +++ b/ratis-server/src/main/java/org/apache/ratis/server/raftlog/segmented/SegmentedRaftLogCache.java @@ -365,6 +365,10 @@ TruncationSegments truncate(long index, LogSegment openSegment, Runnable clearOp TruncationSegments purge(long index) { try (AutoCloseableLock writeLock = writeLock()) { int segmentIndex = binarySearch(index); + if (segmentIndex == -1) { + // nothing to purge + return null; + } List list = new LinkedList<>(); if (segmentIndex == -segments.size() - 1) { From e6f81ba82fc6176e80a57ac2592340d1242b4062 Mon Sep 17 00:00:00 2001 From: "Doroszlai, Attila" <6454655+adoroszlai@users.noreply.github.com> Date: Mon, 23 Dec 2024 18:56:14 +0100 Subject: [PATCH 371/397] RATIS-2220. Skip further tests after leak detected (#1193) (cherry picked from commit ee56932231f7e639eab6570a6fc89f5f02074568) --- .../org/apache/ratis/util/LeakDetector.java | 4 ++++ .../test/java/org/apache/ratis/BaseTest.java | 19 ++++++------------- 2 files changed, 10 insertions(+), 13 deletions(-) diff --git a/ratis-common/src/main/java/org/apache/ratis/util/LeakDetector.java b/ratis-common/src/main/java/org/apache/ratis/util/LeakDetector.java index a6b2ec28bd..6f12205795 100644 --- a/ratis-common/src/main/java/org/apache/ratis/util/LeakDetector.java +++ b/ratis-common/src/main/java/org/apache/ratis/util/LeakDetector.java @@ -156,6 +156,10 @@ Runnable track(Object leakable, Supplier reportLeak) { return trackers.add(leakable, queue, reportLeak)::remove; } + public int getLeakCount() { + return trackers.getNumLeaks(false); + } + public void assertNoLeaks(int maxRetries, TimeDuration retrySleep) throws InterruptedException { synchronized (leakMessages) { // leakMessages are all the leaks discovered so far. diff --git a/ratis-common/src/test/java/org/apache/ratis/BaseTest.java b/ratis-common/src/test/java/org/apache/ratis/BaseTest.java index 8773fc101e..653213b3c3 100644 --- a/ratis-common/src/test/java/org/apache/ratis/BaseTest.java +++ b/ratis-common/src/test/java/org/apache/ratis/BaseTest.java @@ -21,6 +21,7 @@ import org.apache.ratis.util.ExitUtils; import org.apache.ratis.util.FileUtils; import org.apache.ratis.util.JavaUtils; +import org.apache.ratis.util.ReferenceCountedLeakDetector; import org.apache.ratis.util.Slf4jUtils; import org.apache.ratis.util.StringUtils; import org.apache.ratis.util.TimeDuration; @@ -73,20 +74,12 @@ public void setFirstException(Throwable e) { @BeforeEach public void setup(TestInfo testInfo) { - checkAssumptions(); + testCaseName = testInfo.getTestMethod() + .orElseThrow(() -> new RuntimeException("Exception while getting test name.")) + .getName(); - final Method method = testInfo.getTestMethod().orElse(null); - testCaseName = testInfo.getTestClass().orElse(getClass()).getSimpleName() - + "." + (method == null? null : method.getName()); - } - - @BeforeEach - public void checkAssumptions() { - final Throwable first = firstException.get(); - Assumptions.assumeTrue(first == null, () -> "Already failed with " + first); - - final Throwable exited = ExitUtils.getFirstExitException(); - Assumptions.assumeTrue(exited == null, () -> "Already exited with " + exited); + final int leaks = ReferenceCountedLeakDetector.getLeakDetector().getLeakCount(); + Assumptions.assumeFalse(0 < leaks, () -> "numLeaks " + leaks + " > 0"); } @AfterEach From 50e2d7d192a30c1c94430b97a81d8d01b10fe859 Mon Sep 17 00:00:00 2001 From: Tsz-Wo Nicholas Sze Date: Mon, 23 Dec 2024 11:33:05 -0800 Subject: [PATCH 372/397] RATIS-2225. RaftClientRequest leak in RaftServerImpl. (#1198) (cherry picked from commit 0514e0926202b50054643f677510c7bfe16c01f1) --- .../ratis/server/impl/RaftServerImpl.java | 34 +++++++++---------- 1 file changed, 17 insertions(+), 17 deletions(-) diff --git a/ratis-server/src/main/java/org/apache/ratis/server/impl/RaftServerImpl.java b/ratis-server/src/main/java/org/apache/ratis/server/impl/RaftServerImpl.java index 069ed82d4c..f480f72f2e 100644 --- a/ratis-server/src/main/java/org/apache/ratis/server/impl/RaftServerImpl.java +++ b/ratis-server/src/main/java/org/apache/ratis/server/impl/RaftServerImpl.java @@ -635,6 +635,15 @@ synchronized void changeToLeader() { @Override public Collection getCommitInfos() { + try { + return getCommitInfosImpl(); + } catch (Throwable t) { + LOG.warn("{} Failed to getCommitInfos", getMemberId(), t); + return Collections.emptyList(); + } + } + + private Collection getCommitInfosImpl() { final List infos = new ArrayList<>(); // add the commit info of this server final long commitIndex = updateCommitInfoCache(); @@ -925,26 +934,12 @@ CompletableFuture executeSubmitClientRequestAsync(RaftClientReq @Override public CompletableFuture submitClientRequestAsync( - RaftClientRequest request) throws IOException { - return TraceServer.traceAsyncMethod( - () -> submitClientRequestAsyncInternal(request), - request, getMemberId().toString(), "raft.server.submitClientRequestAsync"); - } - - private CompletableFuture submitClientRequestAsyncInternal( - RaftClientRequest request) throws IOException { - assertLifeCycleState(LifeCycle.States.RUNNING); - LOG.debug("{}: receive client request({})", getMemberId(), request); - + ReferenceCountedObject requestRef) { + final RaftClientRequest request = requestRef.retain(); try { + LOG.debug("{}: receive client request({})", getMemberId(), request); assertLifeCycleState(LifeCycle.States.RUNNING); - } catch (ServerNotReadyException e) { - final RaftClientReply reply = newExceptionReply(request, e); - requestRef.release(); - return CompletableFuture.completedFuture(reply); - } - try { RaftClientRequest.Type type = request.getType(); final Timekeeper timer = raftServerMetrics.getClientRequestTimer(type); final Optional timerContext = Optional.ofNullable(timer).map(Timekeeper::time); @@ -954,6 +949,11 @@ private CompletableFuture submitClientRequestAsyncInternal( raftServerMetrics.incFailedRequestCount(type); } }); + } catch (RaftException e) { + return CompletableFuture.completedFuture(newExceptionReply(request, e)); + } catch (Throwable t) { + LOG.error("{} Failed to submitClientRequestAsync for {}", getMemberId(), request, t); + return CompletableFuture.completedFuture(newExceptionReply(request, new RaftException(t))); } finally { requestRef.release(); } From faeeb08fadab6cb2bb9aa885b727e6b0b0bfaaa6 Mon Sep 17 00:00:00 2001 From: jianghuazhu <740087514@qq.com> Date: Tue, 24 Dec 2024 03:57:55 +0800 Subject: [PATCH 373/397] RATIS-2184. Improve TestRaftWithGrpc test stability (#1177) (cherry picked from commit b210965db17cc255a0d97d4e3d7450145063425a) --- .../ratis/grpc/server/GrpcLogAppender.java | 13 +++++++++++-- .../ratis/server/leader/LogAppenderBase.java | 17 ++++++++++++----- .../server/raftlog/segmented/LogSegment.java | 18 +++++++++++------- .../ratis/server/impl/MiniRaftCluster.java | 14 ++++++-------- 4 files changed, 40 insertions(+), 22 deletions(-) diff --git a/ratis-grpc/src/main/java/org/apache/ratis/grpc/server/GrpcLogAppender.java b/ratis-grpc/src/main/java/org/apache/ratis/grpc/server/GrpcLogAppender.java index dd5c365306..1131447242 100644 --- a/ratis-grpc/src/main/java/org/apache/ratis/grpc/server/GrpcLogAppender.java +++ b/ratis-grpc/src/main/java/org/apache/ratis/grpc/server/GrpcLogAppender.java @@ -315,8 +315,14 @@ private long errorWaitTimeMs() { @Override public CompletableFuture stopAsync() { - grpcServerMetrics.unregister(); - return super.stopAsync(); + try (AutoCloseableLock ignored = lock.writeLock(caller, LOG::trace)) { + if (appendLogRequestObserver != null) { + appendLogRequestObserver.stop(); + appendLogRequestObserver = null; + } + grpcServerMetrics.unregister(); + return super.stopAsync(); + } } @Override @@ -395,6 +401,9 @@ private void appendLog(boolean heartbeat) throws IOException { final ReferenceCountedObject pending; final AppendEntriesRequest request; try (AutoCloseableLock writeLock = lock.writeLock(caller, LOG::trace)) { + if (!isRunning()) { + return; + } // Prepare and send the append request. // Note changes on follower's nextIndex and ops on pendingRequests should always be done under the write-lock pending = nextAppendEntriesRequest(callId.getAndIncrement(), heartbeat); diff --git a/ratis-server/src/main/java/org/apache/ratis/server/leader/LogAppenderBase.java b/ratis-server/src/main/java/org/apache/ratis/server/leader/LogAppenderBase.java index e424be86cc..4b6fc0a568 100644 --- a/ratis-server/src/main/java/org/apache/ratis/server/leader/LogAppenderBase.java +++ b/ratis-server/src/main/java/org/apache/ratis/server/leader/LogAppenderBase.java @@ -259,12 +259,19 @@ protected ReferenceCountedObject nextAppendEntriesReq final long halfMs = heartbeatWaitTimeMs/2; final Map> offered = new HashMap<>(); for (long next = followerNext; leaderNext > next && getHeartbeatWaitTimeMs() - halfMs > 0; next++) { - final ReferenceCountedObject entryWithData = getRaftLog().retainEntryWithData(next); - if (!buffer.offer(entryWithData.get())) { - entryWithData.release(); - break; + ReferenceCountedObject entryWithData = null; + try { + entryWithData = getRaftLog().retainEntryWithData(next); + if (!buffer.offer(entryWithData.get())) { + entryWithData.release(); + break; + } + offered.put(next, entryWithData); + } catch (Exception e){ + if (entryWithData != null) { + entryWithData.release(); + } } - offered.put(next, entryWithData); } if (buffer.isEmpty()) { return null; diff --git a/ratis-server/src/main/java/org/apache/ratis/server/raftlog/segmented/LogSegment.java b/ratis-server/src/main/java/org/apache/ratis/server/raftlog/segmented/LogSegment.java index f2bb5a63aa..20fd123d89 100644 --- a/ratis-server/src/main/java/org/apache/ratis/server/raftlog/segmented/LogSegment.java +++ b/ratis-server/src/main/java/org/apache/ratis/server/raftlog/segmented/LogSegment.java @@ -281,13 +281,17 @@ public ReferenceCountedObject load(LogRecord key) throws IOExcept // the on-disk log file should be truncated but has not been done yet. final AtomicReference> toReturn = new AtomicReference<>(); final LogSegmentStartEnd startEnd = LogSegmentStartEnd.valueOf(startIndex, endIndex, isOpen); - readSegmentFile(file, startEnd, maxOpSize, - getLogCorruptionPolicy(), raftLogMetrics, entry -> { - final TermIndex ti = TermIndex.valueOf(entry); - putEntryCache(ti, entry, Op.LOAD_SEGMENT_FILE); - if (ti.equals(key.getTermIndex())) { - toReturn.set(entryRef); - } else { + readSegmentFile(file, startEnd, maxOpSize, getLogCorruptionPolicy(), raftLogMetrics, entryRef -> { + final LogEntryProto entry = entryRef.retain(); + try { + final TermIndex ti = TermIndex.valueOf(entry); + putEntryCache(ti, entryRef, Op.LOAD_SEGMENT_FILE); + if (ti.equals(key.getTermIndex())) { + toReturn.set(entryRef); + } else { + entryRef.release(); + } + } catch (Exception e) { entryRef.release(); } }); diff --git a/ratis-server/src/test/java/org/apache/ratis/server/impl/MiniRaftCluster.java b/ratis-server/src/test/java/org/apache/ratis/server/impl/MiniRaftCluster.java index 4f83549b91..f673098376 100644 --- a/ratis-server/src/test/java/org/apache/ratis/server/impl/MiniRaftCluster.java +++ b/ratis-server/src/test/java/org/apache/ratis/server/impl/MiniRaftCluster.java @@ -821,12 +821,11 @@ public void shutdown() { getServers().forEach(proxy -> executor.submit(() -> JavaUtils.runAsUnchecked(proxy::close))); final int maxRetries = 30; final TimeDuration retrySleep = TimeDuration.ONE_SECOND; - try { - executor.shutdown(); - // just wait for a few seconds - boolean terminated = false; + executor.shutdown(); + boolean terminated = false; - for(int i = 0; i < maxRetries && !terminated; ) { + for(int i = 0; i < maxRetries && !terminated; ) { + try { terminated = executor.awaitTermination(retrySleep.getDuration(), retrySleep.getUnit()); if (!terminated) { i++; @@ -836,10 +835,9 @@ public void shutdown() { LOG.error("Failed to shutdown executor, some servers may be still running:\n{}", printServers()); } } - } - } catch (InterruptedException e) { + } catch (InterruptedException e) { LOG.warn("shutdown interrupted", e); - Thread.currentThread().interrupt(); + } } Optional.ofNullable(timer.get()).ifPresent(Timer::cancel); From 0ed7113ca08d42073afacce19eae5af3491841d6 Mon Sep 17 00:00:00 2001 From: jianghuazhu <740087514@qq.com> Date: Thu, 26 Dec 2024 01:30:47 +0800 Subject: [PATCH 374/397] RATIS-2227. LogEntryProto leak in SegmentedRaftLog (#1199) (cherry picked from commit accb612a8cc6f719f72a12cd949b8add9d754d42) --- .../apache/ratis/util/ReferenceCountedLeakDetector.java | 2 +- .../org/apache/ratis/server/leader/LogAppenderBase.java | 8 +++++--- 2 files changed, 6 insertions(+), 4 deletions(-) diff --git a/ratis-common/src/main/java/org/apache/ratis/util/ReferenceCountedLeakDetector.java b/ratis-common/src/main/java/org/apache/ratis/util/ReferenceCountedLeakDetector.java index acf6fb8cfc..ec99eee58e 100644 --- a/ratis-common/src/main/java/org/apache/ratis/util/ReferenceCountedLeakDetector.java +++ b/ratis-common/src/main/java/org/apache/ratis/util/ReferenceCountedLeakDetector.java @@ -271,7 +271,7 @@ static class TraceInfo { } static int findFirstUnequalFromTail(T[] current, T[] previous) { - int c = current.length - 1; + int c = current.length == 0 ? 0 : current.length - 1; for(int p = previous.length - 1; p >= 0; p--, c--) { if (!previous[p].equals(current[c])) { return c; diff --git a/ratis-server/src/main/java/org/apache/ratis/server/leader/LogAppenderBase.java b/ratis-server/src/main/java/org/apache/ratis/server/leader/LogAppenderBase.java index 4b6fc0a568..337e169bba 100644 --- a/ratis-server/src/main/java/org/apache/ratis/server/leader/LogAppenderBase.java +++ b/ratis-server/src/main/java/org/apache/ratis/server/leader/LogAppenderBase.java @@ -259,7 +259,7 @@ protected ReferenceCountedObject nextAppendEntriesReq final long halfMs = heartbeatWaitTimeMs/2; final Map> offered = new HashMap<>(); for (long next = followerNext; leaderNext > next && getHeartbeatWaitTimeMs() - halfMs > 0; next++) { - ReferenceCountedObject entryWithData = null; + final ReferenceCountedObject entryWithData; try { entryWithData = getRaftLog().retainEntryWithData(next); if (!buffer.offer(entryWithData.get())) { @@ -268,9 +268,11 @@ protected ReferenceCountedObject nextAppendEntriesReq } offered.put(next, entryWithData); } catch (Exception e){ - if (entryWithData != null) { - entryWithData.release(); + for (ReferenceCountedObject ref : offered.values()) { + ref.release(); } + offered.clear(); + throw e; } } if (buffer.isEmpty()) { From 9658b90ebf4f9bf9f0ea5baa572ffbc62cdabec6 Mon Sep 17 00:00:00 2001 From: Tsz-Wo Nicholas Sze Date: Fri, 3 Jan 2025 07:11:05 -0800 Subject: [PATCH 375/397] RATIS-2228. Refactor the offered map in LogAppenderBase.nextAppendEntriesRequest (#1201) (cherry picked from commit c454d78393588d634aa50437ea8a0e7d6de80976) --- .../ratis/server/leader/LogAppenderBase.java | 198 +++++++++++------- .../server/raftlog/segmented/LogSegment.java | 23 +- .../raftlog/segmented/SegmentedRaftLog.java | 5 +- .../raftlog/segmented/TestLogSegment.java | 20 +- 4 files changed, 151 insertions(+), 95 deletions(-) diff --git a/ratis-server/src/main/java/org/apache/ratis/server/leader/LogAppenderBase.java b/ratis-server/src/main/java/org/apache/ratis/server/leader/LogAppenderBase.java index 337e169bba..ad579235b7 100644 --- a/ratis-server/src/main/java/org/apache/ratis/server/leader/LogAppenderBase.java +++ b/ratis-server/src/main/java/org/apache/ratis/server/leader/LogAppenderBase.java @@ -37,14 +37,13 @@ import org.apache.ratis.util.SizeInBytes; import org.apache.ratis.util.TimeDuration; +import java.util.Collection; import java.util.Collections; import java.util.HashMap; import java.util.List; import java.util.Map; import java.util.Objects; -import java.util.Optional; import java.util.concurrent.CompletableFuture; -import java.util.concurrent.TimeUnit; import java.util.concurrent.atomic.AtomicBoolean; import java.util.function.LongUnaryOperator; @@ -52,12 +51,90 @@ * An abstract implementation of {@link LogAppender}. */ public abstract class LogAppenderBase implements LogAppender { + /** For buffering log entries to create an {@link EntryList}. */ + private static class EntryBuffer { + /** A queue for limiting the byte size, number of elements and poll time. */ + private final DataQueue queue; + /** A map for releasing {@link ReferenceCountedObject}s. */ + private final Map> references = new HashMap<>(); + + EntryBuffer(Object name, RaftProperties properties) { + final SizeInBytes bufferByteLimit = RaftServerConfigKeys.Log.Appender.bufferByteLimit(properties); + final int bufferElementLimit = RaftServerConfigKeys.Log.Appender.bufferElementLimit(properties); + this.queue = new DataQueue<>(name, bufferByteLimit, bufferElementLimit, EntryWithData::getSerializedSize); + } + + boolean putNew(long index, ReferenceCountedObject retained) { + if (!queue.offer(retained.get())) { + retained.release(); + return false; + } + final ReferenceCountedObject previous = references.put(index, retained); + Preconditions.assertNull(previous, () -> "previous with index " + index); + return true; + } + + void releaseAllAndClear() { + for (ReferenceCountedObject ref : references.values()) { + ref.release(); + } + references.clear(); + queue.clear(); + } + + EntryList pollList(long heartbeatWaitTimeMs) throws RaftLogIOException { + final List protos; + try { + protos = queue.pollList(heartbeatWaitTimeMs, EntryWithData::getEntry, null); + } catch (Exception e) { + releaseAllAndClear(); + throw e; + } finally { + for (EntryWithData entry : queue) { + // Remove and release remaining entries. + final ReferenceCountedObject removed = references.remove(entry.getIndex()); + Objects.requireNonNull(removed, "removed == null"); + removed.release(); + } + queue.clear(); + } + return new EntryList(protos, references); + } + } + + /** Storing log entries and their references. */ + private static class EntryList { + private final List protos; + private final Collection> references; + + EntryList(List protos, Map> references) { + Preconditions.assertSame(references.size(), protos.size(), "#entries"); + this.protos = Collections.unmodifiableList(protos); + this.references = Collections.unmodifiableCollection(references.values()); + } + + List getProtos() { + return protos; + } + + void retain() { + for (ReferenceCountedObject ref : references) { + ref.retain(); + } + } + + void release() { + for (ReferenceCountedObject ref : references) { + ref.release(); + } + } + } + private final String name; private final RaftServer.Division server; private final LeaderState leaderState; private final FollowerInfo follower; - private final DataQueue buffer; private final int snapshotChunkMaxSize; private final LogAppenderDaemon daemon; @@ -75,9 +152,6 @@ protected LogAppenderBase(RaftServer.Division server, LeaderState leaderState, F final RaftProperties properties = server.getRaftServer().getProperties(); this.snapshotChunkMaxSize = RaftServerConfigKeys.Log.Appender.snapshotChunkSizeMax(properties).getSizeInt(); - final SizeInBytes bufferByteLimit = RaftServerConfigKeys.Log.Appender.bufferByteLimit(properties); - final int bufferElementLimit = RaftServerConfigKeys.Log.Appender.bufferElementLimit(properties); - this.buffer = new DataQueue<>(this, bufferByteLimit, bufferElementLimit, EntryWithData::getSerializedSize); this.daemon = new LogAppenderDaemon(this); this.eventAwaitForSignal = new AwaitForSignal(name); @@ -200,13 +274,13 @@ protected LongUnaryOperator getNextIndexForError(long newNextIndex) { final long n = oldNextIndex <= 0L ? oldNextIndex : Math.min(oldNextIndex - 1, newNextIndex); if (m > n) { if (m > newNextIndex) { - LOG.info("Set nextIndex to matchIndex + 1 (= " + m + ")"); + LOG.info("{}: Set nextIndex to matchIndex + 1 (= {})", name, m); } return m; } else if (oldNextIndex <= 0L) { return oldNextIndex; // no change. } else { - LOG.info("Decrease nextIndex to " + n); + LOG.info("{}: Decrease nextIndex to {}", name, n); return n; } }; @@ -217,18 +291,18 @@ public AppendEntriesRequestProto newAppendEntriesRequest(long callId, boolean he throw new UnsupportedOperationException("Use nextAppendEntriesRequest(" + callId + ", " + heartbeat +") instead."); } -/** - * Create a {@link AppendEntriesRequestProto} object using the {@link FollowerInfo} of this {@link LogAppender}. - * The {@link AppendEntriesRequestProto} object may contain zero or more log entries. - * When there is zero log entries, the {@link AppendEntriesRequestProto} object is a heartbeat. - * - * @param callId The call id of the returned request. - * @param heartbeat the returned request must be a heartbeat. - * - * @return a retained reference of {@link AppendEntriesRequestProto} object. - * Since the returned reference is retained, the caller must call {@link ReferenceCountedObject#release()}} - * after use. - */ + /** + * Create a {@link AppendEntriesRequestProto} object using the {@link FollowerInfo} of this {@link LogAppender}. + * The {@link AppendEntriesRequestProto} object may contain zero or more log entries. + * When there is zero log entries, the {@link AppendEntriesRequestProto} object is a heartbeat. + * + * @param callId The call id of the returned request. + * @param heartbeat the returned request must be a heartbeat. + * + * @return a retained reference of {@link AppendEntriesRequestProto} object. + * Since the returned reference is retained, + * the caller must call {@link ReferenceCountedObject#release()}} after use. + */ protected ReferenceCountedObject nextAppendEntriesRequest(long callId, boolean heartbeat) throws RaftLogIOException { final long heartbeatWaitTimeMs = getHeartbeatWaitTimeMs(); @@ -243,64 +317,23 @@ protected ReferenceCountedObject nextAppendEntriesReq return ref; } - Preconditions.assertTrue(buffer.isEmpty(), () -> "buffer has " + buffer.getNumElements() + " elements."); - final long snapshotIndex = follower.getSnapshotIndex(); - final long leaderNext = getRaftLog().getNextIndex(); final long followerNext = follower.getNextIndex(); - - if (previous == null && followerNext > RaftLog.LEAST_VALID_LOG_INDEX && followerNext != snapshotIndex + 1) { - LOG.info("{}: Skipping appendEntries since the previous log entry is unavailable:" + - " follower {} nextIndex={} and snapshotIndex={} but leader startIndex={}", - this, follower.getName(), followerNext, snapshotIndex, getRaftLog().getStartIndex()); - return null; - } - - final long halfMs = heartbeatWaitTimeMs/2; - final Map> offered = new HashMap<>(); - for (long next = followerNext; leaderNext > next && getHeartbeatWaitTimeMs() - halfMs > 0; next++) { - final ReferenceCountedObject entryWithData; - try { - entryWithData = getRaftLog().retainEntryWithData(next); - if (!buffer.offer(entryWithData.get())) { - entryWithData.release(); - break; - } - offered.put(next, entryWithData); - } catch (Exception e){ - for (ReferenceCountedObject ref : offered.values()) { - ref.release(); - } - offered.clear(); - throw e; - } - } - if (buffer.isEmpty()) { + final EntryBuffer entryBuffer = readLogEntries(followerNext, heartbeatWaitTimeMs); + if (entryBuffer == null) { return null; } - final List protos; - try { - protos = buffer.pollList(getHeartbeatWaitTimeMs(), EntryWithData::getEntry, - (entry, time, exception) -> LOG.warn("Failed to get {} in {}", - entry, time.toString(TimeUnit.MILLISECONDS, 3), exception)); - } catch (RaftLogIOException e) { - for (ReferenceCountedObject ref : offered.values()) { - ref.release(); - } - offered.clear(); - throw e; - } finally { - for (EntryWithData entry : buffer) { - // Release remaining entries. - Optional.ofNullable(offered.remove(entry.getIndex())).ifPresent(ReferenceCountedObject::release); - } - buffer.clear(); - } + final EntryList entryList = entryBuffer.pollList(heartbeatWaitTimeMs); + final List protos = entryList.getProtos(); assertProtos(protos, followerNext, previous, snapshotIndex); AppendEntriesRequestProto appendEntriesProto = leaderState.newAppendEntriesRequestProto(follower, protos, previous, callId); - return ReferenceCountedObject.delegateFrom(offered.values(), appendEntriesProto); + final ReferenceCountedObject ref = ReferenceCountedObject.wrap( + appendEntriesProto, entryList::retain, entryList::release); + ref.retain(); + entryList.release(); + return ref; } private void assertProtos(List protos, long nextIndex, TermIndex previous, long snapshotIndex) { @@ -322,6 +355,31 @@ private void assertProtos(List protos, long nextIndex, TermIndex } } + private EntryBuffer readLogEntries(long followerNext, long heartbeatWaitTimeMs) throws RaftLogIOException { + final RaftLog raftLog = getRaftLog(); + final long leaderNext = raftLog.getNextIndex(); + final long halfMs = heartbeatWaitTimeMs/2; + EntryBuffer entryBuffer = null; + for (long next = followerNext; leaderNext > next && getHeartbeatWaitTimeMs() - halfMs > 0; next++) { + final ReferenceCountedObject retained; + try { + retained = raftLog.retainEntryWithData(next); + if (entryBuffer == null) { + entryBuffer = new EntryBuffer(name, server.getRaftServer().getProperties()); + } + if (!entryBuffer.putNew(next, retained)) { + break; + } + } catch (Exception e) { + if (entryBuffer != null) { + entryBuffer.releaseAllAndClear(); + } + throw e; + } + } + return entryBuffer; + } + @Override public InstallSnapshotRequestProto newInstallSnapshotNotificationRequest(TermIndex firstAvailableLogTermIndex) { Preconditions.assertTrue(firstAvailableLogTermIndex.getIndex() >= 0); diff --git a/ratis-server/src/main/java/org/apache/ratis/server/raftlog/segmented/LogSegment.java b/ratis-server/src/main/java/org/apache/ratis/server/raftlog/segmented/LogSegment.java index 20fd123d89..1f280fba69 100644 --- a/ratis-server/src/main/java/org/apache/ratis/server/raftlog/segmented/LogSegment.java +++ b/ratis-server/src/main/java/org/apache/ratis/server/raftlog/segmented/LogSegment.java @@ -26,7 +26,6 @@ import org.apache.ratis.server.raftlog.RaftLogIOException; import org.apache.ratis.server.storage.RaftStorage; import org.apache.ratis.thirdparty.com.google.common.annotations.VisibleForTesting; -import org.apache.ratis.thirdparty.com.google.common.cache.CacheLoader; import org.apache.ratis.thirdparty.com.google.protobuf.CodedOutputStream; import org.apache.ratis.util.FileUtils; import org.apache.ratis.util.JavaUtils; @@ -267,15 +266,14 @@ private void assertSegment(long expectedStart, int expectedEntryCount, boolean c * * In the future we can make the cache loader configurable if necessary. */ - class LogEntryLoader extends CacheLoader> { + class LogEntryLoader { private final SegmentedRaftLogMetrics raftLogMetrics; LogEntryLoader(SegmentedRaftLogMetrics raftLogMetrics) { this.raftLogMetrics = raftLogMetrics; } - @Override - public ReferenceCountedObject load(LogRecord key) throws IOException { + ReferenceCountedObject load(TermIndex key) throws IOException { final File file = getFile(); // note the loading should not exceed the endIndex: it is possible that // the on-disk log file should be truncated but has not been done yet. @@ -286,17 +284,16 @@ public ReferenceCountedObject load(LogRecord key) throws IOExcept try { final TermIndex ti = TermIndex.valueOf(entry); putEntryCache(ti, entryRef, Op.LOAD_SEGMENT_FILE); - if (ti.equals(key.getTermIndex())) { + if (ti.equals(key)) { + entryRef.retain(); toReturn.set(entryRef); - } else { - entryRef.release(); } - } catch (Exception e) { + } finally { entryRef.release(); } }); loadingTimes.incrementAndGet(); - return Objects.requireNonNull(toReturn.get()); + return Objects.requireNonNull(toReturn.get(), () -> "toReturn == null for " + key); } } @@ -493,8 +490,8 @@ ReferenceCountedObject getEntryFromCache(TermIndex ti) { /** * Acquire LogSegment's monitor so that there is no concurrent loading. */ - synchronized ReferenceCountedObject loadCache(LogRecord record) throws RaftLogIOException { - ReferenceCountedObject entry = entryCache.get(record.getTermIndex()); + synchronized ReferenceCountedObject loadCache(TermIndex ti) throws RaftLogIOException { + final ReferenceCountedObject entry = entryCache.get(ti); if (entry != null) { try { entry.retain(); @@ -505,9 +502,7 @@ synchronized ReferenceCountedObject loadCache(LogRecord record) t } } try { - return cacheLoader.load(record); - } catch (RaftLogIOException e) { - throw e; + return cacheLoader.load(ti); } catch (Exception e) { throw new RaftLogIOException("Failed to loadCache for log entry " + record, e); } diff --git a/ratis-server/src/main/java/org/apache/ratis/server/raftlog/segmented/SegmentedRaftLog.java b/ratis-server/src/main/java/org/apache/ratis/server/raftlog/segmented/SegmentedRaftLog.java index 6e9a43aa5d..2c7fedbe0c 100644 --- a/ratis-server/src/main/java/org/apache/ratis/server/raftlog/segmented/SegmentedRaftLog.java +++ b/ratis-server/src/main/java/org/apache/ratis/server/raftlog/segmented/SegmentedRaftLog.java @@ -310,7 +310,8 @@ public ReferenceCountedObject retainLog(long index) throws RaftLo if (record == null) { return null; } - final ReferenceCountedObject entry = segment.getEntryFromCache(record.getTermIndex()); + final TermIndex ti = record.getTermIndex(); + final ReferenceCountedObject entry = segment.getEntryFromCache(ti); if (entry != null) { try { entry.retain(); @@ -325,7 +326,7 @@ public ReferenceCountedObject retainLog(long index) throws RaftLo // the entry is not in the segment's cache. Load the cache without holding the lock. getRaftLogMetrics().onRaftLogCacheMiss(); cacheEviction.signal(); - return segment.loadCache(record); + return segment.loadCache(ti); } @Override diff --git a/ratis-test/src/test/java/org/apache/ratis/server/raftlog/segmented/TestLogSegment.java b/ratis-test/src/test/java/org/apache/ratis/server/raftlog/segmented/TestLogSegment.java index 5d87fde3c4..259f163070 100644 --- a/ratis-test/src/test/java/org/apache/ratis/server/raftlog/segmented/TestLogSegment.java +++ b/ratis-test/src/test/java/org/apache/ratis/server/raftlog/segmented/TestLogSegment.java @@ -33,6 +33,7 @@ import org.apache.ratis.thirdparty.com.google.protobuf.CodedOutputStream; import org.apache.ratis.util.FileUtils; import org.apache.ratis.util.Preconditions; +import org.apache.ratis.util.ReferenceCountedObject; import org.apache.ratis.util.SizeInBytes; import org.apache.ratis.util.TraditionalBinaryPrefix; import org.junit.jupiter.api.AfterEach; @@ -57,6 +58,7 @@ /** * Test basic functionality of {@link LogSegment} */ +@SuppressWarnings({"try"}) public class TestLogSegment extends BaseTest { public static final LogSegmentStartEnd ZERO_START_NULL_END = LogSegmentStartEnd.valueOf(0); @@ -142,11 +144,11 @@ static void checkLogSegment(LogSegment segment, long start, long end, Assertions.assertEquals(term, ti.getTerm()); Assertions.assertEquals(offset, record.getOffset()); - ReferenceCountedObject entry = segment.getEntryFromCache(ti); - if (entry == null) { - entry = segment.loadCache(record); + ReferenceCountedObject entryRef = segment.getEntryFromCache(ti); + if (entryRef == null) { + entryRef = segment.loadCache(ti); } - offset += getEntrySize(entry.get(), Op.WRITE_CACHE_WITHOUT_STATE_MACHINE_CACHE); + offset += getEntrySize(entryRef.get(), LogSegment.Op.WRITE_CACHE_WITHOUT_STATE_MACHINE_CACHE); } } @@ -206,7 +208,7 @@ public void testAppendEntries() throws Exception { SimpleOperation op = new SimpleOperation("m" + i); LogEntryProto entry = LogProtoUtils.toLogEntryProto(op.getLogEntryContent(), term, i++ + start); size += getEntrySize(entry, LogSegment.Op.WRITE_CACHE_WITHOUT_STATE_MACHINE_CACHE); - segment.appendToOpenSegment(entry, LogSegment.Op.WRITE_CACHE_WITHOUT_STATE_MACHINE_CACHE, true); + segment.appendToOpenSegment(LogSegment.Op.WRITE_CACHE_WITHOUT_STATE_MACHINE_CACHE, ReferenceCountedObject.wrap(entry)); } Assertions.assertTrue(segment.getTotalFileSize() >= max); @@ -238,18 +240,18 @@ public void testAppendWithGap() throws Exception { final StateMachineLogEntryProto m = op.getLogEntryContent(); try { LogEntryProto entry = LogProtoUtils.toLogEntryProto(m, 0, 1001); - segment.appendToOpenSegment(Op.WRITE_CACHE_WITHOUT_STATE_MACHINE_CACHE, ReferenceCountedObject.wrap(entry)); + segment.appendToOpenSegment(LogSegment.Op.WRITE_CACHE_WITHOUT_STATE_MACHINE_CACHE, ReferenceCountedObject.wrap(entry)); Assertions.fail("should fail since the entry's index needs to be 1000"); } catch (IllegalStateException e) { // the exception is expected. } LogEntryProto entry = LogProtoUtils.toLogEntryProto(m, 0, 1000); - segment.appendToOpenSegment(entry, LogSegment.Op.WRITE_CACHE_WITHOUT_STATE_MACHINE_CACHE, true); + segment.appendToOpenSegment(LogSegment.Op.WRITE_CACHE_WITHOUT_STATE_MACHINE_CACHE, ReferenceCountedObject.wrap(entry)); try { entry = LogProtoUtils.toLogEntryProto(m, 0, 1002); - segment.appendToOpenSegment(Op.WRITE_CACHE_WITHOUT_STATE_MACHINE_CACHE, ReferenceCountedObject.wrap(entry)); + segment.appendToOpenSegment(LogSegment.Op.WRITE_CACHE_WITHOUT_STATE_MACHINE_CACHE, ReferenceCountedObject.wrap(entry)); Assertions.fail("should fail since the entry's index needs to be 1001"); } catch (IllegalStateException e) { // the exception is expected. @@ -264,7 +266,7 @@ public void testTruncate() throws Exception { for (int i = 0; i < 100; i++) { LogEntryProto entry = LogProtoUtils.toLogEntryProto( new SimpleOperation("m" + i).getLogEntryContent(), term, i + start); - segment.appendToOpenSegment(entry, LogSegment.Op.WRITE_CACHE_WITHOUT_STATE_MACHINE_CACHE, true); + segment.appendToOpenSegment(LogSegment.Op.WRITE_CACHE_WITHOUT_STATE_MACHINE_CACHE, ReferenceCountedObject.wrap(entry)); } // truncate an open segment (remove 1080~1099) From 460ba5c57fbc243269f653aecf48a6a2a46eaa96 Mon Sep 17 00:00:00 2001 From: William Song <48054931+SzyWilliam@users.noreply.github.com> Date: Tue, 7 Jan 2025 02:06:08 +0800 Subject: [PATCH 376/397] RATIS-2235. Allow only one thread to perform appendLog (#1206) (cherry picked from commit 9b7440123fe543bb0d0fba1a6dc97da40d0f681a) --- .../ratis/server/impl/RaftServerImpl.java | 18 +++++++++++++----- 1 file changed, 13 insertions(+), 5 deletions(-) diff --git a/ratis-server/src/main/java/org/apache/ratis/server/impl/RaftServerImpl.java b/ratis-server/src/main/java/org/apache/ratis/server/impl/RaftServerImpl.java index f480f72f2e..344bd0d661 100644 --- a/ratis-server/src/main/java/org/apache/ratis/server/impl/RaftServerImpl.java +++ b/ratis-server/src/main/java/org/apache/ratis/server/impl/RaftServerImpl.java @@ -131,6 +131,7 @@ import java.util.concurrent.ThreadLocalRandom; import java.util.concurrent.TimeUnit; import java.util.concurrent.atomic.AtomicBoolean; +import java.util.concurrent.atomic.AtomicReference; import java.util.function.Function; import java.util.function.Supplier; import java.util.stream.Collectors; @@ -254,6 +255,8 @@ public long[] getFollowerMatchIndices() { private final AtomicBoolean firstElectionSinceStartup = new AtomicBoolean(true); private final ThreadGroup threadGroup; + private final AtomicReference> appendLogFuture; + RaftServerImpl(RaftGroup group, StateMachine stateMachine, RaftServerProxy proxy, RaftStorage.StartupOption option) throws IOException { final RaftPeerId id = proxy.getId(); @@ -285,8 +288,7 @@ public long[] getFollowerMatchIndices() { this.transferLeadership = new TransferLeadership(this, properties); this.snapshotRequestHandler = new SnapshotManagementRequestHandler(this); this.snapshotInstallationHandler = new SnapshotInstallationHandler(this, properties); - this.appendLogTermIndices = RaftServerConfigKeys.Log.appendEntriesComposeEnabled(properties) ? - new NavigableIndices() : null; + this.appendLogFuture = new AtomicReference<>(CompletableFuture.completedFuture(null)); this.serverExecutor = ConcurrentUtils.newThreadPoolWithMax( RaftServerConfigKeys.ThreadPool.serverCached(properties), @@ -1642,9 +1644,9 @@ leaderId, getMemberId(), currentTerm, followerCommit, inconsistencyReplyNextInde state.updateConfiguration(entries); } future.join(); + final CompletableFuture appendLog = entries.isEmpty()? CompletableFuture.completedFuture(null) + : appendLog(requestRef.delegate(entries)); - final List> futures = entries.isEmpty() ? Collections.emptyList() - : state.getLog().append(requestRef.delegate(entries)); proto.getCommitInfosList().forEach(commitInfoCache::update); CodeInjectionForTesting.execute(LOG_SYNC, getId(), null); @@ -1658,7 +1660,7 @@ leaderId, getMemberId(), currentTerm, followerCommit, inconsistencyReplyNextInde final long commitIndex = effectiveCommitIndex(proto.getLeaderCommit(), previous, entries.size()); final long matchIndex = isHeartbeat? RaftLog.INVALID_LOG_INDEX: entries.get(entries.size() - 1).getIndex(); - return JavaUtils.allOf(futures).whenCompleteAsync((r, t) -> { + return appendLog.whenCompleteAsync((r, t) -> { followerState.ifPresent(fs -> fs.updateLastRpcTime(FollowerState.UpdateType.APPEND_COMPLETE)); timer.stop(); }, getServerExecutor()).thenApply(v -> { @@ -1675,6 +1677,12 @@ leaderId, getMemberId(), currentTerm, followerCommit, inconsistencyReplyNextInde return reply; }); } + private CompletableFuture appendLog(ReferenceCountedObject> entriesRef) { + entriesRef.retain(); + return appendLogFuture.updateAndGet(f -> f.thenCompose( + ignored -> JavaUtils.allOf(state.getLog().append(entriesRef)))) + .whenComplete((v, e) -> entriesRef.release()); + } private long checkInconsistentAppendEntries(TermIndex previous, List entries) { // Check if a snapshot installation through state machine is in progress. From 867c5f329bc190ba6f2336152d681f9ed941ef70 Mon Sep 17 00:00:00 2001 From: venkatsambath Date: Sun, 2 Feb 2025 23:13:34 -0500 Subject: [PATCH 377/397] RATIS-2244. Reduce the number of log messages during bootstrap (#1217) (cherry picked from commit 2664ac8e50bf52202b581555134cbdf33c057603) --- .../java/org/apache/ratis/client/impl/OrderedAsync.java | 3 ++- .../java/org/apache/ratis/grpc/server/GrpcLogAppender.java | 6 +----- .../apache/ratis/grpc/server/GrpcServerProtocolService.java | 3 +-- 3 files changed, 4 insertions(+), 8 deletions(-) diff --git a/ratis-client/src/main/java/org/apache/ratis/client/impl/OrderedAsync.java b/ratis-client/src/main/java/org/apache/ratis/client/impl/OrderedAsync.java index 791adc8982..a380989641 100644 --- a/ratis-client/src/main/java/org/apache/ratis/client/impl/OrderedAsync.java +++ b/ratis-client/src/main/java/org/apache/ratis/client/impl/OrderedAsync.java @@ -213,7 +213,8 @@ private void sendRequestWithRetry(PendingOrderedRequest pending) { }).exceptionally(e -> { final Throwable exception = e; final String key = client.getId() + "-" + request.getCallId() + "-" + exception; - BatchLogger.print(BatchLogKey.SEND_REQUEST_EXCEPTION, key, prefix -> logError(prefix, request, exception)); + final Consumer op = suffix -> LOG.error("{} {}: Failed* {}", suffix, client.getId(), request, exception); + BatchLogger.print(BatchLogKey.SEND_REQUEST_EXCEPTION, key, op); handleException(pending, request, e); return null; }); diff --git a/ratis-grpc/src/main/java/org/apache/ratis/grpc/server/GrpcLogAppender.java b/ratis-grpc/src/main/java/org/apache/ratis/grpc/server/GrpcLogAppender.java index 1131447242..b3d43808c1 100644 --- a/ratis-grpc/src/main/java/org/apache/ratis/grpc/server/GrpcLogAppender.java +++ b/ratis-grpc/src/main/java/org/apache/ratis/grpc/server/GrpcLogAppender.java @@ -74,11 +74,7 @@ public class GrpcLogAppender extends LogAppenderBase { private enum BatchLogKey implements BatchLogger.Key { RESET_CLIENT, INCONSISTENCY_REPLY, - APPEND_LOG_RESPONSE_HANDLER_ON_ERROR, - INSTALL_SNAPSHOT_NOTIFY, - INSTALL_SNAPSHOT_REPLY, - INSTALL_SNAPSHOT_IN_PROGRESS, - SNAPSHOT_UNAVAILABLE + APPEND_LOG_RESPONSE_HANDLER_ON_ERROR } public static final int INSTALL_SNAPSHOT_NOTIFICATION_INDEX = 0; diff --git a/ratis-grpc/src/main/java/org/apache/ratis/grpc/server/GrpcServerProtocolService.java b/ratis-grpc/src/main/java/org/apache/ratis/grpc/server/GrpcServerProtocolService.java index a02582ff4c..4b5097fb6a 100644 --- a/ratis-grpc/src/main/java/org/apache/ratis/grpc/server/GrpcServerProtocolService.java +++ b/ratis-grpc/src/main/java/org/apache/ratis/grpc/server/GrpcServerProtocolService.java @@ -216,8 +216,7 @@ public void onCompleted() { getId(), op, getPreviousRequestString(), suffix)); requestFuture.get().thenAccept(reply -> { BatchLogger.print(BatchLogKey.COMPLETED_REPLY, getName(), - suffix -> LOG.info("{}: Completed {}, lastReply: {} {}", - getId(), op, ProtoUtils.shortDebugString(reply), suffix)); + suffix -> LOG.info("{}: Completed {}, lastReply: {} {}", getId(), op, reply, suffix)); responseObserver.onCompleted(); }); releaseLast(); From c3159a3a7cd560fc6790e57694d4311704035672 Mon Sep 17 00:00:00 2001 From: Swaminathan Balachandran <47532440+swamirishi@users.noreply.github.com> Date: Wed, 19 Feb 2025 15:06:25 -0800 Subject: [PATCH 378/397] RATIS-2245. Ratis should wait for all apply transaction futures before taking snapshot and group remove (#1218) (cherry picked from commit 663a44b5fe39ceb08f9f2e94a0b85c26da302c54) --- .../server/impl/StateMachineUpdater.java | 2 +- .../impl/StateMachineShutdownTests.java | 53 +++++++++++-------- 2 files changed, 33 insertions(+), 22 deletions(-) diff --git a/ratis-server/src/main/java/org/apache/ratis/server/impl/StateMachineUpdater.java b/ratis-server/src/main/java/org/apache/ratis/server/impl/StateMachineUpdater.java index 125776555f..8c7cb83621 100644 --- a/ratis-server/src/main/java/org/apache/ratis/server/impl/StateMachineUpdater.java +++ b/ratis-server/src/main/java/org/apache/ratis/server/impl/StateMachineUpdater.java @@ -265,7 +265,7 @@ private CompletableFuture applyLog(CompletableFuture applyLogFutures if (f != null) { CompletableFuture exceptionHandledFuture = f.exceptionally(ex -> { LOG.error("Exception while {}: applying txn index={}, nextLog={}", this, nextIndex, - LogProtoUtils.toLogEntryString(next), ex); + LogProtoUtils.toLogEntryString(entry), ex); return null; }); applyLogFutures = applyLogFutures.thenCombine(exceptionHandledFuture, (v, message) -> null); diff --git a/ratis-server/src/test/java/org/apache/ratis/server/impl/StateMachineShutdownTests.java b/ratis-server/src/test/java/org/apache/ratis/server/impl/StateMachineShutdownTests.java index 9fa629420b..e77071ae12 100644 --- a/ratis-server/src/test/java/org/apache/ratis/server/impl/StateMachineShutdownTests.java +++ b/ratis-server/src/test/java/org/apache/ratis/server/impl/StateMachineShutdownTests.java @@ -28,25 +28,21 @@ import org.apache.ratis.statemachine.impl.SimpleStateMachine4Testing; import org.apache.ratis.statemachine.StateMachine; import org.apache.ratis.statemachine.TransactionContext; -import org.junit.jupiter.api.Assertions; -import org.junit.jupiter.api.Test; +import org.junit.*; +import org.mockito.MockedStatic; +import org.mockito.Mockito; import org.slf4j.Logger; import org.slf4j.LoggerFactory; -import java.util.Comparator; -import java.util.HashSet; -import java.util.Map; -import java.util.Set; -import java.util.concurrent.CompletableFuture; -import java.util.concurrent.ConcurrentHashMap; -import java.util.concurrent.ExecutorService; -import java.util.concurrent.Executors; +import java.util.*; +import java.util.concurrent.*; import java.util.concurrent.atomic.AtomicLong; public abstract class StateMachineShutdownTests extends BaseTest implements MiniRaftCluster.Factory.Get { public static Logger LOG = LoggerFactory.getLogger(StateMachineUpdater.class); + private static MockedStatic mocked; protected static class StateMachineWithConditionalWait extends SimpleStateMachine4Testing { boolean unblockAllTxns = false; @@ -76,7 +72,7 @@ public void notifyTermIndexUpdated(long term, long index) { @Override public CompletableFuture applyTransaction(TransactionContext trx) { - final RaftProtos.LogEntryProto entry = trx.getLogEntry(); + final RaftProtos.LogEntryProto entry = trx.getLogEntryUnsafe(); CompletableFuture future = new CompletableFuture<>(); futures.computeIfAbsent(Thread.currentThread().getId(), k -> new HashSet<>()).add(future); @@ -93,10 +89,12 @@ public CompletableFuture applyTransaction(TransactionContext trx) { } } } - } - final RaftProtos.LogEntryProto entry = trx.getLogEntryUnsafe(); - updateLastAppliedTermIndex(entry.getTerm(), entry.getIndex()); - return CompletableFuture.completedFuture(new RaftTestUtil.SimpleMessage("done")); + numTxns.computeIfAbsent(getId(), (k) -> new AtomicLong()).incrementAndGet(); + appliedTxns.put(entry.getIndex(), entry.getTerm()); + updateTxns(); + future.complete(new RaftTestUtil.SimpleMessage("done")); + }); + return future; } public void unBlockApplyTxn(long txnId) { @@ -117,6 +115,19 @@ public void unblockAllTxns() { } } + @Before + public void setup() { + mocked = Mockito.mockStatic(CompletableFuture.class, Mockito.CALLS_REAL_METHODS); + } + + @After + public void tearDownClass() { + if (mocked != null) { + mocked.close(); + } + + } + @Test public void testStateMachineShutdownWaitsForApplyTxn() throws Exception { final RaftProperties prop = getProperties(); @@ -159,24 +170,24 @@ public void testStateMachineShutdownWaitsForApplyTxn() throws Exception { // Now unblock the second follower long minIndex = ((StateMachineWithConditionalWait) secondFollower.getStateMachine()).blockTxns.stream() .min(Comparator.naturalOrder()).get(); - Assertions.assertEquals(2, StateMachineWithConditionalWait.numTxns.values().stream() + Assert.assertEquals(2, StateMachineWithConditionalWait.numTxns.values().stream() .filter(val -> val.get() == 3).count()); // The second follower should still be blocked in apply transaction - Assertions.assertTrue(secondFollower.getInfo().getLastAppliedIndex() < minIndex); + Assert.assertTrue(secondFollower.getInfo().getLastAppliedIndex() < minIndex); for (long index : ((StateMachineWithConditionalWait) secondFollower.getStateMachine()).blockTxns) { if (minIndex != index) { ((StateMachineWithConditionalWait) secondFollower.getStateMachine()).unBlockApplyTxn(index); } } - Assertions.assertEquals(2, StateMachineWithConditionalWait.numTxns.values().stream() + Assert.assertEquals(2, StateMachineWithConditionalWait.numTxns.values().stream() .filter(val -> val.get() == 3).count()); - Assertions.assertTrue(secondFollower.getInfo().getLastAppliedIndex() < minIndex); + Assert.assertTrue(secondFollower.getInfo().getLastAppliedIndex() < minIndex); ((StateMachineWithConditionalWait) secondFollower.getStateMachine()).unBlockApplyTxn(minIndex); // Now wait for the thread t.join(5000); - Assertions.assertTrue(logIndex <= secondFollower.getInfo().getLastAppliedIndex()); - Assertions.assertEquals(3, StateMachineWithConditionalWait.numTxns.values().stream() + Assert.assertEquals(logIndex, secondFollower.getInfo().getLastAppliedIndex()); + Assert.assertEquals(3, StateMachineWithConditionalWait.numTxns.values().stream() .filter(val -> val.get() == 3).count()); cluster.shutdown(); From b55fca98eddf6eb82580833316528ebae57d99f9 Mon Sep 17 00:00:00 2001 From: William Song <48054931+SzyWilliam@users.noreply.github.com> Date: Fri, 21 Feb 2025 03:38:50 +0800 Subject: [PATCH 379/397] RATIS-2242 change consistency criteria of heartbeat during appendLog (#1215) (cherry picked from commit fde8c7d4f37f37c61d5e270cf069a8a604ea0d6e) --- .../ratis/server/impl/RaftServerImpl.java | 21 +++++++++--- .../ratis/server/impl/RaftServerProxy.java | 3 +- .../ratis/server/impl/ServerImplUtils.java | 32 ++++++------------- 3 files changed, 29 insertions(+), 27 deletions(-) diff --git a/ratis-server/src/main/java/org/apache/ratis/server/impl/RaftServerImpl.java b/ratis-server/src/main/java/org/apache/ratis/server/impl/RaftServerImpl.java index 344bd0d661..6736f2faf9 100644 --- a/ratis-server/src/main/java/org/apache/ratis/server/impl/RaftServerImpl.java +++ b/ratis-server/src/main/java/org/apache/ratis/server/impl/RaftServerImpl.java @@ -28,6 +28,7 @@ import org.apache.ratis.proto.RaftProtos.InstallSnapshotRequestProto; import org.apache.ratis.proto.RaftProtos.InstallSnapshotResult; import org.apache.ratis.proto.RaftProtos.LogEntryProto; +import org.apache.ratis.proto.RaftProtos.LogInfoProto; import org.apache.ratis.proto.RaftProtos.RaftClientRequestProto; import org.apache.ratis.proto.RaftProtos.RaftClientRequestProto.TypeCase; import org.apache.ratis.proto.RaftProtos.RaftConfigurationProto; @@ -80,6 +81,8 @@ import org.apache.ratis.server.RaftServerRpc; import org.apache.ratis.server.impl.LeaderElection.Phase; import org.apache.ratis.server.impl.RetryCacheImpl.CacheEntry; +import org.apache.ratis.server.impl.ServerImplUtils.ConsecutiveIndices; +import org.apache.ratis.server.impl.ServerImplUtils.NavigableIndices; import org.apache.ratis.server.leader.LeaderState; import org.apache.ratis.server.metrics.LeaderElectionMetrics; import org.apache.ratis.server.metrics.RaftServerMetricsImpl; @@ -111,6 +114,7 @@ import org.apache.ratis.util.ReferenceCountedObject; import org.apache.ratis.util.TimeDuration; import org.apache.ratis.util.function.CheckedSupplier; +import org.apache.ratis.util.function.UncheckedAutoCloseableSupplier; import java.io.File; import java.io.IOException; @@ -126,6 +130,7 @@ import java.util.Set; import java.util.concurrent.CompletableFuture; import java.util.concurrent.CompletionException; +import java.util.concurrent.CountDownLatch; import java.util.concurrent.ExecutionException; import java.util.concurrent.ExecutorService; import java.util.concurrent.ThreadLocalRandom; @@ -256,6 +261,7 @@ public long[] getFollowerMatchIndices() { private final ThreadGroup threadGroup; private final AtomicReference> appendLogFuture; + private final NavigableIndices appendLogTermIndices = new NavigableIndices(); RaftServerImpl(RaftGroup group, StateMachine stateMachine, RaftServerProxy proxy, RaftStorage.StartupOption option) throws IOException { @@ -1678,10 +1684,19 @@ leaderId, getMemberId(), currentTerm, followerCommit, inconsistencyReplyNextInde }); } private CompletableFuture appendLog(ReferenceCountedObject> entriesRef) { + final List entriesTermIndices; + try(UncheckedAutoCloseableSupplier> entries = entriesRef.retainAndReleaseOnClose()) { + entriesTermIndices = ConsecutiveIndices.convert(entries.get()); + appendLogTermIndices.append(entriesTermIndices); + } + entriesRef.retain(); return appendLogFuture.updateAndGet(f -> f.thenCompose( ignored -> JavaUtils.allOf(state.getLog().append(entriesRef)))) - .whenComplete((v, e) -> entriesRef.release()); + .whenComplete((v, e) -> { + entriesRef.release(); + appendLogTermIndices.removeExisting(entriesTermIndices); + }); } private long checkInconsistentAppendEntries(TermIndex previous, List entries) { @@ -1708,9 +1723,7 @@ private long checkInconsistentAppendEntries(TermIndex previous, List appendEntriesAsync( try { final RaftGroupId groupId = ProtoUtils.toRaftGroupId(request.getServerRequest().getRaftGroupId()); return getImplFuture(groupId) - .thenCompose(impl -> impl.executeSubmitServerRequestAsync(() -> impl.appendEntriesAsync(requestRef))); + .thenCompose(impl -> JavaUtils.callAsUnchecked( + () -> impl.appendEntriesAsync(requestRef), CompletionException::new)); } finally { requestRef.release(); } diff --git a/ratis-server/src/main/java/org/apache/ratis/server/impl/ServerImplUtils.java b/ratis-server/src/main/java/org/apache/ratis/server/impl/ServerImplUtils.java index d6035ff778..2425e40f6e 100644 --- a/ratis-server/src/main/java/org/apache/ratis/server/impl/ServerImplUtils.java +++ b/ratis-server/src/main/java/org/apache/ratis/server/impl/ServerImplUtils.java @@ -43,6 +43,9 @@ import java.util.ArrayList; import java.util.Collections; import java.util.List; +import java.util.Map; +import java.util.NavigableMap; +import java.util.TreeMap; import java.util.concurrent.TimeUnit; import java.util.concurrent.atomic.AtomicReference; import java.util.function.Function; @@ -117,8 +120,6 @@ Long getTerm(long index) { /** A data structure to support the {@link #contains(TermIndex)} method. */ static class NavigableIndices { private final NavigableMap map = new TreeMap<>(); - private final AtomicReference> future - = new AtomicReference<>(CompletableFuture.completedFuture(null)); boolean contains(TermIndex ti) { final Long term = getTerm(ti.getIndex()); @@ -137,28 +138,15 @@ synchronized Long getTerm(long index) { return floorEntry.getValue().getTerm(index); } - CompletableFuture append(List entries, - Function, CompletableFuture> appendLog) { - final List entriesTermIndices = ConsecutiveIndices.convert(entries); - return alreadyExists(entriesTermIndices) ? future.get() - : future.updateAndGet(f -> f.thenComposeAsync(ignored -> appendLog.apply(entries))) - .whenComplete((v, e) -> removeExisting(entriesTermIndices)); - } - - private synchronized boolean alreadyExists(List entriesTermIndices) { - for(int i = 0; i < entriesTermIndices.size(); i++) { - final ConsecutiveIndices indices = entriesTermIndices.get(i); - final ConsecutiveIndices previous = map.put(indices.startIndex, indices); - if (previous != null) { - // index already exists, revert this append - map.put(previous.startIndex, previous); - for(int j = 0; j < i; j++) { - map.remove(entriesTermIndices.get(j).startIndex); - } - return true; + synchronized void append(List entriesTermIndices) { + for(ConsecutiveIndices indices : entriesTermIndices) { + // validate startIndex + final Map.Entry lastEntry = map.lastEntry(); + if (lastEntry != null) { + Preconditions.assertSame(lastEntry.getValue().getNextIndex(), indices.startIndex, "startIndex"); } + map.put(indices.startIndex, indices); } - return false; } synchronized void removeExisting(List entriesTermIndices) { From fd56bd31ccc9b810d8fc4d6317a558fb7a06eddb Mon Sep 17 00:00:00 2001 From: "Doroszlai, Attila" <6454655+adoroszlai@users.noreply.github.com> Date: Wed, 26 Feb 2025 18:40:39 +0100 Subject: [PATCH 380/397] RATIS-2217. Automatically re-try flaky tests in CI (#1229) (cherry picked from commit e604110af2a31708e047858db45f4d47498495df) --- pom.xml | 7 +++---- ratis-docs/pom.xml | 13 ------------- .../java/org/apache/ratis/TestMultiRaftGroup.java | 2 ++ ratis-replicated-map/pom.xml | 5 +++++ .../ratis/server/impl/LeaderElectionTests.java | 1 + .../org/apache/ratis/grpc/TestRaftWithGrpc.java | 1 + 6 files changed, 12 insertions(+), 17 deletions(-) diff --git a/pom.xml b/pom.xml index 972be8aaa8..1c23fe78d4 100644 --- a/pom.xml +++ b/pom.xml @@ -190,10 +190,9 @@ 4 - 2.0.17 - 5.14.3 - 4.11.0 - 0.8.14 + 2.0.7 + 5.11.2 + 0.8.12 flaky | org.apache.ratis.test.tag.FlakyTest diff --git a/ratis-docs/pom.xml b/ratis-docs/pom.xml index e0cbc5fcee..1bb1c64589 100644 --- a/ratis-docs/pom.xml +++ b/ratis-docs/pom.xml @@ -32,17 +32,4 @@ https://maven.apache.org/xsd/maven-4.0.0.xsd"> true - - - org.junit.jupiter - junit-jupiter-engine - test - - - org.junit.platform - junit-platform-launcher - test - - - diff --git a/ratis-examples/src/test/java/org/apache/ratis/TestMultiRaftGroup.java b/ratis-examples/src/test/java/org/apache/ratis/TestMultiRaftGroup.java index ea3962c088..5ce96da39a 100644 --- a/ratis-examples/src/test/java/org/apache/ratis/TestMultiRaftGroup.java +++ b/ratis-examples/src/test/java/org/apache/ratis/TestMultiRaftGroup.java @@ -24,6 +24,7 @@ import org.apache.ratis.protocol.RaftGroup; import org.apache.ratis.server.impl.GroupManagementBaseTest; import org.apache.ratis.server.impl.MiniRaftCluster; +import org.apache.ratis.test.tag.Flaky; import org.apache.ratis.util.function.CheckedBiConsumer; import org.junit.jupiter.api.Timeout; import org.junit.jupiter.params.ParameterizedTest; @@ -33,6 +34,7 @@ import java.util.Collection; import java.util.concurrent.atomic.AtomicInteger; +@Flaky("RATIS-2218") @Timeout(value = 300) public class TestMultiRaftGroup extends BaseTest { public static Collection data() { diff --git a/ratis-replicated-map/pom.xml b/ratis-replicated-map/pom.xml index bded7cd36d..e6144fd103 100644 --- a/ratis-replicated-map/pom.xml +++ b/ratis-replicated-map/pom.xml @@ -23,6 +23,11 @@ ratis-replicated-map Apache Ratis Replicated Map + + + true + + org.apache.ratis diff --git a/ratis-server/src/test/java/org/apache/ratis/server/impl/LeaderElectionTests.java b/ratis-server/src/test/java/org/apache/ratis/server/impl/LeaderElectionTests.java index ff604f5631..0b8f51bb51 100644 --- a/ratis-server/src/test/java/org/apache/ratis/server/impl/LeaderElectionTests.java +++ b/ratis-server/src/test/java/org/apache/ratis/server/impl/LeaderElectionTests.java @@ -37,6 +37,7 @@ import org.apache.ratis.server.RaftServerConfigKeys; import org.apache.ratis.server.metrics.LeaderElectionMetrics; import org.apache.ratis.server.protocol.TermIndex; +import org.apache.ratis.test.tag.Flaky; import org.apache.ratis.thirdparty.com.codahale.metrics.Timer; import org.apache.ratis.util.CodeInjectionForTesting; import org.apache.ratis.util.JavaUtils; diff --git a/ratis-test/src/test/java/org/apache/ratis/grpc/TestRaftWithGrpc.java b/ratis-test/src/test/java/org/apache/ratis/grpc/TestRaftWithGrpc.java index 3bd3b3f754..958ceefa54 100644 --- a/ratis-test/src/test/java/org/apache/ratis/grpc/TestRaftWithGrpc.java +++ b/ratis-test/src/test/java/org/apache/ratis/grpc/TestRaftWithGrpc.java @@ -53,6 +53,7 @@ public static Collection data() { return Arrays.asList((new Boolean[][] {{Boolean.FALSE}, {Boolean.TRUE}})); } + @Flaky("RATIS-2253") @ParameterizedTest @MethodSource("data") public void testWithLoad(Boolean separateHeartbeat) throws Exception { From 07294e303ade587b9c7cc32c81e34b67f00d2c7a Mon Sep 17 00:00:00 2001 From: Tsz-Wo Nicholas Sze Date: Wed, 26 Feb 2025 12:39:03 -0800 Subject: [PATCH 381/397] RATIS-2254. Replace junit 4 Timeout with junit 5. (#1230) (cherry picked from commit a477d6d289982a20d479f58fdc6a9daf835b1162) --- .../test/java/org/apache/ratis/BaseTest.java | 32 ++++++++++++------- .../java/org/apache/ratis/RaftTestUtil.java | 5 +++ .../DataStreamAsyncClusterTests.java | 1 + .../apache/ratis/grpc/TestRaftWithGrpc.java | 23 +++++++------ .../retry/TestMultipleLinearRandomRetry.java | 10 ++++-- .../apache/ratis/retry/TestRetryPolicy.java | 1 + 6 files changed, 48 insertions(+), 24 deletions(-) diff --git a/ratis-common/src/test/java/org/apache/ratis/BaseTest.java b/ratis-common/src/test/java/org/apache/ratis/BaseTest.java index 653213b3c3..7f0b489f92 100644 --- a/ratis-common/src/test/java/org/apache/ratis/BaseTest.java +++ b/ratis-common/src/test/java/org/apache/ratis/BaseTest.java @@ -26,6 +26,9 @@ import org.apache.ratis.util.StringUtils; import org.apache.ratis.util.TimeDuration; import org.apache.ratis.util.function.CheckedRunnable; +import org.junit.After; +import org.junit.Before; +import org.junit.Rule; import org.junit.jupiter.api.AfterEach; import org.junit.jupiter.api.Assertions; import org.junit.jupiter.api.Assumptions; @@ -70,16 +73,30 @@ public void setFirstException(Throwable e) { } } + // TODO: Junit 4 reference should be removed once all the unit tests are migrated to Junit 5. + private String testCaseName; @BeforeEach public void setup(TestInfo testInfo) { - testCaseName = testInfo.getTestMethod() - .orElseThrow(() -> new RuntimeException("Exception while getting test name.")) - .getName(); + checkAssumptions(); + + final Method method = testInfo.getTestMethod().orElse(null); + testCaseName = testInfo.getTestClass().orElse(getClass()).getSimpleName() + + "." + (method == null? null : method.getName()); + } + // @Before annotation is retained to support junit 4 tests. + @Before + public void checkAssumptions() { final int leaks = ReferenceCountedLeakDetector.getLeakDetector().getLeakCount(); Assumptions.assumeFalse(0 < leaks, () -> "numLeaks " + leaks + " > 0"); + + final Throwable first = firstException.get(); + Assumptions.assumeTrue(first == null, () -> "Already failed with " + first); + + final Throwable exited = ExitUtils.getFirstExitException(); + Assumptions.assumeTrue(exited == null, () -> "Already exited with " + exited); } @AfterEach @@ -92,19 +109,10 @@ public void assertNoFailures() { ExitUtils.assertNotTerminated(); } - // Retained to support junit 4 tests. - @Rule - public final org.junit.rules.Timeout globalTimeout = new org.junit.rules.Timeout( - getGlobalTimeoutSeconds(), TimeUnit.SECONDS ); - // Retained to support junit 4 tests. @Rule public final TestName testName = new TestName(); - public int getGlobalTimeoutSeconds() { - return 100; - } - private static final Supplier ROOT_TEST_DIR = JavaUtils.memoize( () -> JavaUtils.callAsUnchecked(() -> { final File dir = new File(System.getProperty("test.build.data", "target/test/data"), diff --git a/ratis-server/src/test/java/org/apache/ratis/RaftTestUtil.java b/ratis-server/src/test/java/org/apache/ratis/RaftTestUtil.java index 360cb922c6..c94d16b039 100644 --- a/ratis-server/src/test/java/org/apache/ratis/RaftTestUtil.java +++ b/ratis-server/src/test/java/org/apache/ratis/RaftTestUtil.java @@ -478,6 +478,11 @@ static List getPeersWithPriority(List peers, RaftPeer sugges return peersWithPriority; } + static RaftPeerId changeLeader(MiniRaftCluster cluster, RaftPeerId oldLeader) + throws Exception { + return changeLeader(cluster, oldLeader, AssumptionViolatedException::new); + } + static RaftPeerId changeLeader(MiniRaftCluster cluster, RaftPeerId oldLeader) throws Exception { return changeLeader(cluster, oldLeader, Assumptions::abort); diff --git a/ratis-test/src/test/java/org/apache/ratis/datastream/DataStreamAsyncClusterTests.java b/ratis-test/src/test/java/org/apache/ratis/datastream/DataStreamAsyncClusterTests.java index 6fd5b47213..eb25a369e6 100644 --- a/ratis-test/src/test/java/org/apache/ratis/datastream/DataStreamAsyncClusterTests.java +++ b/ratis-test/src/test/java/org/apache/ratis/datastream/DataStreamAsyncClusterTests.java @@ -36,6 +36,7 @@ import org.apache.ratis.util.function.CheckedBiFunction; import org.junit.jupiter.api.Assertions; import org.junit.jupiter.api.Test; +import org.junit.jupiter.api.Timeout; import org.slf4j.event.Level; import java.io.IOException; diff --git a/ratis-test/src/test/java/org/apache/ratis/grpc/TestRaftWithGrpc.java b/ratis-test/src/test/java/org/apache/ratis/grpc/TestRaftWithGrpc.java index 958ceefa54..42211cefc1 100644 --- a/ratis-test/src/test/java/org/apache/ratis/grpc/TestRaftWithGrpc.java +++ b/ratis-test/src/test/java/org/apache/ratis/grpc/TestRaftWithGrpc.java @@ -32,8 +32,10 @@ import org.apache.ratis.util.JavaUtils; import org.apache.ratis.util.TimeDuration; import org.junit.jupiter.api.Assertions; +import org.junit.jupiter.api.Disabled; +import org.junit.jupiter.api.Timeout; import org.junit.jupiter.params.ParameterizedTest; -import org.junit.jupiter.params.provider.MethodSource; +import org.junit.jupiter.params.provider.ValueSource; import java.util.concurrent.CompletableFuture; import java.util.concurrent.TimeUnit; @@ -49,29 +51,32 @@ public class TestRaftWithGrpc SimpleStateMachine4Testing.class, StateMachine.class); } - public static Collection data() { - return Arrays.asList((new Boolean[][] {{Boolean.FALSE}, {Boolean.TRUE}})); + @Disabled + @Override + public void testWithLoad() { + // skip testWithLoad() from parent, called from parameterized testWithLoad(boolean) } @Flaky("RATIS-2253") + @Timeout(300) @ParameterizedTest - @MethodSource("data") - public void testWithLoad(Boolean separateHeartbeat) throws Exception { + @ValueSource(booleans = {true, false}) + public void testWithLoad(boolean separateHeartbeat) throws Exception { GrpcConfigKeys.Server.setHeartbeatChannel(getProperties(), separateHeartbeat); super.testWithLoad(); BlockRequestHandlingInjection.getInstance().unblockAll(); } @ParameterizedTest - @MethodSource("data") - public void testRequestTimeout(Boolean separateHeartbeat) throws Exception { + @ValueSource(booleans = {true, false}) + public void testRequestTimeout(boolean separateHeartbeat) throws Exception { GrpcConfigKeys.Server.setHeartbeatChannel(getProperties(), separateHeartbeat); runWithNewCluster(NUM_SERVERS, cluster -> testRequestTimeout(false, cluster, LOG)); } @ParameterizedTest - @MethodSource("data") - public void testUpdateViaHeartbeat(Boolean separateHeartbeat) throws Exception { + @ValueSource(booleans = {true, false}) + public void testUpdateViaHeartbeat(boolean separateHeartbeat) throws Exception { GrpcConfigKeys.Server.setHeartbeatChannel(getProperties(), separateHeartbeat); runWithNewCluster(NUM_SERVERS, this::runTestUpdateViaHeartbeat); } diff --git a/ratis-test/src/test/java/org/apache/ratis/retry/TestMultipleLinearRandomRetry.java b/ratis-test/src/test/java/org/apache/ratis/retry/TestMultipleLinearRandomRetry.java index 2a37062602..621d46b5b3 100644 --- a/ratis-test/src/test/java/org/apache/ratis/retry/TestMultipleLinearRandomRetry.java +++ b/ratis-test/src/test/java/org/apache/ratis/retry/TestMultipleLinearRandomRetry.java @@ -21,6 +21,7 @@ import org.apache.ratis.util.TimeDuration; import org.junit.jupiter.api.Assertions; import org.junit.jupiter.api.Test; +import org.junit.jupiter.api.Timeout; @Timeout(value = 1) public class TestMultipleLinearRandomRetry extends BaseTest { @@ -43,9 +44,12 @@ public void testParseCommaSeparated() { assertLegalInput("[10x100ms, 20x1s, 30x5s]", "100,10, 1s,20, 5s,30"); } - private static void assertIllegalInput(String input) { - final MultipleLinearRandomRetry computed = MultipleLinearRandomRetry.parseCommaSeparated(input); - Assertions.assertNull(computed); + private void assertIllegalInput(String input) { + try { + MultipleLinearRandomRetry.parseCommaSeparated(input); + } catch (IllegalArgumentException e) { + LOG.info("Expected to catch: {}", String.valueOf(e)); + } } private static MultipleLinearRandomRetry assertLegalInput(String expected, String input) { diff --git a/ratis-test/src/test/java/org/apache/ratis/retry/TestRetryPolicy.java b/ratis-test/src/test/java/org/apache/ratis/retry/TestRetryPolicy.java index e1d41045ff..43b2fedd1a 100644 --- a/ratis-test/src/test/java/org/apache/ratis/retry/TestRetryPolicy.java +++ b/ratis-test/src/test/java/org/apache/ratis/retry/TestRetryPolicy.java @@ -36,6 +36,7 @@ import org.apache.ratis.util.Timestamp; import org.junit.jupiter.api.Assertions; import org.junit.jupiter.api.Test; +import org.junit.jupiter.api.Timeout; import java.io.IOException; import java.util.ArrayList; From 52b1930644be5e9376c8b6109c585026f92e8f5c Mon Sep 17 00:00:00 2001 From: slfan1989 <55643692+slfan1989@users.noreply.github.com> Date: Wed, 5 Mar 2025 02:51:04 +0800 Subject: [PATCH 382/397] RATIS-2124. Remove the use of org.junit.Rule. (#1232) (cherry picked from commit 0091c1ca4975efefd1c137877c5b5fb08fb662b4) --- .../test/java/org/apache/ratis/BaseTest.java | 7 ++-- .../statemachine/RaftSnapshotBaseTest.java | 33 +++++++++++++++++-- 2 files changed, 32 insertions(+), 8 deletions(-) diff --git a/ratis-common/src/test/java/org/apache/ratis/BaseTest.java b/ratis-common/src/test/java/org/apache/ratis/BaseTest.java index 7f0b489f92..e3e81a0a71 100644 --- a/ratis-common/src/test/java/org/apache/ratis/BaseTest.java +++ b/ratis-common/src/test/java/org/apache/ratis/BaseTest.java @@ -28,7 +28,6 @@ import org.apache.ratis.util.function.CheckedRunnable; import org.junit.After; import org.junit.Before; -import org.junit.Rule; import org.junit.jupiter.api.AfterEach; import org.junit.jupiter.api.Assertions; import org.junit.jupiter.api.Assumptions; @@ -88,6 +87,7 @@ public void setup(TestInfo testInfo) { // @Before annotation is retained to support junit 4 tests. @Before + @BeforeEach public void checkAssumptions() { final int leaks = ReferenceCountedLeakDetector.getLeakDetector().getLeakCount(); Assumptions.assumeFalse(0 < leaks, () -> "numLeaks " + leaks + " > 0"); @@ -109,10 +109,6 @@ public void assertNoFailures() { ExitUtils.assertNotTerminated(); } - // Retained to support junit 4 tests. - @Rule - public final TestName testName = new TestName(); - private static final Supplier ROOT_TEST_DIR = JavaUtils.memoize( () -> JavaUtils.callAsUnchecked(() -> { final File dir = new File(System.getProperty("test.build.data", "target/test/data"), @@ -135,6 +131,7 @@ public File getClassTestDir() { } public File getTestDir() { + // This will work for both junit 4 and 5. return new File(getClassTestDir(), testCaseName); } diff --git a/ratis-server/src/test/java/org/apache/ratis/statemachine/RaftSnapshotBaseTest.java b/ratis-server/src/test/java/org/apache/ratis/statemachine/RaftSnapshotBaseTest.java index b4469b800d..c87116e762 100644 --- a/ratis-server/src/test/java/org/apache/ratis/statemachine/RaftSnapshotBaseTest.java +++ b/ratis-server/src/test/java/org/apache/ratis/statemachine/RaftSnapshotBaseTest.java @@ -52,6 +52,8 @@ import org.apache.ratis.util.LifeCycle; import org.apache.ratis.util.Slf4jUtils; import org.junit.jupiter.api.Assertions; +import org.junit.jupiter.api.AfterEach; +import org.junit.jupiter.api.BeforeEach; import org.junit.jupiter.api.Test; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -101,10 +103,12 @@ public static void checkMetadataEntry(RaftServer.Division server) throws Excepti final RaftLog log = server.getRaftLog(); final long lastIndex = log.getLastEntryTermIndex().getIndex(); final LogEntryProto e = getLogUnsafe(log, lastIndex); - Assert.assertTrue(e.hasMetadataEntry()); + Assertions.assertTrue(e.hasMetadataEntry()); - public static void assertLogContent(RaftServer.Division server, boolean isLeader) throws Exception { - JavaUtils.attempt(() -> checkMetadataEntry(server), 50, HUNDRED_MILLIS, "checkMetadataEntry", LOG); + JavaUtils.attemptRepeatedly(() -> { + Assertions.assertEquals(log.getLastCommittedIndex() - 1, e.getMetadataEntry().getCommitIndex()); + return null; + }, 50, BaseTest.HUNDRED_MILLIS, "CheckMetadataEntry", LOG); SimpleStateMachine4Testing simpleStateMachine = SimpleStateMachine4Testing.get(server); if (isLeader) { @@ -122,6 +126,29 @@ public static void assertLogContent(RaftServer.Division server, boolean isLeader } } + private MiniRaftCluster cluster; + + public abstract MiniRaftCluster.Factory getFactory(); + + @BeforeEach + public void setup() throws IOException { + final RaftProperties prop = new RaftProperties(); + prop.setClass(MiniRaftCluster.STATEMACHINE_CLASS_KEY, + SimpleStateMachine4Testing.class, StateMachine.class); + RaftServerConfigKeys.Snapshot.setAutoTriggerThreshold( + prop, SNAPSHOT_TRIGGER_THRESHOLD); + RaftServerConfigKeys.Snapshot.setAutoTriggerEnabled(prop, true); + this.cluster = getFactory().newCluster(1, prop); + cluster.start(); + } + + @AfterEach + public void tearDown() { + if (cluster != null) { + cluster.shutdown(); + } + } + /** * Keep generating writing traffic and make sure snapshots are taken. * We then restart the whole raft peer and check if it can correctly load From 60f5491364a0d0801caf461624a9c08fcd2bea58 Mon Sep 17 00:00:00 2001 From: Tsz-Wo Nicholas Sze Date: Wed, 5 Mar 2025 08:38:37 -0800 Subject: [PATCH 383/397] RATIS-2255. Migrate ratis-grpc to junit 5. (#1233) (cherry picked from commit d740e513555a3c4bbf26cab061bca5ff63bb1859) --- ratis-grpc/pom.xml | 5 +++++ .../org/apache/ratis/grpc/MiniRaftClusterWithGrpc.java | 8 ++++---- 2 files changed, 9 insertions(+), 4 deletions(-) diff --git a/ratis-grpc/pom.xml b/ratis-grpc/pom.xml index e73b1f03dc..ced645fa79 100644 --- a/ratis-grpc/pom.xml +++ b/ratis-grpc/pom.xml @@ -73,6 +73,11 @@ junit-jupiter-api test + + org.mockito + mockito-core + test + diff --git a/ratis-grpc/src/test/java/org/apache/ratis/grpc/MiniRaftClusterWithGrpc.java b/ratis-grpc/src/test/java/org/apache/ratis/grpc/MiniRaftClusterWithGrpc.java index 1519298f38..bd1c72b241 100644 --- a/ratis-grpc/src/test/java/org/apache/ratis/grpc/MiniRaftClusterWithGrpc.java +++ b/ratis-grpc/src/test/java/org/apache/ratis/grpc/MiniRaftClusterWithGrpc.java @@ -33,7 +33,7 @@ import org.apache.ratis.server.impl.RaftServerTestUtil; import org.apache.ratis.util.NetUtils; import org.apache.ratis.util.ReferenceCountedLeakDetector; -import org.junit.Assert; +import org.junit.jupiter.api.Assertions; import java.util.Optional; @@ -104,9 +104,9 @@ public void assertZeroCopyMetrics() { RaftServer.Division division = RaftServerTestUtil.getDivision(server, id); final GrpcServicesImpl service = (GrpcServicesImpl) RaftServerTestUtil.getServerRpc(division); ZeroCopyMetrics zeroCopyMetrics = service.getZeroCopyMetrics(); - Assert.assertEquals(0, zeroCopyMetrics.nonZeroCopyMessages()); - Assert.assertEquals("Zero copy messages are not released, please check logs to find leaks. ", - zeroCopyMetrics.zeroCopyMessages(), zeroCopyMetrics.releasedMessages()); + Assertions.assertEquals(0, zeroCopyMetrics.nonZeroCopyMessages()); + Assertions.assertEquals(zeroCopyMetrics.zeroCopyMessages(), zeroCopyMetrics.releasedMessages(), + "Unreleased zero copy messages: please check logs to find the leaks. "); })); } } From cd996f884ea9e254940eb9a882751f67cb495582 Mon Sep 17 00:00:00 2001 From: Tsz-Wo Nicholas Sze Date: Wed, 19 Mar 2025 02:30:47 -0700 Subject: [PATCH 384/397] RATIS-2258. Caching TermIndex objects (#1239) (cherry picked from commit ac35188a4fb285842a05224141e3879aaf71e621) --- .../org/apache/ratis/util/BiWeakValueCache.java | 8 +++----- .../org/apache/ratis/server/protocol/TermIndex.java | 13 ------------- 2 files changed, 3 insertions(+), 18 deletions(-) diff --git a/ratis-common/src/main/java/org/apache/ratis/util/BiWeakValueCache.java b/ratis-common/src/main/java/org/apache/ratis/util/BiWeakValueCache.java index d7eaf5744a..c1aa6bcd5a 100644 --- a/ratis-common/src/main/java/org/apache/ratis/util/BiWeakValueCache.java +++ b/ratis-common/src/main/java/org/apache/ratis/util/BiWeakValueCache.java @@ -33,15 +33,13 @@ * Note that the cached values are weakly referenced. * A cached value could be garage-collected (i.e. evicted from the cache) * when there are no external (strong) references. - *

- * For key types with a component, use {@link WeakValueCache}. * * @param the type of the outer keys. * @param the type of the inner keys. * @param the type to be cached. */ public final class BiWeakValueCache { - static ConcurrentMap newMap() { + private static ConcurrentMap newMap() { return new MapMaker().weakValues().makeMap(); } @@ -63,8 +61,8 @@ static ConcurrentMap newMap() { /** * Create a cache for mapping ({@link OUTER}, {@link INNER}) keys to {@link T} values. * - * @param outerName the name of the outer keys. - * @param innerName the name of the inner keys. + * @param outerName the name of the outer long. + * @param innerName the name of the inner long. * @param constructor for constructing {@link T} values. */ public BiWeakValueCache(String outerName, String innerName, BiFunction constructor) { diff --git a/ratis-server-api/src/main/java/org/apache/ratis/server/protocol/TermIndex.java b/ratis-server-api/src/main/java/org/apache/ratis/server/protocol/TermIndex.java index 369aefc85f..745f366d13 100644 --- a/ratis-server-api/src/main/java/org/apache/ratis/server/protocol/TermIndex.java +++ b/ratis-server-api/src/main/java/org/apache/ratis/server/protocol/TermIndex.java @@ -21,7 +21,6 @@ import org.apache.ratis.proto.RaftProtos.TermIndexProto; import org.apache.ratis.server.raftlog.RaftLog; import org.apache.ratis.util.BiWeakValueCache; -import org.apache.ratis.util.MemoizedSupplier; import java.util.Comparator; import java.util.Optional; @@ -96,8 +95,6 @@ static BiWeakValueCache getCache() { private static TermIndex newTermIndex(long term, long index) { return new TermIndex() { - private final Supplier protoSupplier = MemoizedSupplier.valueOf(TermIndex.super::toProto); - @Override public long getTerm() { return term; @@ -126,22 +123,12 @@ public int hashCode() { return Long.hashCode(term) ^ Long.hashCode(index); } - @Override - public TermIndexProto toProto() { - return protoSupplier.get(); - } - private String longToString(long n) { return n >= 0L ? String.valueOf(n) : "~"; } @Override public String toString() { - if (this.equals(INITIAL_VALUE)) { - return ""; - } else if (this.equals(PROTO_DEFAULT)) { - return ""; - } return String.format("(t:%s, i:%s)", longToString(term), longToString(index)); } }; From c81ce58065217601e6e5c7ca9500dc2245f233cd Mon Sep 17 00:00:00 2001 From: hfutatzhanghb Date: Tue, 25 Mar 2025 23:29:54 +0800 Subject: [PATCH 385/397] RATIS-2269. Remove all import java.util.* statements. (#1243) (cherry picked from commit 72c17652f0bd8446f59fd13ed9e295c278e1fb95) --- .../src/main/java/org/apache/ratis/util/FileUtils.java | 1 - .../raftlog/segmented/SegmentedRaftLogCache.java | 1 + .../raftlog/segmented/SegmentedRaftLogWorker.java | 4 +++- .../ratis/server/impl/StateMachineShutdownTests.java | 10 ++++++++-- .../test/java/org/apache/ratis/util/TestLifeCycle.java | 7 ++++++- 5 files changed, 18 insertions(+), 5 deletions(-) diff --git a/ratis-common/src/main/java/org/apache/ratis/util/FileUtils.java b/ratis-common/src/main/java/org/apache/ratis/util/FileUtils.java index 79c00b5a35..59db028986 100644 --- a/ratis-common/src/main/java/org/apache/ratis/util/FileUtils.java +++ b/ratis-common/src/main/java/org/apache/ratis/util/FileUtils.java @@ -33,7 +33,6 @@ import java.nio.file.FileVisitResult; import java.nio.file.Files; import java.nio.file.LinkOption; -import java.nio.file.NotDirectoryException; import java.nio.file.OpenOption; import java.nio.file.Path; import java.nio.file.Paths; diff --git a/ratis-server/src/main/java/org/apache/ratis/server/raftlog/segmented/SegmentedRaftLogCache.java b/ratis-server/src/main/java/org/apache/ratis/server/raftlog/segmented/SegmentedRaftLogCache.java index 66a7c8e511..5c482db41c 100644 --- a/ratis-server/src/main/java/org/apache/ratis/server/raftlog/segmented/SegmentedRaftLogCache.java +++ b/ratis-server/src/main/java/org/apache/ratis/server/raftlog/segmented/SegmentedRaftLogCache.java @@ -43,6 +43,7 @@ import java.util.Collections; import java.util.Comparator; import java.util.Iterator; +import java.util.LinkedList; import java.util.List; import java.util.NoSuchElementException; import java.util.Objects; diff --git a/ratis-server/src/main/java/org/apache/ratis/server/raftlog/segmented/SegmentedRaftLogWorker.java b/ratis-server/src/main/java/org/apache/ratis/server/raftlog/segmented/SegmentedRaftLogWorker.java index a44be6ee55..e13d7105a1 100644 --- a/ratis-server/src/main/java/org/apache/ratis/server/raftlog/segmented/SegmentedRaftLogWorker.java +++ b/ratis-server/src/main/java/org/apache/ratis/server/raftlog/segmented/SegmentedRaftLogWorker.java @@ -51,7 +51,9 @@ import java.util.Objects; import java.util.Optional; import java.util.Queue; -import java.util.concurrent.*; +import java.util.concurrent.CompletableFuture; +import java.util.concurrent.ExecutorService; +import java.util.concurrent.TimeUnit; import java.util.concurrent.atomic.AtomicReference; import java.util.function.Consumer; import java.util.function.Supplier; diff --git a/ratis-server/src/test/java/org/apache/ratis/server/impl/StateMachineShutdownTests.java b/ratis-server/src/test/java/org/apache/ratis/server/impl/StateMachineShutdownTests.java index e77071ae12..3ab1cf07bd 100644 --- a/ratis-server/src/test/java/org/apache/ratis/server/impl/StateMachineShutdownTests.java +++ b/ratis-server/src/test/java/org/apache/ratis/server/impl/StateMachineShutdownTests.java @@ -34,8 +34,14 @@ import org.slf4j.Logger; import org.slf4j.LoggerFactory; -import java.util.*; -import java.util.concurrent.*; +import java.util.Comparator; +import java.util.HashSet; +import java.util.Map; +import java.util.Set; +import java.util.concurrent.CompletableFuture; +import java.util.concurrent.ConcurrentHashMap; +import java.util.concurrent.ExecutorService; +import java.util.concurrent.Executors; import java.util.concurrent.atomic.AtomicLong; public abstract class StateMachineShutdownTests diff --git a/ratis-test/src/test/java/org/apache/ratis/util/TestLifeCycle.java b/ratis-test/src/test/java/org/apache/ratis/util/TestLifeCycle.java index 9f61905e5e..201b510571 100644 --- a/ratis-test/src/test/java/org/apache/ratis/util/TestLifeCycle.java +++ b/ratis-test/src/test/java/org/apache/ratis/util/TestLifeCycle.java @@ -21,6 +21,11 @@ import org.junit.jupiter.api.Test; import org.junit.jupiter.api.Timeout; +import java.util.Arrays; +import java.util.EnumMap; +import java.util.List; +import java.util.Map; + import static org.apache.ratis.util.LifeCycle.State.*; import static org.junit.jupiter.api.Assertions.assertEquals; import static org.junit.jupiter.api.Assertions.assertFalse; @@ -34,7 +39,7 @@ public class TestLifeCycle { * while this test uses successors. */ @Test - @Timeout(value = 1000) + @Timeout(value = 1) public void testIsValid() { final Map> successors = new EnumMap<>(LifeCycle.State.class); From b70a235d9a3f2cf671b56fb58852581b0503db06 Mon Sep 17 00:00:00 2001 From: GewuNewOne <89496957+RkGrit@users.noreply.github.com> Date: Sat, 5 Apr 2025 21:26:18 +0800 Subject: [PATCH 386/397] RATIS-2271 Leadership Loss Causes ClosedByInterruptException and NullPointerException in LogAppender Thread (#1245) (cherry picked from commit 688bbaee7b7b388c0e1eaf49b210a6a3877e4bd4) --- .../ratis/server/raftlog/segmented/LogSegment.java | 10 ++++++++-- 1 file changed, 8 insertions(+), 2 deletions(-) diff --git a/ratis-server/src/main/java/org/apache/ratis/server/raftlog/segmented/LogSegment.java b/ratis-server/src/main/java/org/apache/ratis/server/raftlog/segmented/LogSegment.java index 1f280fba69..b50de99b69 100644 --- a/ratis-server/src/main/java/org/apache/ratis/server/raftlog/segmented/LogSegment.java +++ b/ratis-server/src/main/java/org/apache/ratis/server/raftlog/segmented/LogSegment.java @@ -293,7 +293,11 @@ ReferenceCountedObject load(TermIndex key) throws IOException { } }); loadingTimes.incrementAndGet(); - return Objects.requireNonNull(toReturn.get(), () -> "toReturn == null for " + key); + final ReferenceCountedObject proto = toReturn.get(); + if (proto == null) { + throw new RaftLogIOException("Failed to load log entry " + key); + } + return proto; } } @@ -503,8 +507,10 @@ synchronized ReferenceCountedObject loadCache(TermIndex ti) throw } try { return cacheLoader.load(ti); + } catch (RaftLogIOException e) { + throw e; } catch (Exception e) { - throw new RaftLogIOException("Failed to loadCache for log entry " + record, e); + throw new RaftLogIOException("Failed to loadCache for log entry " + ti, e); } } From 193c826096ef26eebfe71853dd5d9b636c1cdd59 Mon Sep 17 00:00:00 2001 From: GewuNewOne <89496957+RkGrit@users.noreply.github.com> Date: Fri, 11 Apr 2025 23:38:20 +0800 Subject: [PATCH 387/397] RATIS-2278. Follower Fails to Append Entries Due to Index Validation in NavigableIndices (#1247) (cherry picked from commit f5e993549330708d85570af8462ee00c879aba2c) --- .../java/org/apache/ratis/server/impl/ServerImplUtils.java | 4 +++- 1 file changed, 3 insertions(+), 1 deletion(-) diff --git a/ratis-server/src/main/java/org/apache/ratis/server/impl/ServerImplUtils.java b/ratis-server/src/main/java/org/apache/ratis/server/impl/ServerImplUtils.java index 2425e40f6e..e9514a4036 100644 --- a/ratis-server/src/main/java/org/apache/ratis/server/impl/ServerImplUtils.java +++ b/ratis-server/src/main/java/org/apache/ratis/server/impl/ServerImplUtils.java @@ -143,7 +143,9 @@ synchronized void append(List entriesTermIndices) { // validate startIndex final Map.Entry lastEntry = map.lastEntry(); if (lastEntry != null) { - Preconditions.assertSame(lastEntry.getValue().getNextIndex(), indices.startIndex, "startIndex"); + final long nextIndex = lastEntry.getValue().getNextIndex(); + Preconditions.assertTrue(indices.startIndex >= nextIndex, + () -> "startIndex = " + indices.startIndex + " < nextIndex = " + nextIndex); } map.put(indices.startIndex, indices); } From b1373cced6caa8ee00b48779ea516cdd9e97fa4b Mon Sep 17 00:00:00 2001 From: Tsz-Wo Nicholas Sze Date: Thu, 17 Apr 2025 09:03:02 -0700 Subject: [PATCH 388/397] Revert "RATIS-2278. Follower Fails to Append Entries Due to Index Validation in NavigableIndices (#1247)" This reverts commit f5e993549330708d85570af8462ee00c879aba2c. (cherry picked from commit ca81a218f31b4e686bcfdb1d5cb3eb5a8c0ce8fd) --- .../java/org/apache/ratis/server/impl/ServerImplUtils.java | 4 +--- 1 file changed, 1 insertion(+), 3 deletions(-) diff --git a/ratis-server/src/main/java/org/apache/ratis/server/impl/ServerImplUtils.java b/ratis-server/src/main/java/org/apache/ratis/server/impl/ServerImplUtils.java index e9514a4036..2425e40f6e 100644 --- a/ratis-server/src/main/java/org/apache/ratis/server/impl/ServerImplUtils.java +++ b/ratis-server/src/main/java/org/apache/ratis/server/impl/ServerImplUtils.java @@ -143,9 +143,7 @@ synchronized void append(List entriesTermIndices) { // validate startIndex final Map.Entry lastEntry = map.lastEntry(); if (lastEntry != null) { - final long nextIndex = lastEntry.getValue().getNextIndex(); - Preconditions.assertTrue(indices.startIndex >= nextIndex, - () -> "startIndex = " + indices.startIndex + " < nextIndex = " + nextIndex); + Preconditions.assertSame(lastEntry.getValue().getNextIndex(), indices.startIndex, "startIndex"); } map.put(indices.startIndex, indices); } From 092dbea953ee83430f35c7a3581918ec392feea7 Mon Sep 17 00:00:00 2001 From: GewuNewOne <89496957+RkGrit@users.noreply.github.com> Date: Sun, 20 Apr 2025 17:40:10 +0800 Subject: [PATCH 389/397] RATIS-2282. LogAppender Restart Due to Premature Log Entry Access During Concurrent Write Processing (#1249) (cherry picked from commit de58f05bbd5d2773a6719d34d288ec9ad7a57abd) --- .../server/raftlog/segmented/LogSegment.java | 55 ++++++++++--------- 1 file changed, 29 insertions(+), 26 deletions(-) diff --git a/ratis-server/src/main/java/org/apache/ratis/server/raftlog/segmented/LogSegment.java b/ratis-server/src/main/java/org/apache/ratis/server/raftlog/segmented/LogSegment.java index b50de99b69..d0b356c6fd 100644 --- a/ratis-server/src/main/java/org/apache/ratis/server/raftlog/segmented/LogSegment.java +++ b/ratis-server/src/main/java/org/apache/ratis/server/raftlog/segmented/LogSegment.java @@ -456,35 +456,41 @@ void appendToOpenSegment(LogEntryProto entry, Op op, boolean verifyEntryIndex) { append(true, entry, op, verifyEntryIndex); } - public static final String APPEND_RECORD = LogSegment.class.getSimpleName() + ".append"; - private void append(boolean keepEntryInCache, LogEntryProto entry, Op op, boolean verifyEntryIndex) { - Objects.requireNonNull(entry, "entry == null"); - final LogRecord currentLast = records.getLast(); - if (currentLast == null) { - Preconditions.assertTrue(entry.getIndex() == startIndex, - "gap between start index %s and first entry to append %s", - startIndex, entry.getIndex()); - } else { - Preconditions.assertTrue(entry.getIndex() == currentLast.getTermIndex().getIndex() + 1, - "gap between entries %s and %s", entry.getIndex(), currentLast.getTermIndex().getIndex()); - } + private void append(Op op, ReferenceCountedObject entryRef, + boolean keepEntryInCache, Consumer logConsumer) { + final LogEntryProto entry = entryRef.retain(); + try { + final LogRecord record = new LogRecord(totalFileSize, entry); + if (keepEntryInCache) { + putEntryCache(record.getTermIndex(), entryRef, op); + } + appendLogRecord(op, record); + totalFileSize += getEntrySize(entry, op); - final LogRecord record = new LogRecord(totalFileSize, entry); - records.append(record); - totalFileSize += getEntrySize(entry, op); - endIndex = entry.getIndex(); + if (logConsumer != null) { + logConsumer.accept(entry); + } + } finally { + entryRef.release(); + } } - void verifyEntryIndex(long entryIndex) { + private void appendLogRecord(Op op, LogRecord record) { + Objects.requireNonNull(record, "record == null"); final LogRecord currentLast = records.getLast(); + + final long index = record.getTermIndex().getIndex(); if (currentLast == null) { - Preconditions.assertTrue(entryIndex == startIndex, - "gap between start index %s and first entry to append %s", - startIndex, entryIndex); + Preconditions.assertTrue(index == startIndex, + "%s: gap between start index %s and the entry to append %s", op, startIndex, index); } else { - Preconditions.assertTrue(entryIndex == currentLast.getTermIndex().getIndex() + 1, - "gap between entries %s and %s", entryIndex, currentLast.getTermIndex().getIndex()); + final long currentLastIndex = currentLast.getTermIndex().getIndex(); + Preconditions.assertTrue(index == currentLastIndex + 1, + "%s: gap between last entry %s and the entry to append %s", op, currentLastIndex, index); } + + records.append(record); + endIndex = index; } ReferenceCountedObject getEntryFromCache(TermIndex ti) { @@ -515,10 +521,7 @@ synchronized ReferenceCountedObject loadCache(TermIndex ti) throw } LogRecord getLogRecord(long index) { - if (index >= startIndex && index <= endIndex) { - return records.get(index); - } - return null; + return records.get(index); } TermIndex getLastTermIndex() { From 71c93b7df125d45ab1456c3bef88f02af457611f Mon Sep 17 00:00:00 2001 From: GewuNewOne <89496957+RkGrit@users.noreply.github.com> Date: Wed, 23 Apr 2025 00:23:24 +0800 Subject: [PATCH 390/397] RATIS-2278. Follower Fails to Append Entries Due to Index Validation Race Condition in NavigableIndices (#1248) (cherry picked from commit 8ab57e1c064c6ccfec504597289590ac7ce7b106) --- .../ratis/server/impl/RaftServerImpl.java | 5 ++++- .../ratis/server/impl/ServerImplUtils.java | 19 ++++++++++++------- 2 files changed, 16 insertions(+), 8 deletions(-) diff --git a/ratis-server/src/main/java/org/apache/ratis/server/impl/RaftServerImpl.java b/ratis-server/src/main/java/org/apache/ratis/server/impl/RaftServerImpl.java index 6736f2faf9..463621067f 100644 --- a/ratis-server/src/main/java/org/apache/ratis/server/impl/RaftServerImpl.java +++ b/ratis-server/src/main/java/org/apache/ratis/server/impl/RaftServerImpl.java @@ -1687,7 +1687,10 @@ private CompletableFuture appendLog(ReferenceCountedObject entriesTermIndices; try(UncheckedAutoCloseableSupplier> entries = entriesRef.retainAndReleaseOnClose()) { entriesTermIndices = ConsecutiveIndices.convert(entries.get()); - appendLogTermIndices.append(entriesTermIndices); + if (!appendLogTermIndices.append(entriesTermIndices)) { + // index already exists, return the last future + return appendLogFuture.get(); + } } entriesRef.retain(); diff --git a/ratis-server/src/main/java/org/apache/ratis/server/impl/ServerImplUtils.java b/ratis-server/src/main/java/org/apache/ratis/server/impl/ServerImplUtils.java index 2425e40f6e..edaaff6c7a 100644 --- a/ratis-server/src/main/java/org/apache/ratis/server/impl/ServerImplUtils.java +++ b/ratis-server/src/main/java/org/apache/ratis/server/impl/ServerImplUtils.java @@ -138,15 +138,20 @@ synchronized Long getTerm(long index) { return floorEntry.getValue().getTerm(index); } - synchronized void append(List entriesTermIndices) { - for(ConsecutiveIndices indices : entriesTermIndices) { - // validate startIndex - final Map.Entry lastEntry = map.lastEntry(); - if (lastEntry != null) { - Preconditions.assertSame(lastEntry.getValue().getNextIndex(), indices.startIndex, "startIndex"); + synchronized boolean append(List entriesTermIndices) { + for(int i = 0; i < entriesTermIndices.size(); i++) { + final ConsecutiveIndices indices = entriesTermIndices.get(i); + final ConsecutiveIndices previous = map.put(indices.startIndex, indices); + if (previous != null) { + // index already exists, revert this append + map.put(previous.startIndex, previous); + for(int j = 0; j < i; j++) { + map.remove(entriesTermIndices.get(j).startIndex); + } + return false; } - map.put(indices.startIndex, indices); } + return true; } synchronized void removeExisting(List entriesTermIndices) { From 2161b16ab50962484b2d61eb5d692aa6aaa56e5c Mon Sep 17 00:00:00 2001 From: jianghuazhu <740087514@qq.com> Date: Mon, 12 May 2025 23:33:29 +0800 Subject: [PATCH 391/397] RATIS-2180. Use Objects.requireNonNull instead of Preconditions.assertNotNull (#1256) (cherry picked from commit 21f9e5b707265e9ff7c8be6c98d6704412f85eb2) --- .../org/apache/ratis/retry/ExponentialBackoffRetry.java | 5 +++-- .../apache/ratis/util/ReferenceCountedLeakDetector.java | 3 ++- .../org/apache/ratis/server/impl/RaftServerImpl.java | 1 + .../org/apache/ratis/server/impl/ServerImplUtils.java | 2 ++ .../server/raftlog/segmented/SegmentedRaftLogCache.java | 2 +- .../raftlog/segmented/TestSegmentedRaftLogCache.java | 9 ++++----- 6 files changed, 13 insertions(+), 9 deletions(-) diff --git a/ratis-common/src/main/java/org/apache/ratis/retry/ExponentialBackoffRetry.java b/ratis-common/src/main/java/org/apache/ratis/retry/ExponentialBackoffRetry.java index 3c9ffbf457..5af7b75d36 100644 --- a/ratis-common/src/main/java/org/apache/ratis/retry/ExponentialBackoffRetry.java +++ b/ratis-common/src/main/java/org/apache/ratis/retry/ExponentialBackoffRetry.java @@ -17,7 +17,6 @@ */ package org.apache.ratis.retry; -import org.apache.ratis.util.Preconditions; import org.apache.ratis.util.TimeDuration; import java.util.Objects; @@ -56,7 +55,9 @@ public Builder setMaxSleepTime(TimeDuration maxSleepTime) { } public ExponentialBackoffRetry build() { - return new ExponentialBackoffRetry(baseSleepTime, maxSleepTime, maxAttempts); + Objects.requireNonNull(baseSleepTime, "baseSleepTime == null"); + return new ExponentialBackoffRetry(baseSleepTime, maxSleepTime, + maxAttempts); } } diff --git a/ratis-common/src/main/java/org/apache/ratis/util/ReferenceCountedLeakDetector.java b/ratis-common/src/main/java/org/apache/ratis/util/ReferenceCountedLeakDetector.java index ec99eee58e..271058e549 100644 --- a/ratis-common/src/main/java/org/apache/ratis/util/ReferenceCountedLeakDetector.java +++ b/ratis-common/src/main/java/org/apache/ratis/util/ReferenceCountedLeakDetector.java @@ -22,6 +22,7 @@ import java.util.ArrayList; import java.util.List; +import java.util.Objects; import java.util.concurrent.atomic.AtomicInteger; import java.util.concurrent.atomic.AtomicReference; import java.util.function.Consumer; @@ -202,7 +203,7 @@ public synchronized boolean release() { } if (released) { - Preconditions.assertNotNull(removeMethod, () -> "Not yet retained (removeMethod == null): " + valueClass); + Objects.requireNonNull(removeMethod, () -> "Not yet retained (removeMethod == null): " + valueClass); removeMethod.run(); } return released; diff --git a/ratis-server/src/main/java/org/apache/ratis/server/impl/RaftServerImpl.java b/ratis-server/src/main/java/org/apache/ratis/server/impl/RaftServerImpl.java index 463621067f..b28961b816 100644 --- a/ratis-server/src/main/java/org/apache/ratis/server/impl/RaftServerImpl.java +++ b/ratis-server/src/main/java/org/apache/ratis/server/impl/RaftServerImpl.java @@ -1288,6 +1288,7 @@ CompletableFuture takeSnapshotAsync(SnapshotManagementRequest r LOG.info("{}: takeSnapshotAsync {}", getMemberId(), request); assertLifeCycleState(LifeCycle.States.RUNNING); assertGroup(getMemberId(), request); + Objects.requireNonNull(request.getCreate(), "create == null"); final long creationGap = request.getCreate().getCreationGap(); long minGapValue = creationGap > 0? creationGap : RaftServerConfigKeys.Snapshot.creationGap(proxy.getProperties()); diff --git a/ratis-server/src/main/java/org/apache/ratis/server/impl/ServerImplUtils.java b/ratis-server/src/main/java/org/apache/ratis/server/impl/ServerImplUtils.java index edaaff6c7a..ae6a7ee005 100644 --- a/ratis-server/src/main/java/org/apache/ratis/server/impl/ServerImplUtils.java +++ b/ratis-server/src/main/java/org/apache/ratis/server/impl/ServerImplUtils.java @@ -46,6 +46,7 @@ import java.util.Map; import java.util.NavigableMap; import java.util.TreeMap; +import java.util.Objects; import java.util.concurrent.TimeUnit; import java.util.concurrent.atomic.AtomicReference; import java.util.function.Function; @@ -173,6 +174,7 @@ public static RaftServerProxy newRaftServer( RaftServer.LOG.debug("newRaftServer: {}, {}", id, group); Objects.requireNonNull(id, "id == null"); if (group != null && !group.getPeers().isEmpty()) { + Objects.requireNonNull(id, () -> "RaftPeerId " + id + " is not in RaftGroup " + group); Objects.requireNonNull(group.getPeer(id), () -> "RaftPeerId " + id + " is not in RaftGroup " + group); } final RaftServerProxy proxy = newRaftServer(id, stateMachineRegistry, threadGroup, properties, parameters); diff --git a/ratis-server/src/main/java/org/apache/ratis/server/raftlog/segmented/SegmentedRaftLogCache.java b/ratis-server/src/main/java/org/apache/ratis/server/raftlog/segmented/SegmentedRaftLogCache.java index 5c482db41c..d9c9fde337 100644 --- a/ratis-server/src/main/java/org/apache/ratis/server/raftlog/segmented/SegmentedRaftLogCache.java +++ b/ratis-server/src/main/java/org/apache/ratis/server/raftlog/segmented/SegmentedRaftLogCache.java @@ -635,7 +635,7 @@ void appendEntry(LogEntryProto entry, LogSegment.Op op) { // SegmentedRaftLog does the segment creation/rolling work. Here we just // simply append the entry into the open segment. Objects.requireNonNull(openSegment, "openSegment == null"); - openSegment.appendToOpenSegment(entry, op, false); + openSegment.appendToOpenSegment(op, entry); } /** diff --git a/ratis-test/src/test/java/org/apache/ratis/server/raftlog/segmented/TestSegmentedRaftLogCache.java b/ratis-test/src/test/java/org/apache/ratis/server/raftlog/segmented/TestSegmentedRaftLogCache.java index 9bf6116bcf..d50b2d8a54 100644 --- a/ratis-test/src/test/java/org/apache/ratis/server/raftlog/segmented/TestSegmentedRaftLogCache.java +++ b/ratis-test/src/test/java/org/apache/ratis/server/raftlog/segmented/TestSegmentedRaftLogCache.java @@ -64,7 +64,7 @@ private LogSegment prepareLogSegment(long start, long end, boolean isOpen) { for (long i = start; i <= end; i++) { SimpleOperation m = new SimpleOperation("m" + i); LogEntryProto entry = LogProtoUtils.toLogEntryProto(m.getLogEntryContent(), 0, i); - s.appendToOpenSegment(entry, LogSegment.Op.WRITE_CACHE_WITHOUT_STATE_MACHINE_CACHE, true); + s.appendToOpenSegment(LogSegment.Op.WRITE_CACHE_WITHOUT_STATE_MACHINE_CACHE, ReferenceCountedObject.wrap(entry)); } if (!isOpen) { s.close(); @@ -155,17 +155,16 @@ public void testAppendEntry() throws Exception { final SimpleOperation m = new SimpleOperation("m"); try { LogEntryProto entry = LogProtoUtils.toLogEntryProto(m.getLogEntryContent(), 0, 0); - cache.appendEntry(Op.WRITE_CACHE_WITHOUT_STATE_MACHINE_CACHE, ReferenceCountedObject.wrap(entry) - ); + cache.appendEntry(LogSegment.Op.WRITE_CACHE_WITHOUT_STATE_MACHINE_CACHE, ReferenceCountedObject.wrap(entry)); Assertions.fail("the open segment is null"); - } catch (IllegalStateException ignored) { + } catch (IllegalStateException | NullPointerException ignored) { } LogSegment openSegment = prepareLogSegment(100, 100, true); cache.addSegment(openSegment); for (long index = 101; index < 200; index++) { LogEntryProto entry = LogProtoUtils.toLogEntryProto(m.getLogEntryContent(), 0, index); - cache.appendEntry(entry, LogSegment.Op.WRITE_CACHE_WITHOUT_STATE_MACHINE_CACHE); + cache.appendEntry(LogSegment.Op.WRITE_CACHE_WITHOUT_STATE_MACHINE_CACHE, ReferenceCountedObject.wrap(entry)); } Assertions.assertNotNull(cache.getOpenSegment()); From 300bb2f6c78ef4dc1cceb214139037e3ff6d8c57 Mon Sep 17 00:00:00 2001 From: slfan1989 <55643692+slfan1989@users.noreply.github.com> Date: Mon, 12 May 2025 23:50:32 +0800 Subject: [PATCH 392/397] RATIS-2291. Fix failing TestInstallSnapshotNotificationWithGrpc#testAddNewFollowersNoSnapshot. (#1257) (cherry picked from commit b0f5330aeae83c41e19be6759fe56240f6f6c105) --- .../org/apache/ratis/InstallSnapshotNotificationTests.java | 4 ++-- .../java/org/apache/ratis/server/impl/MiniRaftCluster.java | 2 +- 2 files changed, 3 insertions(+), 3 deletions(-) diff --git a/ratis-server/src/test/java/org/apache/ratis/InstallSnapshotNotificationTests.java b/ratis-server/src/test/java/org/apache/ratis/InstallSnapshotNotificationTests.java index f4fae7bb06..699a73e807 100644 --- a/ratis-server/src/test/java/org/apache/ratis/InstallSnapshotNotificationTests.java +++ b/ratis-server/src/test/java/org/apache/ratis/InstallSnapshotNotificationTests.java @@ -221,7 +221,7 @@ private void testAddNewFollowers(CLUSTER cluster, int numRequests) throws Except // delete the log segments from the leader LOG.info("Delete logs {}", logs); for (LogSegmentPath path : logs) { - FileUtils.deleteFully(path.getPath()); // the log may be already puged + FileUtils.deleteFully(path.getPath()); // the log may be already purged } // restart the peer @@ -251,7 +251,7 @@ private void testAddNewFollowers(CLUSTER cluster, int numRequests) throws Except // leader snapshot. for (RaftServer.Division follower : cluster.getFollowers()) { final long expected = leaderSnapshotInfo.getIndex(); - Assertions.assertEquals(expected, RaftServerTestUtil.getLatestInstalledSnapshotIndex(follower)); + Assert.assertEquals(expected, RaftServerTestUtil.getLatestInstalledSnapshotIndex(follower)); RaftSnapshotBaseTest.assertLogContent(follower, false); } diff --git a/ratis-server/src/test/java/org/apache/ratis/server/impl/MiniRaftCluster.java b/ratis-server/src/test/java/org/apache/ratis/server/impl/MiniRaftCluster.java index f673098376..df78069952 100644 --- a/ratis-server/src/test/java/org/apache/ratis/server/impl/MiniRaftCluster.java +++ b/ratis-server/src/test/java/org/apache/ratis/server/impl/MiniRaftCluster.java @@ -146,7 +146,7 @@ default void runWithNewCluster(int numServers, int numListeners, boolean startCl testCase.accept(cluster); } catch(Throwable t) { LOG.info(cluster.printServers()); - LOG.error("Failed " + caller, t); + LOG.error("Failed {}", caller, t); failed = t; throw t; } finally { From 6d7a5b9591c6888594bf183c4ea0c59bb1173d5b Mon Sep 17 00:00:00 2001 From: slfan1989 <55643692+slfan1989@users.noreply.github.com> Date: Wed, 21 May 2025 00:09:01 +0800 Subject: [PATCH 393/397] RATIS-2251. Migrate ratis-test tests to Junit 5 - Part 3. (#1227) (cherry picked from commit 0557974fa2d7409e9f5089a780a4b6024c53ec99) --- .../InstallSnapshotNotificationTests.java | 18 +- .../org/apache/ratis/LogAppenderTests.java | 90 -------- .../apache/ratis/OutputStreamBaseTest.java | 5 +- .../apache/ratis/RaftExceptionBaseTest.java | 6 +- .../java/org/apache/ratis/RaftTestUtil.java | 19 +- .../apache/ratis/ReadOnlyRequestTests.java | 193 ++++++++++++++++-- .../impl/RaftReconfigurationBaseTest.java | 11 +- .../impl/RaftStateMachineExceptionTests.java | 27 --- .../statemachine/RaftSnapshotBaseTest.java | 2 +- ratis-test/pom.xml | 10 + 10 files changed, 210 insertions(+), 171 deletions(-) diff --git a/ratis-server/src/test/java/org/apache/ratis/InstallSnapshotNotificationTests.java b/ratis-server/src/test/java/org/apache/ratis/InstallSnapshotNotificationTests.java index 699a73e807..a8558287ff 100644 --- a/ratis-server/src/test/java/org/apache/ratis/InstallSnapshotNotificationTests.java +++ b/ratis-server/src/test/java/org/apache/ratis/InstallSnapshotNotificationTests.java @@ -221,7 +221,7 @@ private void testAddNewFollowers(CLUSTER cluster, int numRequests) throws Except // delete the log segments from the leader LOG.info("Delete logs {}", logs); for (LogSegmentPath path : logs) { - FileUtils.deleteFully(path.getPath()); // the log may be already purged + FileUtils.deleteFully(path.getPath()); // the log may be already puged } // restart the peer @@ -238,7 +238,7 @@ private void testAddNewFollowers(CLUSTER cluster, int numRequests) throws Except final SnapshotInfo leaderSnapshotInfo = cluster.getLeader().getStateMachine().getLatestSnapshot(); LOG.info("LeaderSnapshotInfo: {}", leaderSnapshotInfo.getTermIndex()); final boolean set = LEADER_SNAPSHOT_INFO_REF.compareAndSet(null, leaderSnapshotInfo); - Assert.assertTrue(set); + Assertions.assertTrue(set); // Add new peer(s) final PeerChanges change = cluster.addNewPeers(1, true); @@ -251,7 +251,7 @@ private void testAddNewFollowers(CLUSTER cluster, int numRequests) throws Except // leader snapshot. for (RaftServer.Division follower : cluster.getFollowers()) { final long expected = leaderSnapshotInfo.getIndex(); - Assert.assertEquals(expected, RaftServerTestUtil.getLatestInstalledSnapshotIndex(follower)); + Assertions.assertEquals(expected, RaftServerTestUtil.getLatestInstalledSnapshotIndex(follower)); RaftSnapshotBaseTest.assertLogContent(follower, false); } @@ -349,9 +349,9 @@ private void testInstallSnapshotNotificationCount(CLUSTER cluster) throws Except long snapshotIndex = cluster.getLeader().getStateMachine().takeSnapshot(); Assertions.assertEquals(20, snapshotIndex); final SnapshotInfo leaderSnapshotInfo = cluster.getLeader().getStateMachine().getLatestSnapshot(); - Assert.assertEquals(20, leaderSnapshotInfo.getIndex()); + Assertions.assertEquals(20, leaderSnapshotInfo.getIndex()); final boolean set = LEADER_SNAPSHOT_INFO_REF.compareAndSet(null, leaderSnapshotInfo); - Assert.assertTrue(set); + Assertions.assertTrue(set); // Wait for the snapshot to be done. final RaftServer.Division leader = cluster.getLeader(); @@ -408,7 +408,7 @@ private void testInstallSnapshotNotificationCount(CLUSTER cluster) throws Except } // Make sure each new peer got one snapshot notification. - Assertions.assertEquals(numNewPeers, numSnapshotRequests.get()); + Assertions.assertEquals(2, numSnapshotRequests.get()); } finally { cluster.shutdown(); @@ -472,7 +472,7 @@ private void testInstallSnapshotInstalledEvent(CLUSTER cluster) throws Exception final SnapshotInfo leaderSnapshotInfo = cluster.getLeader().getStateMachine().getLatestSnapshot(); LOG.info("LeaderSnapshotInfo: {}", leaderSnapshotInfo.getTermIndex()); final boolean set = LEADER_SNAPSHOT_INFO_REF.compareAndSet(null, leaderSnapshotInfo); - Assert.assertTrue(set); + Assertions.assertTrue(set); // add one new peer final PeerChanges change = cluster.addNewPeers(1, true); @@ -547,7 +547,7 @@ private void testInstallSnapshotDuringBootstrap(CLUSTER cluster) throws Exceptio final SnapshotInfo leaderSnapshotInfo = cluster.getLeader().getStateMachine().getLatestSnapshot(); final boolean set = LEADER_SNAPSHOT_INFO_REF.compareAndSet(null, leaderSnapshotInfo); - Assert.assertTrue(set); + Assertions.assertTrue(set); // Add new peer(s) final int numNewPeers = 1; @@ -565,7 +565,7 @@ private void testInstallSnapshotDuringBootstrap(CLUSTER cluster) throws Exceptio } // Make sure each new peer got at least one snapshot notification. - Assertions.assertTrue(numNewPeers <= numSnapshotRequests.get()); + Assertions.assertTrue(2 <= numSnapshotRequests.get()); } finally { cluster.shutdown(); } diff --git a/ratis-server/src/test/java/org/apache/ratis/LogAppenderTests.java b/ratis-server/src/test/java/org/apache/ratis/LogAppenderTests.java index dc9cd8c0db..e5d0ee0ef8 100644 --- a/ratis-server/src/test/java/org/apache/ratis/LogAppenderTests.java +++ b/ratis-server/src/test/java/org/apache/ratis/LogAppenderTests.java @@ -226,94 +226,4 @@ void runTest(CLUSTER cluster) throws Exception { Assertions.assertNotNull(last); Assertions.assertTrue(last.getIndex() <= leader.getInfo().getLastAppliedIndex()); } - - @Test - public void testNewAppendEntriesRequestAfterPurgeFollowerBehindStartIndex() throws Exception { - final RaftProperties prop = getProperties(); - RaftServerConfigKeys.Log.setPurgeGap(prop, 1); - RaftServerConfigKeys.Log.setSegmentSizeMax(prop, SizeInBytes.valueOf("1KB")); - runWithNewCluster(3, cluster -> { - final long startIndexAfterPurge = setupPurgedLeaderLog(cluster); - // Test when followerNextIndex < leader's logStartIndex - runTestNewAppendEntriesRequestAfterPurge(cluster, startIndexAfterPurge - 1); - }); - } - - @Test - public void testNewAppendEntriesRequestAfterPurgeFollowerAtStartIndex() throws Exception { - final RaftProperties prop = getProperties(); - RaftServerConfigKeys.Log.setPurgeGap(prop, 1); - RaftServerConfigKeys.Log.setSegmentSizeMax(prop, SizeInBytes.valueOf("1KB")); - runWithNewCluster(3, cluster -> { - final long startIndexAfterPurge = setupPurgedLeaderLog(cluster); - // Test when followerNextIndex == leader's logStartIndex, but the previous index is already purged - runTestNewAppendEntriesRequestAfterPurge(cluster, startIndexAfterPurge); - }); - } - - private long setupPurgedLeaderLog(CLUSTER cluster) throws Exception { - final RaftServer.Division leader = waitForLeader(cluster); - final RaftLog leaderLog = leader.getRaftLog(); - - try (RaftClient client = cluster.createClient(leader.getId())) { - for (SimpleMessage msg : generateMsgs(5)) { - client.io().send(msg); - } - } - - final long lastLogIndex = leaderLog.getLastEntryTermIndex().getIndex(); - LOG.info("Leader log lastIndex={}, startIndex={}", lastLogIndex, leaderLog.getStartIndex()); - Assertions.assertTrue(lastLogIndex > 5, "Need enough log entries for the test"); - - // Take a snapshot so that shouldInstallSnapshot() can return it - final long snapshotIndex = SimpleStateMachine4Testing.get(leader).takeSnapshot(); - LOG.info("Snapshot taken at index {}", snapshotIndex); - Assertions.assertTrue(snapshotIndex > 0, "Snapshot should have been taken"); - - final long purgeUpTo = lastLogIndex - 2; - LOG.info("Purging leader log up to index {}", purgeUpTo); - leaderLog.purge(purgeUpTo).get(); - - final long startIndexAfterPurge = leaderLog.getStartIndex(); - LOG.info("Leader log after purge: startIndex={}", startIndexAfterPurge); - Assertions.assertTrue(startIndexAfterPurge > 1, - "Purge should have advanced startIndex, but got " + startIndexAfterPurge); - - return startIndexAfterPurge; - } - - void runTestNewAppendEntriesRequestAfterPurge(CLUSTER cluster, - long targetNextIndex) throws Exception { - final RaftServer.Division leader = waitForLeader(cluster); - final RaftLog leaderLog = leader.getRaftLog(); - final long startIndexAfterPurge = leaderLog.getStartIndex(); - - final Stream appenders = RaftServerTestUtil.getLogAppenders(leader); - Assertions.assertNotNull(appenders, "Leader should have log appenders"); - final LogAppender appender = appenders.findFirst().orElseThrow( - () -> new AssertionError("No log appender found")); - - Assertions.assertTrue(targetNextIndex > RaftLog.LEAST_VALID_LOG_INDEX, - "targetNextIndex should be > LEAST_VALID_LOG_INDEX"); - appender.getFollower().setNextIndex(targetNextIndex); - - LOG.info("Set follower nextIndex={}, startIndexAfterPurge={}, snapshotIndex={}", - targetNextIndex, startIndexAfterPurge, appender.getFollower().getSnapshotIndex()); - Assertions.assertEquals(0, appender.getFollower().getSnapshotIndex(), - "Follower snapshotIndex should be 0 (default, never installed snapshot)"); - - Assertions.assertNull(leaderLog.getTermIndex(targetNextIndex - 1), - "Entry at previousIndex=" + (targetNextIndex - 1) + " should have been purged"); - - // Should return null instead of throwing NPE - Assertions.assertNull(appender.newAppendEntriesRequest(0, false), - "newAppendEntriesRequest should return null when previous TermIndex is not found"); - - Assertions.assertEquals(targetNextIndex, appender.getFollower().getNextIndex(), - "Follower nextIndex should remain unchanged"); - - Assertions.assertNotNull(appender.shouldInstallSnapshot(), - "shouldInstallSnapshot should return non-null when followerNextIndex (" - + targetNextIndex + ") and previous entry has been purged"); - } } diff --git a/ratis-server/src/test/java/org/apache/ratis/OutputStreamBaseTest.java b/ratis-server/src/test/java/org/apache/ratis/OutputStreamBaseTest.java index 874c953553..9ce54b6f54 100644 --- a/ratis-server/src/test/java/org/apache/ratis/OutputStreamBaseTest.java +++ b/ratis-server/src/test/java/org/apache/ratis/OutputStreamBaseTest.java @@ -26,8 +26,8 @@ import org.apache.ratis.server.raftlog.RaftLog; import org.apache.ratis.util.SizeInBytes; import org.apache.ratis.util.StringUtils; -import org.junit.Assert; -import org.junit.Test; +import org.junit.jupiter.api.Assertions; +import org.junit.jupiter.api.Test; import java.io.IOException; import java.io.OutputStream; @@ -303,7 +303,6 @@ private void runTestKillLeader(CLUSTER cluster) throws Exception { final boolean latchCompleted = latch.await(5, TimeUnit.SECONDS); Assertions.assertTrue(latchCompleted, "Writer thread did not finish within the timeout"); LOG.info("Writer success? " + success.get()); - Assertions.assertNotNull(success.get(), "Writer thread completed but success was not set"); Assertions.assertTrue(success.get()); // total number of tx should be >= result + 2, where 2 means two NoOp from // leaders. It may be larger than result+2 because the client may resend diff --git a/ratis-server/src/test/java/org/apache/ratis/RaftExceptionBaseTest.java b/ratis-server/src/test/java/org/apache/ratis/RaftExceptionBaseTest.java index 65e8063251..cfdda48fbb 100644 --- a/ratis-server/src/test/java/org/apache/ratis/RaftExceptionBaseTest.java +++ b/ratis-server/src/test/java/org/apache/ratis/RaftExceptionBaseTest.java @@ -35,7 +35,7 @@ import org.apache.ratis.util.Slf4jUtils; import org.apache.ratis.util.SizeInBytes; import org.junit.jupiter.api.Assertions; -import org.junit.jupiter.api.Assumptions; +import org.junit.Assume; import org.junit.jupiter.api.Test; import org.slf4j.event.Level; @@ -85,7 +85,7 @@ RaftClientReply assertNotLeaderException(RaftPeerId expectedSuggestedLeader, final SimpleMessage message = new SimpleMessage(messageId); final RaftClientReply reply = rpc.sendRequest(cluster.newRaftClientRequest(ClientId.randomId(), server, message)); Assertions.assertNotNull(reply); - Assumptions.assumeFalse(reply.isSuccess()); + Assume.assumeFalse(reply.isSuccess()); final NotLeaderException nle = reply.getNotLeaderException(); Objects.requireNonNull(nle); Assertions.assertEquals(expectedSuggestedLeader, nle.getSuggestedLeader().getId()); @@ -114,7 +114,7 @@ void runTestNotLeaderExceptionWithReconf(CLUSTER cluster) throws Exception { // trigger setConfiguration LOG.info("Start changing the configuration: {}", change.getPeersInNewConf()); try (final RaftClient c2 = cluster.createClient(newLeader)) { - RaftClientReply reply = c2.admin().setConfiguration(change.getPeersInNewConf()); + RaftClientReply reply = c2.admin().setConfiguration(change.allPeersInNewConf); Assertions.assertTrue(reply.isSuccess()); } LOG.info(cluster.printServers()); diff --git a/ratis-server/src/test/java/org/apache/ratis/RaftTestUtil.java b/ratis-server/src/test/java/org/apache/ratis/RaftTestUtil.java index c94d16b039..2ab208bad5 100644 --- a/ratis-server/src/test/java/org/apache/ratis/RaftTestUtil.java +++ b/ratis-server/src/test/java/org/apache/ratis/RaftTestUtil.java @@ -44,8 +44,7 @@ import org.apache.ratis.util.Preconditions; import org.apache.ratis.util.ProtoUtils; import org.apache.ratis.util.TimeDuration; -import org.apache.ratis.util.function.CheckedConsumer; -import org.junit.jupiter.api.Assumptions; +import org.junit.AssumptionViolatedException; import org.junit.jupiter.api.Assertions; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -566,7 +565,7 @@ static void assertSameLog(RaftLog expected, RaftLog computed) throws Exception { final long lastIndex = expected.getNextIndex() - 1; Assertions.assertEquals(expected.getLastEntryTermIndex().getIndex(), lastIndex); for(long i = 0; i < lastIndex; i++) { - Assert.assertEquals("Checking " + TermIndex.valueOf(expected.get(i)), expected.get(i), computed.get(i)); + Assertions.assertEquals(expected.get(i), computed.get(i), "Checking " + TermIndex.valueOf(expected.get(i))); } } @@ -614,18 +613,4 @@ static void gc() throws InterruptedException { Thread.sleep(100); } } - - static void gc() throws InterruptedException { - // use WeakReference to detect gc - Object obj = new Object(); - final WeakReference weakRef = new WeakReference<>(obj); - obj = null; - - // loop until gc has completed. - for (int i = 0; weakRef.get() != null; i++) { - LOG.info("gc {}", i); - System.gc(); - Thread.sleep(100); - } - } } diff --git a/ratis-server/src/test/java/org/apache/ratis/ReadOnlyRequestTests.java b/ratis-server/src/test/java/org/apache/ratis/ReadOnlyRequestTests.java index a2060da9ec..573434a1fb 100644 --- a/ratis-server/src/test/java/org/apache/ratis/ReadOnlyRequestTests.java +++ b/ratis-server/src/test/java/org/apache/ratis/ReadOnlyRequestTests.java @@ -35,6 +35,7 @@ import org.apache.ratis.statemachine.TransactionContext; import org.apache.ratis.statemachine.impl.BaseStateMachine; import org.apache.ratis.util.Slf4jUtils; +import org.apache.ratis.util.TimeDuration; import org.junit.jupiter.api.Assertions; import org.junit.jupiter.api.BeforeEach; import org.junit.jupiter.api.Test; @@ -58,10 +59,6 @@ public abstract class ReadOnlyRequestTests static final String WAIT_AND_INCREMENT_STRING = "WAIT_AND_INCREMENT"; static final String QUERY_STRING = "QUERY"; - public static final Message INCREMENT = new RaftTestUtil.SimpleMessage(INCREMENT_STRING); - public static final Message WAIT_AND_INCREMENT = new RaftTestUtil.SimpleMessage(WAIT_AND_INCREMENT_STRING); - public static final Message QUERY = new RaftTestUtil.SimpleMessage(QUERY_STRING); - @BeforeEach public void setup() { final RaftProperties p = getProperties(); @@ -86,8 +83,10 @@ static void runTestReadOnly(C cluster) throws Except try (final RaftClient client = cluster.createClient(leaderId)) { for (int i = 1; i <= 10; i++) { - assertReplyExact(i, client.io().send(INCREMENT)); - assertReplyExact(i, client.io().sendReadOnly(QUERY)); + RaftClientReply reply = client.io().send(incrementMessage); + Assertions.assertTrue(reply.isSuccess()); + reply = client.io().sendReadOnly(queryMessage); + Assertions.assertEquals(i, retrieve(reply)); } } } finally { @@ -109,12 +108,18 @@ static void runTestReadTimeout(Class { - final RaftClientReply timeoutReply = noRetry.io().sendReadOnly(QUERY); - Assertions.assertFalse(timeoutReply.isSuccess()); - Assertions.assertNotNull(timeoutReply.getException()); - Assertions.assertInstanceOf(ReadException.class, timeoutReply.getException()); - }); + CompletableFuture result = client.async().send(incrementMessage); + client.admin().transferLeadership(null, 200); + + Assertions.assertThrows(ReadIndexException.class, () -> { + RaftClientReply timeoutReply = noRetry.io().sendReadOnly(queryMessage); + Assertions.assertNotNull(timeoutReply.getException()); + Assertions.assertTrue(timeoutReply.getException() instanceof ReadException); + }); + } + + } finally { + cluster.shutdown(); } } @@ -127,15 +132,171 @@ static void runTestReadOnlyRetryWhenLeaderDown(Retry throws Exception { final RaftPeerId leaderId = RaftTestUtil.waitForLeader(cluster).getId(); - try (RaftClient client = cluster.createClient(leaderId, retryPolicy)) { - assertReplyExact(1, client.io().send(INCREMENT)); - assertReplyExact(1, client.io().sendReadOnly(QUERY)); + private void testFollowerReadOnlyImpl(CLUSTER cluster) throws Exception { + try { + RaftTestUtil.waitForLeader(cluster); + + List followers = cluster.getFollowers(); + Assertions.assertEquals(2, followers.size()); + + final RaftPeerId f0 = followers.get(0).getId(); + final RaftPeerId f1 = followers.get(1).getId(); + try (RaftClient client = cluster.createClient(cluster.getLeader().getId())) { + for (int i = 1; i <= 10; i++) { + final RaftClientReply reply = client.io().send(incrementMessage); + Assertions.assertTrue(reply.isSuccess()); + final RaftClientReply read1 = client.io().sendReadOnly(queryMessage, f0); + Assertions.assertEquals(i, retrieve(read1)); + final CompletableFuture read2 = client.async().sendReadOnly(queryMessage, f1); + Assertions.assertEquals(i, retrieve(read2.get(1, TimeUnit.SECONDS))); + } + } + } finally { + cluster.shutdown(); + } + } + + @Test + public void testFollowerLinearizableReadParallel() throws Exception { + getProperties().setEnum(RaftServerConfigKeys.Read.OPTION_KEY, RaftServerConfigKeys.Read.Option.LINEARIZABLE); + runWithNewCluster(NUM_SERVERS, this::testFollowerReadOnlyParallelImpl); + } + + @Test + public void testFollowerLeaseReadParallel() throws Exception { + getProperties().setBoolean(RaftServerConfigKeys.Read.LEADER_LEASE_ENABLED_KEY, true); + runWithNewCluster(NUM_SERVERS, this::testFollowerReadOnlyParallelImpl); + } + + private void testFollowerReadOnlyParallelImpl(CLUSTER cluster) throws Exception { + try { + RaftTestUtil.waitForLeader(cluster); + + List followers = cluster.getFollowers(); + Assertions.assertEquals(2, followers.size()); + + try (RaftClient leaderClient = cluster.createClient(cluster.getLeader().getId()); + RaftClient followerClient1 = cluster.createClient(followers.get(0).getId())) { + + leaderClient.io().send(incrementMessage); + leaderClient.async().send(waitAndIncrementMessage); + Thread.sleep(100); + + RaftClientReply clientReply = followerClient1.io().sendReadOnly(queryMessage, followers.get(0).getId()); + Assertions.assertEquals(2, retrieve(clientReply)); + } + + } finally { + cluster.shutdown(); + } + } + + @Test + public void testFollowerLinearizableReadFailWhenLeaderDown() throws Exception { + getProperties().setEnum(RaftServerConfigKeys.Read.OPTION_KEY, RaftServerConfigKeys.Read.Option.LINEARIZABLE); + runWithNewCluster(NUM_SERVERS, this::testFollowerReadOnlyFailWhenLeaderDownImpl); + } + + @Test + public void testFollowerLeaseReadWhenLeaderDown() throws Exception { + getProperties().setBoolean(RaftServerConfigKeys.Read.LEADER_LEASE_ENABLED_KEY, true); + runWithNewCluster(NUM_SERVERS, this::testFollowerReadOnlyFailWhenLeaderDownImpl); + } + + private void testFollowerReadOnlyFailWhenLeaderDownImpl(CLUSTER cluster) throws Exception { + try { + RaftTestUtil.waitForLeader(cluster); + + List followers = cluster.getFollowers(); + Assertions.assertEquals(2, followers.size()); + + try (RaftClient leaderClient = cluster.createClient(cluster.getLeader().getId()); + RaftClient followerClient1 = cluster.createClient(followers.get(0).getId(), RetryPolicies.noRetry())) { + leaderClient.io().send(incrementMessage); + + RaftClientReply clientReply = followerClient1.io().sendReadOnly(queryMessage); + Assertions.assertEquals(1, retrieve(clientReply)); + + // kill the leader + // read timeout quicker than election timeout + leaderClient.admin().transferLeadership(null, 200); + + Assertions.assertThrows(ReadIndexException.class, () -> { + followerClient1.io().sendReadOnly(queryMessage, followers.get(0).getId()); + }); + } + + } finally { + cluster.shutdown(); + } + } + + @Test + public void testFollowerReadOnlyRetryWhenLeaderDown() throws Exception { + getProperties().setEnum(RaftServerConfigKeys.Read.OPTION_KEY, RaftServerConfigKeys.Read.Option.LINEARIZABLE); + runWithNewCluster(NUM_SERVERS, this::testFollowerReadOnlyRetryWhenLeaderDown); + } + + @Test + public void testFollowerLeaseReadRetryWhenLeaderDown() throws Exception { + getProperties().setBoolean(RaftServerConfigKeys.Read.LEADER_LEASE_ENABLED_KEY, true); + runWithNewCluster(NUM_SERVERS, this::testFollowerReadOnlyRetryWhenLeaderDown); + } + + private void testFollowerReadOnlyRetryWhenLeaderDown(CLUSTER cluster) throws Exception { + // only retry on readIndexException + final RetryPolicy retryPolicy = ExceptionDependentRetry + .newBuilder() + .setDefaultPolicy(RetryPolicies.noRetry()) + .setExceptionToPolicy(ReadIndexException.class, + RetryPolicies.retryForeverWithSleep(TimeDuration.valueOf(500, TimeUnit.MILLISECONDS))) + .build(); + + RaftTestUtil.waitForLeader(cluster); + + try (RaftClient client = cluster.createClient(cluster.getLeader().getId(), retryPolicy)) { + client.io().send(incrementMessage); + + final RaftClientReply clientReply = client.io().sendReadOnly(queryMessage); + Assertions.assertEquals(1, retrieve(clientReply)); // kill the leader client.admin().transferLeadership(null, 200); // readOnly will success after re-election - assertReplyExact(1, client.io().sendReadOnly(QUERY)); + final RaftClientReply replySuccess = client.io().sendReadOnly(queryMessage); + Assertions.assertEquals(1, retrieve(clientReply)); + } + } + + @Test + public void testReadAfterWrite() throws Exception { + runWithNewCluster(NUM_SERVERS, this::testReadAfterWriteImpl); + } + + private void testReadAfterWriteImpl(CLUSTER cluster) throws Exception { + RaftTestUtil.waitForLeader(cluster); + try (RaftClient client = cluster.createClient()) { + // test blocking read-after-write + client.io().send(incrementMessage); + final RaftClientReply blockReply = client.io().sendReadAfterWrite(queryMessage); + Assertions.assertEquals(1, retrieve(blockReply)); + + // test asynchronous read-after-write + client.async().send(incrementMessage); + client.async().sendReadAfterWrite(queryMessage).thenAccept(reply -> { + Assertions.assertEquals(2, retrieve(reply)); + }); + + for (int i = 0; i < 20; i++) { + client.async().send(incrementMessage); + } + final CompletableFuture linearizable = client.async().sendReadOnly(queryMessage); + final CompletableFuture readAfterWrite = client.async().sendReadAfterWrite(queryMessage); + + CompletableFuture.allOf(linearizable, readAfterWrite).get(); + // read-after-write is more consistent than linearizable read + Assertions.assertTrue(retrieve(readAfterWrite.get()) >= retrieve(linearizable.get())); } } diff --git a/ratis-server/src/test/java/org/apache/ratis/server/impl/RaftReconfigurationBaseTest.java b/ratis-server/src/test/java/org/apache/ratis/server/impl/RaftReconfigurationBaseTest.java index 7a2f1a24ee..c06b1565aa 100644 --- a/ratis-server/src/test/java/org/apache/ratis/server/impl/RaftReconfigurationBaseTest.java +++ b/ratis-server/src/test/java/org/apache/ratis/server/impl/RaftReconfigurationBaseTest.java @@ -269,7 +269,7 @@ private void runTestSetConfigurationInAddMode(CLUSTER cluster) throws Exception .setServersInNewConf(peers) .setMode(SetConfigurationRequest.Mode.ADD).build()); Assertions.assertTrue(reply.isSuccess()); - waitAndCheckNewConf(cluster, change.getPeersInNewConf(), 0, null); + waitAndCheckNewConf(cluster, change.allPeersInNewConf, 0, null); } cluster.close(); } @@ -307,7 +307,7 @@ private void runTestSetConfigurationInCasMode(CLUSTER cluster) throws Exception .setMode(SetConfigurationRequest.Mode.COMPARE_AND_SET) .build()); Assertions.assertTrue(reply.isSuccess()); - waitAndCheckNewConf(cluster, change.getPeersInNewConf(), 0, null); + waitAndCheckNewConf(cluster, change.allPeersInNewConf, 0, null); } cluster.close(); } @@ -387,7 +387,8 @@ void runTestReconfTimeout(CLUSTER cluster) throws Exception { try (final RaftClient client = cluster.createClient(leaderId)) { PeerChanges c1 = cluster.addNewPeers(2, false); - LOG.info("Start changing the configuration: {}", c1.getPeersInNewConf()); + LOG.info("Start changing the configuration: {}", + asList(c1.allPeersInNewConf)); Assertions.assertFalse(((RaftConfigurationImpl)cluster.getLeader().getRaftConf()).isTransitional()); final RaftClientRpc sender = client.getClientRpc(); @@ -419,7 +420,7 @@ void runTestReconfTimeout(CLUSTER cluster) throws Exception { for (RaftPeer np : c1.getAddedPeers()) { cluster.restartServer(np.getId(), false); } - Assertions.assertTrue(client.admin().setConfiguration(c1.getPeersInNewConf()).isSuccess()); + Assertions.assertTrue(client.admin().setConfiguration(c1.allPeersInNewConf).isSuccess()); } } @@ -439,7 +440,7 @@ public void testBootstrapReconfWithSingleNodeAddTwo() throws Exception { try (final RaftClient client = cluster.createClient(leaderId)) { final PeerChanges c1 = cluster.addNewPeers(2, true); - assertThrows(SetConfigurationException.class, () -> client.admin().setConfiguration(c1.getPeersInNewConf()), + assertThrows(SetConfigurationException.class, () -> client.admin().setConfiguration(c1.allPeersInNewConf), "Expect change majority error."); } }); diff --git a/ratis-server/src/test/java/org/apache/ratis/server/impl/RaftStateMachineExceptionTests.java b/ratis-server/src/test/java/org/apache/ratis/server/impl/RaftStateMachineExceptionTests.java index 1e46907d10..a1aed3ff46 100644 --- a/ratis-server/src/test/java/org/apache/ratis/server/impl/RaftStateMachineExceptionTests.java +++ b/ratis-server/src/test/java/org/apache/ratis/server/impl/RaftStateMachineExceptionTests.java @@ -187,31 +187,4 @@ private void runTestRetryOnExceptionDuringReplication(CLUSTER cluster) throws Ex failPreAppend = false; } } - - @Test - public void testNoCancelTransactionOnPreAppendFailure() throws Exception { - runWithNewCluster(3, this::runTestNoCancelTransactionOnPreAppendFailure); - } - - private void runTestNoCancelTransactionOnPreAppendFailure(CLUSTER cluster) throws Exception { - final RaftPeerId leaderId = RaftTestUtil.waitForLeader(cluster).getId(); - failPreAppend = true; - numCancelTransaction.set(0); - try (final RaftClient client = cluster.createClient(leaderId)) { - try { - client.io().send(new SimpleMessage("cancel-transaction")); - fail("Exception expected"); - } catch (StateMachineException e) { - Assertions.assertTrue(e.getCause().getMessage().contains("Fake Exception in preAppend")); - } - - JavaUtils.attemptRepeatedly(() -> { - Assertions.assertEquals(0, numCancelTransaction.get(), - () -> "Expected cancelTransaction() not to be called but got " + numCancelTransaction.get()); - return null; - }, 10, ONE_SECOND, "wait for cancelTransaction", LOG); - } finally { - failPreAppend = false; - } - } } diff --git a/ratis-server/src/test/java/org/apache/ratis/statemachine/RaftSnapshotBaseTest.java b/ratis-server/src/test/java/org/apache/ratis/statemachine/RaftSnapshotBaseTest.java index c87116e762..f1106f2cf9 100644 --- a/ratis-server/src/test/java/org/apache/ratis/statemachine/RaftSnapshotBaseTest.java +++ b/ratis-server/src/test/java/org/apache/ratis/statemachine/RaftSnapshotBaseTest.java @@ -51,8 +51,8 @@ import org.apache.ratis.util.JavaUtils; import org.apache.ratis.util.LifeCycle; import org.apache.ratis.util.Slf4jUtils; -import org.junit.jupiter.api.Assertions; import org.junit.jupiter.api.AfterEach; +import org.junit.jupiter.api.Assertions; import org.junit.jupiter.api.BeforeEach; import org.junit.jupiter.api.Test; import org.slf4j.Logger; diff --git a/ratis-test/pom.xml b/ratis-test/pom.xml index ab34105c8b..d8fcb08932 100644 --- a/ratis-test/pom.xml +++ b/ratis-test/pom.xml @@ -151,6 +151,16 @@ junit-jupiter-params test + + org.junit.vintage + junit-vintage-engine + test + + + org.junit.platform + junit-platform-launcher + test + org.mockito mockito-core From c2d220711566af6cb80ee9e1956800a07b93ff1c Mon Sep 17 00:00:00 2001 From: slfan1989 <55643692+slfan1989@users.noreply.github.com> Date: Fri, 23 May 2025 00:21:14 +0800 Subject: [PATCH 394/397] RATIS-2303. Migrate ratis-examples tests to Junit 5. (#1267) (cherry picked from commit 5528be0d13f7ac2f38672e2eaf60fd3a7a561043) --- .../org/apache/ratis/util/ReferenceCountedLeakDetector.java | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/ratis-common/src/main/java/org/apache/ratis/util/ReferenceCountedLeakDetector.java b/ratis-common/src/main/java/org/apache/ratis/util/ReferenceCountedLeakDetector.java index 271058e549..330060a90f 100644 --- a/ratis-common/src/main/java/org/apache/ratis/util/ReferenceCountedLeakDetector.java +++ b/ratis-common/src/main/java/org/apache/ratis/util/ReferenceCountedLeakDetector.java @@ -120,7 +120,7 @@ public V retain() { if (count.getAndUpdate(n -> n < 0? n : n + 1) < 0) { throw new IllegalStateException("Failed to retain: object has already been completely released."); } - + LOG.trace("retain value : {}, count : {}.", value, count); retainMethod.run(); return value; } @@ -136,6 +136,7 @@ public boolean release() { } else if (previous == 0) { throw new IllegalStateException("Failed to release: object has not yet been retained."); } + LOG.trace("release value : {}, count : {}.", value, count); final boolean completedReleased = previous == 1; releaseMethod.accept(completedReleased); return completedReleased; From 8216f04c4d73bad020e2086bf12dc21b6a45b23d Mon Sep 17 00:00:00 2001 From: slfan1989 <55643692+slfan1989@users.noreply.github.com> Date: Fri, 23 May 2025 16:03:05 +0800 Subject: [PATCH 395/397] RATIS-1977. Remove Junit 4 dependencies. (#1269) (cherry picked from commit c56ce422d736104d216b539bafdd091cbdd8eb25) --- pom.xml | 32 ++++-------------- ratis-common/pom.xml | 21 ------------ .../test/java/org/apache/ratis/BaseTest.java | 7 ---- ratis-proto/pom.xml | 10 ++---- .../apache/ratis/RaftExceptionBaseTest.java | 4 +-- .../java/org/apache/ratis/RaftTestUtil.java | 18 +++------- .../server/impl/LeaderElectionTests.java | 2 +- .../impl/StateMachineShutdownTests.java | 33 +++++-------------- ratis-test/pom.xml | 10 ------ 9 files changed, 25 insertions(+), 112 deletions(-) diff --git a/pom.xml b/pom.xml index 1c23fe78d4..549c953319 100644 --- a/pom.xml +++ b/pom.xml @@ -191,8 +191,10 @@ 4 2.0.7 - 5.11.2 + 5.12.2 + 4.11.0 0.8.12 + 1.3.5 flaky | org.apache.ratis.test.tag.FlakyTest @@ -386,32 +388,10 @@ mockito-core ${mockito.version} - - - - io.opentelemetry - opentelemetry-api - ${opentelemetry.version} - - - io.opentelemetry - opentelemetry-sdk - ${opentelemetry.version} - - - io.opentelemetry - opentelemetry-sdk-testing - ${opentelemetry.version} - - - io.opentelemetry.semconv - opentelemetry-semconv - ${opentelemetry-semconv.version} - - io.opentelemetry - opentelemetry-context - ${opentelemetry.version} + jakarta.annotation + jakarta.annotation-api + ${jakarta.annotation.version} diff --git a/ratis-common/pom.xml b/ratis-common/pom.xml index bd7e75eddc..c2c4451d98 100644 --- a/ratis-common/pom.xml +++ b/ratis-common/pom.xml @@ -38,27 +38,6 @@ slf4j-api - - io.opentelemetry - opentelemetry-api - - - io.opentelemetry - opentelemetry-context - - - io.opentelemetry - opentelemetry-sdk - - - io.opentelemetry - opentelemetry-sdk-testing - - - io.opentelemetry.semconv - opentelemetry-semconv - - org.junit.jupiter junit-jupiter-api diff --git a/ratis-common/src/test/java/org/apache/ratis/BaseTest.java b/ratis-common/src/test/java/org/apache/ratis/BaseTest.java index e3e81a0a71..a8509c239a 100644 --- a/ratis-common/src/test/java/org/apache/ratis/BaseTest.java +++ b/ratis-common/src/test/java/org/apache/ratis/BaseTest.java @@ -26,8 +26,6 @@ import org.apache.ratis.util.StringUtils; import org.apache.ratis.util.TimeDuration; import org.apache.ratis.util.function.CheckedRunnable; -import org.junit.After; -import org.junit.Before; import org.junit.jupiter.api.AfterEach; import org.junit.jupiter.api.Assertions; import org.junit.jupiter.api.Assumptions; @@ -72,8 +70,6 @@ public void setFirstException(Throwable e) { } } - // TODO: Junit 4 reference should be removed once all the unit tests are migrated to Junit 5. - private String testCaseName; @BeforeEach @@ -85,8 +81,6 @@ public void setup(TestInfo testInfo) { + "." + (method == null? null : method.getName()); } - // @Before annotation is retained to support junit 4 tests. - @Before @BeforeEach public void checkAssumptions() { final int leaks = ReferenceCountedLeakDetector.getLeakDetector().getLeakCount(); @@ -131,7 +125,6 @@ public File getClassTestDir() { } public File getTestDir() { - // This will work for both junit 4 and 5. return new File(getClassTestDir(), testCaseName); } diff --git a/ratis-proto/pom.xml b/ratis-proto/pom.xml index 8211ff8189..611e3861f0 100644 --- a/ratis-proto/pom.xml +++ b/ratis-proto/pom.xml @@ -171,14 +171,8 @@ ratis-thirdparty-misc - org.junit.jupiter - junit-jupiter-engine - test - - - org.junit.platform - junit-platform-launcher - test + jakarta.annotation + jakarta.annotation-api diff --git a/ratis-server/src/test/java/org/apache/ratis/RaftExceptionBaseTest.java b/ratis-server/src/test/java/org/apache/ratis/RaftExceptionBaseTest.java index cfdda48fbb..53222cd6b8 100644 --- a/ratis-server/src/test/java/org/apache/ratis/RaftExceptionBaseTest.java +++ b/ratis-server/src/test/java/org/apache/ratis/RaftExceptionBaseTest.java @@ -35,7 +35,7 @@ import org.apache.ratis.util.Slf4jUtils; import org.apache.ratis.util.SizeInBytes; import org.junit.jupiter.api.Assertions; -import org.junit.Assume; +import org.junit.jupiter.api.Assumptions; import org.junit.jupiter.api.Test; import org.slf4j.event.Level; @@ -85,7 +85,7 @@ RaftClientReply assertNotLeaderException(RaftPeerId expectedSuggestedLeader, final SimpleMessage message = new SimpleMessage(messageId); final RaftClientReply reply = rpc.sendRequest(cluster.newRaftClientRequest(ClientId.randomId(), server, message)); Assertions.assertNotNull(reply); - Assume.assumeFalse(reply.isSuccess()); + Assumptions.assumeFalse(reply.isSuccess()); final NotLeaderException nle = reply.getNotLeaderException(); Objects.requireNonNull(nle); Assertions.assertEquals(expectedSuggestedLeader, nle.getSuggestedLeader().getId()); diff --git a/ratis-server/src/test/java/org/apache/ratis/RaftTestUtil.java b/ratis-server/src/test/java/org/apache/ratis/RaftTestUtil.java index 2ab208bad5..40a93df17f 100644 --- a/ratis-server/src/test/java/org/apache/ratis/RaftTestUtil.java +++ b/ratis-server/src/test/java/org/apache/ratis/RaftTestUtil.java @@ -32,7 +32,6 @@ import org.apache.ratis.server.impl.BlockRequestHandlingInjection; import org.apache.ratis.server.impl.DelayLocalExecutionInjection; import org.apache.ratis.server.impl.MiniRaftCluster; -import org.apache.ratis.server.protocol.TermIndex; import org.apache.ratis.server.raftlog.LogEntryHeader; import org.apache.ratis.server.raftlog.LogProtoUtils; import org.apache.ratis.server.raftlog.RaftLog; @@ -44,7 +43,8 @@ import org.apache.ratis.util.Preconditions; import org.apache.ratis.util.ProtoUtils; import org.apache.ratis.util.TimeDuration; -import org.junit.AssumptionViolatedException; +import org.apache.ratis.util.function.CheckedConsumer; +import org.junit.jupiter.api.Assumptions; import org.junit.jupiter.api.Assertions; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -188,11 +188,8 @@ static boolean logEntriesContains(RaftLog log, long startIndex, long endIndex, S log.get(termIndices[idxEntries].getIndex()).getStateMachineLogEntry().getLogData().toByteArray())) { ++idxExpected; } - } catch (Exception e) { - throw new IllegalStateException("Failed logEntriesContains: startIndex=" + startIndex - + ", endIndex=" + endIndex - + ", #expectedMessages=" + expectedMessages.length - + ", log=" + log, e); + } catch (IOException e) { + throw new RuntimeException(e); } ++idxEntries; } @@ -477,11 +474,6 @@ static List getPeersWithPriority(List peers, RaftPeer sugges return peersWithPriority; } - static RaftPeerId changeLeader(MiniRaftCluster cluster, RaftPeerId oldLeader) - throws Exception { - return changeLeader(cluster, oldLeader, AssumptionViolatedException::new); - } - static RaftPeerId changeLeader(MiniRaftCluster cluster, RaftPeerId oldLeader) throws Exception { return changeLeader(cluster, oldLeader, Assumptions::abort); @@ -565,7 +557,7 @@ static void assertSameLog(RaftLog expected, RaftLog computed) throws Exception { final long lastIndex = expected.getNextIndex() - 1; Assertions.assertEquals(expected.getLastEntryTermIndex().getIndex(), lastIndex); for(long i = 0; i < lastIndex; i++) { - Assertions.assertEquals(expected.get(i), computed.get(i), "Checking " + TermIndex.valueOf(expected.get(i))); + Assertions.assertEquals(expected.get(i), computed.get(i)); } } diff --git a/ratis-server/src/test/java/org/apache/ratis/server/impl/LeaderElectionTests.java b/ratis-server/src/test/java/org/apache/ratis/server/impl/LeaderElectionTests.java index 0b8f51bb51..c39b8a2c3c 100644 --- a/ratis-server/src/test/java/org/apache/ratis/server/impl/LeaderElectionTests.java +++ b/ratis-server/src/test/java/org/apache/ratis/server/impl/LeaderElectionTests.java @@ -183,7 +183,7 @@ public void testChangeLeader() throws Exception { void runTestChangeLeader(MiniRaftCluster cluster) throws Exception { RaftPeerId leader = RaftTestUtil.waitForLeader(cluster).getId(); for(int i = 0; i < 10; i++) { - leader = RaftTestUtil.changeLeader(cluster, leader, IllegalStateException::new); + leader = RaftTestUtil.changeLeader(cluster, leader, Assertions::fail); } } diff --git a/ratis-server/src/test/java/org/apache/ratis/server/impl/StateMachineShutdownTests.java b/ratis-server/src/test/java/org/apache/ratis/server/impl/StateMachineShutdownTests.java index 3ab1cf07bd..fc00b70bab 100644 --- a/ratis-server/src/test/java/org/apache/ratis/server/impl/StateMachineShutdownTests.java +++ b/ratis-server/src/test/java/org/apache/ratis/server/impl/StateMachineShutdownTests.java @@ -28,9 +28,8 @@ import org.apache.ratis.statemachine.impl.SimpleStateMachine4Testing; import org.apache.ratis.statemachine.StateMachine; import org.apache.ratis.statemachine.TransactionContext; -import org.junit.*; -import org.mockito.MockedStatic; -import org.mockito.Mockito; +import org.junit.jupiter.api.Assertions; +import org.junit.jupiter.api.Test; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -48,7 +47,6 @@ public abstract class StateMachineShutdownTests extends BaseTest implements MiniRaftCluster.Factory.Get { public static Logger LOG = LoggerFactory.getLogger(StateMachineUpdater.class); - private static MockedStatic mocked; protected static class StateMachineWithConditionalWait extends SimpleStateMachine4Testing { boolean unblockAllTxns = false; @@ -78,7 +76,7 @@ public void notifyTermIndexUpdated(long term, long index) { @Override public CompletableFuture applyTransaction(TransactionContext trx) { - final RaftProtos.LogEntryProto entry = trx.getLogEntryUnsafe(); + final RaftProtos.LogEntryProto entry = trx.getLogEntry(); CompletableFuture future = new CompletableFuture<>(); futures.computeIfAbsent(Thread.currentThread().getId(), k -> new HashSet<>()).add(future); @@ -121,19 +119,6 @@ public void unblockAllTxns() { } } - @Before - public void setup() { - mocked = Mockito.mockStatic(CompletableFuture.class, Mockito.CALLS_REAL_METHODS); - } - - @After - public void tearDownClass() { - if (mocked != null) { - mocked.close(); - } - - } - @Test public void testStateMachineShutdownWaitsForApplyTxn() throws Exception { final RaftProperties prop = getProperties(); @@ -176,24 +161,24 @@ public void testStateMachineShutdownWaitsForApplyTxn() throws Exception { // Now unblock the second follower long minIndex = ((StateMachineWithConditionalWait) secondFollower.getStateMachine()).blockTxns.stream() .min(Comparator.naturalOrder()).get(); - Assert.assertEquals(2, StateMachineWithConditionalWait.numTxns.values().stream() + Assertions.assertEquals(2, StateMachineWithConditionalWait.numTxns.values().stream() .filter(val -> val.get() == 3).count()); // The second follower should still be blocked in apply transaction - Assert.assertTrue(secondFollower.getInfo().getLastAppliedIndex() < minIndex); + Assertions.assertTrue(secondFollower.getInfo().getLastAppliedIndex() < minIndex); for (long index : ((StateMachineWithConditionalWait) secondFollower.getStateMachine()).blockTxns) { if (minIndex != index) { ((StateMachineWithConditionalWait) secondFollower.getStateMachine()).unBlockApplyTxn(index); } } - Assert.assertEquals(2, StateMachineWithConditionalWait.numTxns.values().stream() + Assertions.assertEquals(2, StateMachineWithConditionalWait.numTxns.values().stream() .filter(val -> val.get() == 3).count()); - Assert.assertTrue(secondFollower.getInfo().getLastAppliedIndex() < minIndex); + Assertions.assertTrue(secondFollower.getInfo().getLastAppliedIndex() < minIndex); ((StateMachineWithConditionalWait) secondFollower.getStateMachine()).unBlockApplyTxn(minIndex); // Now wait for the thread t.join(5000); - Assert.assertEquals(logIndex, secondFollower.getInfo().getLastAppliedIndex()); - Assert.assertEquals(3, StateMachineWithConditionalWait.numTxns.values().stream() + Assertions.assertTrue(logIndex <= secondFollower.getInfo().getLastAppliedIndex()); + Assertions.assertEquals(3, StateMachineWithConditionalWait.numTxns.values().stream() .filter(val -> val.get() == 3).count()); cluster.shutdown(); diff --git a/ratis-test/pom.xml b/ratis-test/pom.xml index d8fcb08932..5f55e4cde3 100644 --- a/ratis-test/pom.xml +++ b/ratis-test/pom.xml @@ -136,11 +136,6 @@ junit-jupiter-engine test - - org.junit.jupiter - junit-jupiter-engine - test - org.junit.jupiter junit-jupiter-api @@ -151,11 +146,6 @@ junit-jupiter-params test - - org.junit.vintage - junit-vintage-engine - test - org.junit.platform junit-platform-launcher From 645d66ef99cc1fa95045ab240871d861e5779077 Mon Sep 17 00:00:00 2001 From: slfan1989 <55643692+slfan1989@users.noreply.github.com> Date: Sun, 22 Feb 2026 03:16:05 +0800 Subject: [PATCH 396/397] RATIS-2414. Add leak detection for ZeroCopyMessageMarshaller. (#1355) (cherry picked from commit d5bd9c4bdce3cb908b201819043491b6aab9bd5c) --- .../grpc/util/ZeroCopyMessageMarshaller.java | 23 +++++++++++++++++++ .../ratis/grpc/util/TestGrpcZeroCopy.java | 18 +++++++++------ 2 files changed, 34 insertions(+), 7 deletions(-) diff --git a/ratis-grpc/src/main/java/org/apache/ratis/grpc/util/ZeroCopyMessageMarshaller.java b/ratis-grpc/src/main/java/org/apache/ratis/grpc/util/ZeroCopyMessageMarshaller.java index 3cdbc07c71..eddf2495e4 100644 --- a/ratis-grpc/src/main/java/org/apache/ratis/grpc/util/ZeroCopyMessageMarshaller.java +++ b/ratis-grpc/src/main/java/org/apache/ratis/grpc/util/ZeroCopyMessageMarshaller.java @@ -230,4 +230,27 @@ public InputStream popStream(T message) { public int getUnclosedCount() { return unclosedStreams.size(); } + + void assertNoUnclosedStreams() { + // Intended for tests/teardown to fail fast if callers forgot to release streams. + final int size = unclosedStreams.size(); + Preconditions.assertTrue(size == 0, () -> name + ": " + size + " unclosed stream(s)"); + } + + public void close() { + // Cleanup helper for tests/teardown; do not call while streams may still be in use. + synchronized (unclosedStreams) { + if (unclosedStreams.isEmpty()) { + return; + } + for (InputStream stream : unclosedStreams.values()) { + try { + stream.close(); + } catch (IOException e) { + LOG.warn("{}: Failed to close leaked stream.", name, e); + } + } + unclosedStreams.clear(); + } + } } diff --git a/ratis-test/src/test/java/org/apache/ratis/grpc/util/TestGrpcZeroCopy.java b/ratis-test/src/test/java/org/apache/ratis/grpc/util/TestGrpcZeroCopy.java index f2427edd94..dde44e5799 100644 --- a/ratis-test/src/test/java/org/apache/ratis/grpc/util/TestGrpcZeroCopy.java +++ b/ratis-test/src/test/java/org/apache/ratis/grpc/util/TestGrpcZeroCopy.java @@ -26,9 +26,13 @@ import org.apache.ratis.thirdparty.io.netty.buffer.PooledByteBufAllocator; import org.apache.ratis.util.NetUtils; import org.apache.ratis.util.TraditionalBinaryPrefix; -import org.junit.jupiter.api.Assertions; import org.junit.jupiter.api.Test; +import static org.junit.jupiter.api.Assertions.assertEquals; +import static org.junit.jupiter.api.Assertions.assertThrows; +import static org.junit.jupiter.api.Assertions.assertTrue; +import static org.junit.jupiter.api.Assumptions.assumeTrue; + import java.util.ArrayList; import java.util.List; import java.util.Random; @@ -60,8 +64,8 @@ static void verify(long seed, ByteString b) { RANDOM.nextBytes(ARRAY); final ByteString expected = UnsafeByteOperations.unsafeWrap(ARRAY, 0, remaining); final ByteString computed = b.substring(offset, offset + remaining); - Assertions.assertEquals(expected.size(), computed.size()); - Assertions.assertEquals(expected, computed); + assertEquals(expected.size(), computed.size()); + assertEquals(expected, computed); offset += remaining; } } @@ -99,7 +103,7 @@ public static boolean isReady() { /** Test a zero-copy marshaller is available from the versions of gRPC and Protobuf. */ @Test public void testReadiness() { - Assertions.assertTrue(isReady()); + assertTrue(isReady()); } @@ -146,7 +150,7 @@ void sendMessages(int n, GrpcZeroCopyTestClient client, GrpcZeroCopyTestServer s for (int i = 0; i < futures.size(); i++) { final String expected = GrpcZeroCopyTestServer.toReply(i, messages.get(i)); final String reply = futures.get(i).get(); - Assertions.assertEquals(expected, reply, "expected = " + expected + " != reply = " + reply); + assertEquals(expected, reply, "expected = " + expected + " != reply = " + reply); server.assertCounts(numElements, numBytes); } } @@ -171,8 +175,8 @@ void sendBinaries(int n, GrpcZeroCopyTestClient client, GrpcZeroCopyTestServer s } final ByteString reply = future.get(); - Assertions.assertEquals(4, reply.size()); - Assertions.assertEquals(size, reply.asReadOnlyByteBuffer().getInt()); + assertEquals(4, reply.size()); + assertEquals(size, reply.asReadOnlyByteBuffer().getInt()); numElements++; numBytes += size; From 667ecc1538961ee0215b427753f90b577b3329e5 Mon Sep 17 00:00:00 2001 From: slfan1989 <55643692+slfan1989@users.noreply.github.com> Date: Sun, 12 Apr 2026 23:42:50 +0800 Subject: [PATCH 397/397] RATIS-2501. Improve diagnostics for testInstallSnapshotDuringBootstrap timeout failures. (#1427) (cherry picked from commit f76cb2e2dca5101f4d2e7e85e43908b0036e531d) --- pom.xml | 37 ++- ratis-assembly/pom.xml | 2 +- ratis-client/pom.xml | 2 +- .../ratis/client/impl/OrderedAsync.java | 3 +- ratis-common/pom.xml | 23 +- .../RaftClientAsynchronousProtocol.java | 13 +- .../ratis/retry/ExponentialBackoffRetry.java | 1 + .../apache/ratis/util/BiWeakValueCache.java | 2 +- .../java/org/apache/ratis/util/FileUtils.java | 1 + ratis-docs/pom.xml | 15 +- ratis-examples/pom.xml | 2 +- ratis-experiments/pom.xml | 2 +- ratis-grpc/pom.xml | 7 +- .../org/apache/ratis/grpc/GrpcConfigKeys.java | 41 +++- .../org/apache/ratis/grpc/GrpcFactory.java | 52 ++--- .../java/org/apache/ratis/grpc/GrpcUtil.java | 30 ++- .../grpc/client/GrpcClientProtocolClient.java | 28 +-- .../grpc/client/GrpcClientProtocolProxy.java | 108 +++++++++ .../ratis/grpc/client/GrpcClientRpc.java | 72 ++---- .../ratis/grpc/metrics/MessageMetrics.java | 8 + .../ratis/grpc/server/GrpcLogAppender.java | 104 ++++++--- .../grpc/server/GrpcServerProtocolClient.java | 82 ++----- .../server/GrpcServerProtocolService.java | 3 +- .../ratis/grpc/server/GrpcServicesImpl.java | 12 +- ratis-metrics-api/pom.xml | 2 +- ratis-metrics-default/pom.xml | 2 +- ratis-metrics-dropwizard3/pom.xml | 2 +- ratis-netty/pom.xml | 2 +- ratis-proto/pom.xml | 12 +- ratis-resource-bundle/pom.xml | 2 +- ratis-server-api/pom.xml | 2 +- .../ratis/server/leader/LogAppender.java | 83 ++----- .../ratis/server/protocol/TermIndex.java | 13 ++ .../apache/ratis/server/raftlog/RaftLog.java | 3 - .../statemachine/TransactionContext.java | 1 + ratis-server/pom.xml | 2 +- .../server/impl/ConfigurationManager.java | 6 +- .../ratis/server/impl/FollowerState.java | 4 +- .../ratis/server/impl/LeaderElection.java | 10 +- .../ratis/server/impl/LeaderStateImpl.java | 23 +- .../ratis/server/impl/PendingRequests.java | 17 +- .../ratis/server/impl/RaftServerImpl.java | 194 ++++++++-------- .../ratis/server/impl/RaftServerProxy.java | 16 +- .../ratis/server/impl/ReadRequests.java | 50 ++-- .../ratis/server/impl/RetryCacheImpl.java | 23 +- .../ratis/server/impl/ServerImplUtils.java | 3 - .../ratis/server/impl/ServerProtoUtils.java | 21 +- .../apache/ratis/server/impl/ServerState.java | 51 ++--- .../impl/SnapshotInstallationHandler.java | 14 +- .../server/impl/StateMachineUpdater.java | 9 +- .../ratis/server/impl/WriteIndexCache.java | 5 +- .../leader/InstallSnapshotRequests.java | 2 +- .../ratis/server/leader/LogAppenderBase.java | 39 ++-- .../server/leader/LogAppenderDaemon.java | 4 - .../ratis/server/raftlog/RaftLogBase.java | 10 +- .../server/raftlog/memory/MemoryRaftLog.java | 1 + .../server/raftlog/segmented/LogSegment.java | 57 ++--- .../raftlog/segmented/SegmentedRaftLog.java | 16 +- .../segmented/SegmentedRaftLogCache.java | 14 +- .../SegmentedRaftLogInputStream.java | 2 +- .../segmented/SegmentedRaftLogReader.java | 16 +- .../segmented/SegmentedRaftLogWorker.java | 7 +- .../ratis/server/storage/FileChunkReader.java | 22 +- .../ratis/server/storage/SnapshotManager.java | 6 +- .../ratis/server/util/ServerStringUtils.java | 10 +- .../statemachine/impl/BaseStateMachine.java | 6 +- .../impl/SimpleStateMachineStorage.java | 84 +++---- .../impl/SingleFileSnapshotInfo.java | 5 - .../impl/TransactionContextImpl.java | 5 + .../ratis/InstallSnapshotFromLeaderTests.java | 97 ++++++-- .../InstallSnapshotNotificationTests.java | 4 +- .../org/apache/ratis/LogAppenderTests.java | 91 +++++++- .../apache/ratis/OutputStreamBaseTest.java | 1 + .../java/org/apache/ratis/RaftBasicTests.java | 7 + .../apache/ratis/RaftExceptionBaseTest.java | 8 +- .../apache/ratis/ReadOnlyRequestTests.java | 205 ++--------------- .../server/impl/LeaderElectionTests.java | 215 ++++++++++-------- .../ratis/server/impl/MiniRaftCluster.java | 53 +---- .../impl/RaftReconfigurationBaseTest.java | 11 +- .../impl/RaftStateMachineExceptionTests.java | 27 +++ .../statemachine/RaftSnapshotBaseTest.java | 36 +-- .../impl/SimpleStateMachine4Testing.java | 15 +- ratis-shell/pom.xml | 2 +- ratis-test/pom.xml | 2 +- ratis-tools/pom.xml | 2 +- 85 files changed, 1164 insertions(+), 1140 deletions(-) create mode 100644 ratis-grpc/src/main/java/org/apache/ratis/grpc/client/GrpcClientProtocolProxy.java diff --git a/pom.xml b/pom.xml index 549c953319..4ffed3fdc2 100644 --- a/pom.xml +++ b/pom.xml @@ -24,7 +24,7 @@ ratis org.apache.ratis - 3.2.0-SNAPSHOT + 3.3.0-SNAPSHOT Apache Ratis pom @@ -190,11 +190,10 @@ 4 - 2.0.7 - 5.12.2 + 2.0.17 + 5.14.3 4.11.0 - 0.8.12 - 1.3.5 + 0.8.14 flaky | org.apache.ratis.test.tag.FlakyTest @@ -388,10 +387,32 @@ mockito-core ${mockito.version} + + + + io.opentelemetry + opentelemetry-api + ${opentelemetry.version} + + + io.opentelemetry + opentelemetry-sdk + ${opentelemetry.version} + + + io.opentelemetry + opentelemetry-sdk-testing + ${opentelemetry.version} + + + io.opentelemetry.semconv + opentelemetry-semconv + ${opentelemetry-semconv.version} + - jakarta.annotation - jakarta.annotation-api - ${jakarta.annotation.version} + io.opentelemetry + opentelemetry-context + ${opentelemetry.version} diff --git a/ratis-assembly/pom.xml b/ratis-assembly/pom.xml index 7cb758616c..0558693039 100644 --- a/ratis-assembly/pom.xml +++ b/ratis-assembly/pom.xml @@ -17,7 +17,7 @@ ratis org.apache.ratis - 3.2.0-SNAPSHOT + 3.3.0-SNAPSHOT ratis-assembly diff --git a/ratis-client/pom.xml b/ratis-client/pom.xml index f9de230b23..26b2034983 100644 --- a/ratis-client/pom.xml +++ b/ratis-client/pom.xml @@ -17,7 +17,7 @@ ratis org.apache.ratis - 3.2.0-SNAPSHOT + 3.3.0-SNAPSHOT ratis-client diff --git a/ratis-client/src/main/java/org/apache/ratis/client/impl/OrderedAsync.java b/ratis-client/src/main/java/org/apache/ratis/client/impl/OrderedAsync.java index a380989641..50e8093a13 100644 --- a/ratis-client/src/main/java/org/apache/ratis/client/impl/OrderedAsync.java +++ b/ratis-client/src/main/java/org/apache/ratis/client/impl/OrderedAsync.java @@ -51,6 +51,7 @@ import java.util.concurrent.ConcurrentMap; import java.util.concurrent.Semaphore; import java.util.concurrent.atomic.AtomicReference; +import java.util.function.Consumer; import java.util.function.Function; import java.util.function.LongFunction; @@ -231,7 +232,7 @@ private void logError(String prefix, RaftClientRequest request, Throwable e) { } private boolean logError(String prefix, RaftClientRequest request, Throwable e, Class cause) { - if (JavaUtils.isCausedBy(e, cause)) { + if (JavaUtils.unwrapCompletionException(e).getClass().isAssignableFrom(cause)) { LOG.error("{} {}: Failed* {} due to {} caused by {}", prefix, client.getId(), request, e, cause.getSimpleName()); return true; diff --git a/ratis-common/pom.xml b/ratis-common/pom.xml index c2c4451d98..ba19c73e33 100644 --- a/ratis-common/pom.xml +++ b/ratis-common/pom.xml @@ -17,7 +17,7 @@ ratis org.apache.ratis - 3.2.0-SNAPSHOT + 3.3.0-SNAPSHOT ratis-common @@ -38,6 +38,27 @@ slf4j-api + + io.opentelemetry + opentelemetry-api + + + io.opentelemetry + opentelemetry-context + + + io.opentelemetry + opentelemetry-sdk + + + io.opentelemetry + opentelemetry-sdk-testing + + + io.opentelemetry.semconv + opentelemetry-semconv + + org.junit.jupiter junit-jupiter-api diff --git a/ratis-common/src/main/java/org/apache/ratis/protocol/RaftClientAsynchronousProtocol.java b/ratis-common/src/main/java/org/apache/ratis/protocol/RaftClientAsynchronousProtocol.java index 49408aa937..428bdaf18d 100644 --- a/ratis-common/src/main/java/org/apache/ratis/protocol/RaftClientAsynchronousProtocol.java +++ b/ratis-common/src/main/java/org/apache/ratis/protocol/RaftClientAsynchronousProtocol.java @@ -17,13 +17,22 @@ */ package org.apache.ratis.protocol; +import org.apache.ratis.util.JavaUtils; +import org.apache.ratis.util.ReferenceCountedObject; + import java.io.IOException; import java.util.concurrent.CompletableFuture; /** Asynchronous version of {@link RaftClientProtocol}. */ public interface RaftClientAsynchronousProtocol { - CompletableFuture submitClientRequestAsync( - RaftClientRequest request) throws IOException; + /** + * It is recommended to override {@link #submitClientRequestAsync(ReferenceCountedObject)} instead. + * Then, it does not have to override this method. + */ + default CompletableFuture submitClientRequestAsync( + RaftClientRequest request) throws IOException { + return submitClientRequestAsync(ReferenceCountedObject.wrap(request)); + } /** * A referenced counted request is submitted from a client for processing. diff --git a/ratis-common/src/main/java/org/apache/ratis/retry/ExponentialBackoffRetry.java b/ratis-common/src/main/java/org/apache/ratis/retry/ExponentialBackoffRetry.java index 5af7b75d36..90c7efbf8a 100644 --- a/ratis-common/src/main/java/org/apache/ratis/retry/ExponentialBackoffRetry.java +++ b/ratis-common/src/main/java/org/apache/ratis/retry/ExponentialBackoffRetry.java @@ -17,6 +17,7 @@ */ package org.apache.ratis.retry; +import org.apache.ratis.util.Preconditions; import org.apache.ratis.util.TimeDuration; import java.util.Objects; diff --git a/ratis-common/src/main/java/org/apache/ratis/util/BiWeakValueCache.java b/ratis-common/src/main/java/org/apache/ratis/util/BiWeakValueCache.java index c1aa6bcd5a..52940ed883 100644 --- a/ratis-common/src/main/java/org/apache/ratis/util/BiWeakValueCache.java +++ b/ratis-common/src/main/java/org/apache/ratis/util/BiWeakValueCache.java @@ -39,7 +39,7 @@ * @param the type to be cached. */ public final class BiWeakValueCache { - private static ConcurrentMap newMap() { + static ConcurrentMap newMap() { return new MapMaker().weakValues().makeMap(); } diff --git a/ratis-common/src/main/java/org/apache/ratis/util/FileUtils.java b/ratis-common/src/main/java/org/apache/ratis/util/FileUtils.java index 59db028986..79c00b5a35 100644 --- a/ratis-common/src/main/java/org/apache/ratis/util/FileUtils.java +++ b/ratis-common/src/main/java/org/apache/ratis/util/FileUtils.java @@ -33,6 +33,7 @@ import java.nio.file.FileVisitResult; import java.nio.file.Files; import java.nio.file.LinkOption; +import java.nio.file.NotDirectoryException; import java.nio.file.OpenOption; import java.nio.file.Path; import java.nio.file.Paths; diff --git a/ratis-docs/pom.xml b/ratis-docs/pom.xml index 1bb1c64589..894b6a52e4 100644 --- a/ratis-docs/pom.xml +++ b/ratis-docs/pom.xml @@ -20,7 +20,7 @@ https://maven.apache.org/xsd/maven-4.0.0.xsd"> ratis org.apache.ratis - 3.2.0-SNAPSHOT + 3.3.0-SNAPSHOT ratis-docs @@ -32,4 +32,17 @@ https://maven.apache.org/xsd/maven-4.0.0.xsd"> true + + + org.junit.jupiter + junit-jupiter-engine + test + + + org.junit.platform + junit-platform-launcher + test + + + diff --git a/ratis-examples/pom.xml b/ratis-examples/pom.xml index b763da7fd7..c52019334f 100644 --- a/ratis-examples/pom.xml +++ b/ratis-examples/pom.xml @@ -17,7 +17,7 @@ ratis org.apache.ratis - 3.2.0-SNAPSHOT + 3.3.0-SNAPSHOT ratis-examples diff --git a/ratis-experiments/pom.xml b/ratis-experiments/pom.xml index 001d1755bf..e4a8451db2 100644 --- a/ratis-experiments/pom.xml +++ b/ratis-experiments/pom.xml @@ -17,7 +17,7 @@ ratis org.apache.ratis - 3.2.0-SNAPSHOT + 3.3.0-SNAPSHOT ratis-experiments diff --git a/ratis-grpc/pom.xml b/ratis-grpc/pom.xml index ced645fa79..360131d55b 100644 --- a/ratis-grpc/pom.xml +++ b/ratis-grpc/pom.xml @@ -17,7 +17,7 @@ ratis org.apache.ratis - 3.2.0-SNAPSHOT + 3.3.0-SNAPSHOT ratis-grpc @@ -73,11 +73,6 @@ junit-jupiter-api test - - org.mockito - mockito-core - test - diff --git a/ratis-grpc/src/main/java/org/apache/ratis/grpc/GrpcConfigKeys.java b/ratis-grpc/src/main/java/org/apache/ratis/grpc/GrpcConfigKeys.java index 4ed9b5f5b8..2fcb9b6b0a 100644 --- a/ratis-grpc/src/main/java/org/apache/ratis/grpc/GrpcConfigKeys.java +++ b/ratis-grpc/src/main/java/org/apache/ratis/grpc/GrpcConfigKeys.java @@ -54,29 +54,50 @@ static Consumer getDefaultLog() { interface TLS { String PREFIX = GrpcConfigKeys.PREFIX + ".tls"; - @Deprecated + String ENABLED_KEY = PREFIX + ".enabled"; + boolean ENABLED_DEFAULT = false; + static boolean enabled(RaftProperties properties) { + return getBoolean(properties::getBoolean, ENABLED_KEY, ENABLED_DEFAULT, getDefaultLog()); + } static void setEnabled(RaftProperties properties, boolean enabled) { - LOG.warn("This method has no effect. Use setConf(Parameters, GrpcTlsConfig) instead."); + setBoolean(properties::setBoolean, ENABLED_KEY, enabled); } - @Deprecated + String MUTUAL_AUTHN_ENABLED_KEY = PREFIX + ".mutual_authn.enabled"; + boolean MUTUAL_AUTHN_ENABLED_DEFAULT = false; + static boolean mutualAuthnEnabled(RaftProperties properties) { + return getBoolean(properties::getBoolean, + MUTUAL_AUTHN_ENABLED_KEY, MUTUAL_AUTHN_ENABLED_DEFAULT, getDefaultLog()); + } static void setMutualAuthnEnabled(RaftProperties properties, boolean mutualAuthnEnabled) { - LOG.warn("This method has no effect. Use setConf(Parameters, GrpcTlsConfig) instead."); + setBoolean(properties::setBoolean, MUTUAL_AUTHN_ENABLED_KEY, mutualAuthnEnabled); } - @Deprecated + String PRIVATE_KEY_FILE_NAME_KEY = PREFIX + ".private.key.file.name"; + String PRIVATE_KEY_FILE_NAME_DEFAULT = "private.pem"; + static String privateKeyFileName(RaftProperties properties) { + return get(properties::get, PRIVATE_KEY_FILE_NAME_KEY, PRIVATE_KEY_FILE_NAME_DEFAULT, getDefaultLog()); + } static void setPrivateKeyFileName(RaftProperties properties, String privateKeyFileName) { - LOG.warn("This method has no effect. Use setConf(Parameters, GrpcTlsConfig) instead."); + set(properties::set, PRIVATE_KEY_FILE_NAME_KEY, privateKeyFileName); } - @Deprecated + String CERT_CHAIN_FILE_NAME_KEY = PREFIX + ".cert.chain.file.name"; + String CERT_CHAIN_FILE_NAME_DEFAULT = "certificate.crt"; + static String certChainFileName(RaftProperties properties) { + return get(properties::get, CERT_CHAIN_FILE_NAME_KEY, CERT_CHAIN_FILE_NAME_DEFAULT, getDefaultLog()); + } static void setCertChainFileName(RaftProperties properties, String certChainFileName) { - LOG.warn("This method has no effect. Use setConf(Parameters, GrpcTlsConfig) instead."); + set(properties::set, CERT_CHAIN_FILE_NAME_KEY, certChainFileName); } - @Deprecated + String TRUST_STORE_KEY = PREFIX + ".trust.store"; + String TRUST_STORE_DEFAULT = "ca.crt"; + static String trustStore(RaftProperties properties) { + return get(properties::get, TRUST_STORE_KEY, TRUST_STORE_DEFAULT, getDefaultLog()); + } static void setTrustStore(RaftProperties properties, String trustStore) { - LOG.warn("This method has no effect. Use setConf(Parameters, GrpcTlsConfig) instead."); + set(properties::set, TRUST_STORE_KEY, trustStore); } String CONF_PARAMETER = PREFIX + ".conf"; diff --git a/ratis-grpc/src/main/java/org/apache/ratis/grpc/GrpcFactory.java b/ratis-grpc/src/main/java/org/apache/ratis/grpc/GrpcFactory.java index 8af523d55b..331d1a8585 100644 --- a/ratis-grpc/src/main/java/org/apache/ratis/grpc/GrpcFactory.java +++ b/ratis-grpc/src/main/java/org/apache/ratis/grpc/GrpcFactory.java @@ -32,15 +32,11 @@ import org.apache.ratis.server.leader.FollowerInfo; import org.apache.ratis.server.leader.LeaderState; import org.apache.ratis.thirdparty.io.netty.buffer.PooledByteBufAllocator; -import org.apache.ratis.thirdparty.io.netty.handler.ssl.SslContext; import org.apache.ratis.util.JavaUtils; -import org.apache.ratis.util.MemoizedSupplier; import org.slf4j.Logger; import org.slf4j.LoggerFactory; -import java.util.function.BiFunction; import java.util.function.Consumer; -import java.util.function.Supplier; public class GrpcFactory implements ServerFactory, ClientFactory { @@ -76,29 +72,12 @@ static boolean checkPooledByteBufAllocatorUseCacheForAllThreads(Consumer private final GrpcTlsConfig clientTlsConfig; private final GrpcTlsConfig serverTlsConfig; - static final BiFunction BUILD_SSL_CONTEXT_FOR_CLIENT - = (tlsConf, defaultContext) -> tlsConf == null ? defaultContext : GrpcUtil.buildSslContextForClient(tlsConf); - - static final class SslContexts { - private final SslContext adminSslContext; - private final SslContext clientSslContext; - private final SslContext serverSslContext; - - private SslContexts(GrpcTlsConfig tlsConfig, GrpcTlsConfig adminTlsConfig, - GrpcTlsConfig clientTlsConfig, GrpcTlsConfig serverTlsConfig, - BiFunction buildMethod) { - final SslContext defaultSslContext = buildMethod.apply(tlsConfig, null); - this.adminSslContext = buildMethod.apply(adminTlsConfig, defaultSslContext); - this.clientSslContext = buildMethod.apply(clientTlsConfig, defaultSslContext); - this.serverSslContext = buildMethod.apply(serverTlsConfig, defaultSslContext); - } + public static Parameters newRaftParameters(GrpcTlsConfig conf) { + final Parameters p = new Parameters(); + GrpcConfigKeys.TLS.setConf(p, conf); + return p; } - private final GrpcServices.Customizer servicesCustomizer; - - private final Supplier forServerSupplier; - private final Supplier forClientSupplier; - public GrpcFactory(Parameters parameters) { this(GrpcConfigKeys.Server.servicesCustomizer(parameters), GrpcConfigKeys.TLS.conf(parameters), @@ -123,10 +102,20 @@ private GrpcFactory(GrpcServices.Customizer servicesCustomizer, this.serverTlsConfig = serverTlsConfig; } - this.forServerSupplier = MemoizedSupplier.valueOf(() -> new SslContexts( - tlsConfig, adminTlsConfig, clientTlsConfig, serverTlsConfig, BUILD_SSL_CONTEXT_FOR_SERVER)); - this.forClientSupplier = MemoizedSupplier.valueOf(() -> new SslContexts( - tlsConfig, adminTlsConfig, clientTlsConfig, serverTlsConfig, BUILD_SSL_CONTEXT_FOR_CLIENT)); + public GrpcTlsConfig getTlsConfig() { + return tlsConfig; + } + + public GrpcTlsConfig getAdminTlsConfig() { + return adminTlsConfig != null ? adminTlsConfig : tlsConfig; + } + + public GrpcTlsConfig getClientTlsConfig() { + return clientTlsConfig != null ? clientTlsConfig : tlsConfig; + } + + public GrpcTlsConfig getServerTlsConfig() { + return serverTlsConfig != null ? serverTlsConfig : tlsConfig; } @Override @@ -154,8 +143,7 @@ public GrpcServices newRaftServerRpc(RaftServer server) { @Override public GrpcClientRpc newRaftClientRpc(ClientId clientId, RaftProperties properties) { checkPooledByteBufAllocatorUseCacheForAllThreads(LOG::debug); - - final SslContexts forClient = forClientSupplier.get(); - return new GrpcClientRpc(clientId, properties, forClient.adminSslContext, forClient.clientSslContext); + return new GrpcClientRpc(clientId, properties, + getAdminTlsConfig(), getClientTlsConfig()); } } diff --git a/ratis-grpc/src/main/java/org/apache/ratis/grpc/GrpcUtil.java b/ratis-grpc/src/main/java/org/apache/ratis/grpc/GrpcUtil.java index ff1a52c7a7..311bcb8778 100644 --- a/ratis-grpc/src/main/java/org/apache/ratis/grpc/GrpcUtil.java +++ b/ratis-grpc/src/main/java/org/apache/ratis/grpc/GrpcUtil.java @@ -32,10 +32,7 @@ import org.apache.ratis.thirdparty.io.grpc.ServerServiceDefinition; import org.apache.ratis.thirdparty.io.grpc.Status; import org.apache.ratis.thirdparty.io.grpc.StatusRuntimeException; -import org.apache.ratis.thirdparty.io.grpc.netty.GrpcSslContexts; import org.apache.ratis.thirdparty.io.grpc.stub.StreamObserver; -import org.apache.ratis.thirdparty.io.netty.handler.ssl.ClientAuth; -import org.apache.ratis.thirdparty.io.netty.handler.ssl.SslContext; import org.apache.ratis.thirdparty.io.netty.handler.ssl.SslContextBuilder; import org.apache.ratis.util.IOUtils; import org.apache.ratis.util.JavaUtils; @@ -46,7 +43,6 @@ import org.slf4j.LoggerFactory; import javax.net.ssl.KeyManager; -import javax.net.ssl.SSLException; import javax.net.ssl.TrustManager; import java.io.IOException; import java.util.concurrent.CompletableFuture; @@ -54,8 +50,6 @@ import java.util.function.Function; import java.util.function.Supplier; -import static org.apache.ratis.thirdparty.io.netty.handler.ssl.SslProvider.OPENSSL; - public interface GrpcUtil { Logger LOG = LoggerFactory.getLogger(GrpcUtil.class); @@ -82,6 +76,10 @@ static StatusRuntimeException wrapException(Throwable t, long callId, boolean is .build(); return wrapException(t, trailers); } + static StatusRuntimeException wrapException(Throwable t, long callId) { + return wrapException(t, callId, false); + } + static StatusRuntimeException wrapException(Throwable t, Metadata trailers) { return new StatusRuntimeException( @@ -95,7 +93,7 @@ static Throwable unwrapThrowable(Throwable t) { return unwrapped; } } - return JavaUtils.unwrapCompletionException(t); + return t; } static IOException unwrapException(StatusRuntimeException se) { @@ -142,10 +140,8 @@ static Throwable tryUnwrapThrowable(StatusRuntimeException se) { static long getCallId(Throwable t) { if (t instanceof StatusRuntimeException) { final Metadata trailers = ((StatusRuntimeException)t).getTrailers(); - if (trailers != null) { - final String callId = trailers.get(CALL_ID); - return callId != null ? Long.parseUnsignedLong(callId) : -1; - } + String callId = trailers.get(CALL_ID); + return callId != null ? Integer.parseInt(callId) : -1; } return -1; } @@ -153,8 +149,8 @@ static long getCallId(Throwable t) { static boolean isHeartbeat(Throwable t) { if (t instanceof StatusRuntimeException) { final Metadata trailers = ((StatusRuntimeException)t).getTrailers(); - final String isHeartbeat = trailers != null ? trailers.get(HEARTBEAT) : null; - return Boolean.parseBoolean(isHeartbeat); + String isHeartbeat = trailers != null ? trailers.get(HEARTBEAT) : null; + return isHeartbeat != null && Boolean.valueOf(isHeartbeat); } return false; } @@ -162,7 +158,7 @@ static boolean isHeartbeat(Throwable t) { static IOException unwrapIOException(Throwable t) { final IOException e; if (t instanceof StatusRuntimeException) { - e = unwrapException((StatusRuntimeException) t); + e = GrpcUtil.unwrapException((StatusRuntimeException) t); } else { e = IOUtils.asIOException(t); } @@ -178,7 +174,7 @@ static void asyncCall( supplier.get().whenComplete((reply, exception) -> { if (exception != null) { warning.accept(exception); - responseObserver.onError(wrapException(exception)); + responseObserver.onError(GrpcUtil.wrapException(exception)); } else { responseObserver.onNext(toProto.apply(reply)); responseObserver.onCompleted(); @@ -186,7 +182,7 @@ static void asyncCall( }); } catch (Exception e) { warning.accept(e); - responseObserver.onError(wrapException(e)); + responseObserver.onError(GrpcUtil.wrapException(e)); } } @@ -195,7 +191,7 @@ static void warn(Logger log, Supplier message, Throwable t) { } class StatusRuntimeExceptionMetadataBuilder { - private final Metadata trailers = new Metadata(); + private Metadata trailers = new Metadata(); StatusRuntimeExceptionMetadataBuilder(Throwable t) { trailers.put(EXCEPTION_TYPE_KEY, t.getClass().getCanonicalName()); diff --git a/ratis-grpc/src/main/java/org/apache/ratis/grpc/client/GrpcClientProtocolClient.java b/ratis-grpc/src/main/java/org/apache/ratis/grpc/client/GrpcClientProtocolClient.java index 8f089aa1c7..3b9d512683 100644 --- a/ratis-grpc/src/main/java/org/apache/ratis/grpc/client/GrpcClientProtocolClient.java +++ b/ratis-grpc/src/main/java/org/apache/ratis/grpc/client/GrpcClientProtocolClient.java @@ -21,6 +21,7 @@ import org.apache.ratis.client.impl.ClientProtoUtils; import org.apache.ratis.conf.RaftProperties; import org.apache.ratis.grpc.GrpcConfigKeys; +import org.apache.ratis.grpc.GrpcTlsConfig; import org.apache.ratis.grpc.GrpcUtil; import org.apache.ratis.grpc.metrics.intercept.client.MetricClientInterceptor; import org.apache.ratis.proto.RaftProtos.GroupInfoReplyProto; @@ -48,10 +49,11 @@ import org.apache.ratis.protocol.exceptions.TimeoutIOException; import org.apache.ratis.thirdparty.io.grpc.ManagedChannel; import org.apache.ratis.thirdparty.io.grpc.StatusRuntimeException; +import org.apache.ratis.thirdparty.io.grpc.netty.GrpcSslContexts; import org.apache.ratis.thirdparty.io.grpc.netty.NegotiationType; import org.apache.ratis.thirdparty.io.grpc.netty.NettyChannelBuilder; import org.apache.ratis.thirdparty.io.grpc.stub.StreamObserver; -import org.apache.ratis.thirdparty.io.netty.handler.ssl.SslContext; +import org.apache.ratis.thirdparty.io.netty.handler.ssl.SslContextBuilder; import org.apache.ratis.util.CollectionUtils; import org.apache.ratis.util.JavaUtils; import org.apache.ratis.util.SizeInBytes; @@ -82,7 +84,6 @@ public class GrpcClientProtocolClient implements Closeable { private final ManagedChannel clientChannel; private final ManagedChannel adminChannel; - private final SizeInBytes maxMessageSize; private final TimeDuration requestTimeoutDuration; private final TimeDuration watchRequestTimeoutDuration; private final TimeoutExecutor scheduler = TimeoutExecutor.getInstance(); @@ -96,11 +97,11 @@ public class GrpcClientProtocolClient implements Closeable { private final MetricClientInterceptor metricClientInterceptor; GrpcClientProtocolClient(ClientId id, RaftPeer target, RaftProperties properties, - SslContext adminSslContext, SslContext clientSslContext) { + GrpcTlsConfig adminTlsConfig, GrpcTlsConfig clientTlsConfig) { this.name = JavaUtils.memoize(() -> id + "->" + target.getId()); this.target = target; final SizeInBytes flowControlWindow = GrpcConfigKeys.flowControlWindow(properties, LOG::debug); - this.maxMessageSize = GrpcConfigKeys.messageSizeMax(properties, LOG::debug); + final SizeInBytes maxMessageSize = GrpcConfigKeys.messageSizeMax(properties, LOG::debug); metricClientInterceptor = new MetricClientInterceptor(getName()); final String clientAddress = Optional.ofNullable(target.getClientAddress()) @@ -109,9 +110,11 @@ public class GrpcClientProtocolClient implements Closeable { .filter(x -> !x.isEmpty()).orElse(target.getAddress()); final boolean separateAdminChannel = !Objects.equals(clientAddress, adminAddress); - clientChannel = buildChannel(clientAddress, clientSslContext, flowControlWindow); + clientChannel = buildChannel(clientAddress, clientTlsConfig, + flowControlWindow, maxMessageSize); adminChannel = separateAdminChannel - ? buildChannel(adminAddress, adminSslContext, flowControlWindow) + ? buildChannel(adminAddress, adminTlsConfig, + flowControlWindow, maxMessageSize) : clientChannel; asyncStub = RaftClientProtocolServiceGrpc.newStub(clientChannel); @@ -121,8 +124,8 @@ public class GrpcClientProtocolClient implements Closeable { RaftClientConfigKeys.Rpc.watchRequestTimeout(properties); } - private ManagedChannel buildChannel(String address, SslContext sslContext, - SizeInBytes flowControlWindow) { + private ManagedChannel buildChannel(String address, GrpcTlsConfig tlsConf, + SizeInBytes flowControlWindow, SizeInBytes maxMessageSize) { NettyChannelBuilder channelBuilder = NettyChannelBuilder.forTarget(address); // ignore any http proxy for grpc @@ -343,20 +346,13 @@ public void onCompleted() { } CompletableFuture onNext(RaftClientRequest request) { - final RaftClientRequestProto proto = ClientProtoUtils.toRaftClientRequestProto(request); - if (proto.getSerializedSize() > maxMessageSize.getSizeInt()) { - return JavaUtils.completeExceptionally(new IllegalArgumentException(getName() - + ": request serialized size " + proto.getSerializedSize() - + " exceeds maximum " + maxMessageSize + " for " + request)); - } - final long callId = request.getCallId(); final CompletableFuture f = replies.putNew(callId); if (f == null) { return JavaUtils.completeExceptionally(new AlreadyClosedException(getName() + " is closed.")); } try { - if (!requestStreamer.onNext(proto)) { + if (!requestStreamer.onNext(ClientProtoUtils.toRaftClientRequestProto(request))) { return JavaUtils.completeExceptionally(new AlreadyClosedException(getName() + ": the stream is closed.")); } } catch(Exception t) { diff --git a/ratis-grpc/src/main/java/org/apache/ratis/grpc/client/GrpcClientProtocolProxy.java b/ratis-grpc/src/main/java/org/apache/ratis/grpc/client/GrpcClientProtocolProxy.java new file mode 100644 index 0000000000..95119ef7d7 --- /dev/null +++ b/ratis-grpc/src/main/java/org/apache/ratis/grpc/client/GrpcClientProtocolProxy.java @@ -0,0 +1,108 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.ratis.grpc.client; + +import org.apache.ratis.conf.RaftProperties; +import org.apache.ratis.grpc.GrpcTlsConfig; +import org.apache.ratis.protocol.ClientId; +import org.apache.ratis.thirdparty.io.grpc.stub.StreamObserver; +import org.apache.ratis.proto.RaftProtos.RaftClientReplyProto; +import org.apache.ratis.proto.RaftProtos.RaftClientRequestProto; +import org.apache.ratis.protocol.RaftPeer; + +import java.io.Closeable; +import java.io.IOException; +import java.util.function.Function; + +public class GrpcClientProtocolProxy implements Closeable { + private final GrpcClientProtocolClient proxy; + private final Function responseHandlerCreation; + private RpcSession currentSession; + + public GrpcClientProtocolProxy(ClientId clientId, RaftPeer target, + Function responseHandlerCreation, + RaftProperties properties, GrpcTlsConfig tlsConfig) { + proxy = new GrpcClientProtocolClient(clientId, target, properties, tlsConfig, tlsConfig); + this.responseHandlerCreation = responseHandlerCreation; + } + + @Override + public void close() throws IOException { + closeCurrentSession(); + proxy.close(); + } + + @Override + public String toString() { + return "ProxyTo:" + proxy.getTarget(); + } + + public void closeCurrentSession() { + if (currentSession != null) { + currentSession.close(); + currentSession = null; + } + } + + public void onNext(RaftClientRequestProto request) { + if (currentSession == null) { + currentSession = new RpcSession( + responseHandlerCreation.apply(proxy.getTarget())); + } + currentSession.requestObserver.onNext(request); + } + + public void onError() { + if (currentSession != null) { + currentSession.onError(); + } + } + + public interface CloseableStreamObserver + extends StreamObserver, Closeable { + } + + class RpcSession implements Closeable { + private final StreamObserver requestObserver; + private final CloseableStreamObserver responseHandler; + private boolean hasError = false; + + RpcSession(CloseableStreamObserver responseHandler) { + this.responseHandler = responseHandler; + this.requestObserver = proxy.ordered(responseHandler); + } + + void onError() { + hasError = true; + } + + @Override + public void close() { + if (!hasError) { + try { + requestObserver.onCompleted(); + } catch (Exception ignored) { + } + } + try { + responseHandler.close(); + } catch (IOException ignored) { + } + } + } +} diff --git a/ratis-grpc/src/main/java/org/apache/ratis/grpc/client/GrpcClientRpc.java b/ratis-grpc/src/main/java/org/apache/ratis/grpc/client/GrpcClientRpc.java index 65175dc2a1..b825429ae4 100644 --- a/ratis-grpc/src/main/java/org/apache/ratis/grpc/client/GrpcClientRpc.java +++ b/ratis-grpc/src/main/java/org/apache/ratis/grpc/client/GrpcClientRpc.java @@ -17,16 +17,14 @@ */ package org.apache.ratis.grpc.client; -import org.apache.ratis.client.RaftClientConfigKeys; import org.apache.ratis.client.impl.ClientProtoUtils; import org.apache.ratis.client.impl.RaftClientRpcWithProxy; import org.apache.ratis.conf.RaftProperties; import org.apache.ratis.grpc.GrpcConfigKeys; +import org.apache.ratis.grpc.GrpcTlsConfig; import org.apache.ratis.grpc.GrpcUtil; import org.apache.ratis.protocol.*; import org.apache.ratis.protocol.exceptions.AlreadyClosedException; -import org.apache.ratis.protocol.exceptions.TimeoutIOException; -import org.apache.ratis.thirdparty.io.grpc.Status; import org.apache.ratis.thirdparty.io.grpc.StatusRuntimeException; import org.apache.ratis.thirdparty.io.grpc.stub.StreamObserver; import org.apache.ratis.proto.RaftProtos.GroupInfoRequestProto; @@ -38,11 +36,9 @@ import org.apache.ratis.proto.RaftProtos.TransferLeadershipRequestProto; import org.apache.ratis.proto.RaftProtos.SnapshotManagementRequestProto; import org.apache.ratis.proto.RaftProtos.LeaderElectionManagementRequestProto; -import org.apache.ratis.thirdparty.io.netty.handler.ssl.SslContext; import org.apache.ratis.util.IOUtils; import org.apache.ratis.util.JavaUtils; import org.apache.ratis.util.PeerProxyMap; -import org.apache.ratis.util.TimeDuration; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -50,25 +46,19 @@ import java.io.InterruptedIOException; import java.util.concurrent.CompletableFuture; import java.util.concurrent.ExecutionException; -import java.util.concurrent.TimeUnit; -import java.util.concurrent.TimeoutException; public class GrpcClientRpc extends RaftClientRpcWithProxy { public static final Logger LOG = LoggerFactory.getLogger(GrpcClientRpc.class); private final ClientId clientId; private final int maxMessageSize; - private final TimeDuration requestTimeoutDuration; - private final TimeDuration watchRequestTimeoutDuration; public GrpcClientRpc(ClientId clientId, RaftProperties properties, - SslContext adminSslContext, SslContext clientSslContext) { + GrpcTlsConfig adminTlsConfig, GrpcTlsConfig clientTlsConfig) { super(new PeerProxyMap<>(clientId.toString(), - p -> new GrpcClientProtocolClient(clientId, p, properties, adminSslContext, clientSslContext))); + p -> new GrpcClientProtocolClient(clientId, p, properties, adminTlsConfig, clientTlsConfig))); this.clientId = clientId; this.maxMessageSize = GrpcConfigKeys.messageSizeMax(properties, LOG::debug).getSizeInt(); - this.requestTimeoutDuration = RaftClientConfigKeys.Rpc.requestTimeout(properties); - this.watchRequestTimeoutDuration = RaftClientConfigKeys.Rpc.watchRequestTimeout(properties); } @Override @@ -131,11 +121,24 @@ public RaftClientReply sendRequest(RaftClientRequest request) ((LeaderElectionManagementRequest) request); return ClientProtoUtils.toRaftClientReply(proxy.leaderElectionManagement(proto)); } else { - return sendRequest(request, proxy); + final CompletableFuture f = sendRequest(request, proxy); + // TODO: timeout support + try { + return f.get(); + } catch (InterruptedException e) { + Thread.currentThread().interrupt(); + throw new InterruptedIOException( + "Interrupted while waiting for response of request " + request); + } catch (ExecutionException e) { + if (LOG.isTraceEnabled()) { + LOG.trace(clientId + ": failed " + request, e); + } + throw IOUtils.toIOException(e); + } } } - private RaftClientReply sendRequest( + private CompletableFuture sendRequest( RaftClientRequest request, GrpcClientProtocolClient proxy) throws IOException { final RaftClientRequestProto requestProto = toRaftClientRequestProto(request); @@ -164,44 +167,7 @@ public void onCompleted() { requestObserver.onNext(requestProto); requestObserver.onCompleted(); - final TimeDuration timeout = getTimeoutDuration(request); - try { - return replyFuture.thenApply(ClientProtoUtils::toRaftClientReply) - .get(timeout.getDuration(), timeout.getUnit()); - } catch (InterruptedException e) { - Thread.currentThread().interrupt(); - replyFuture.cancel(true); - final InterruptedIOException ioe = new InterruptedIOException(clientId + ": Interrupted " + request); - sendOnError(requestObserver, Status.CANCELLED, ioe.getMessage()); - throw ioe; - } catch (TimeoutException e) { - replyFuture.cancel(true); - final TimeoutIOException ioe = - new TimeoutIOException(clientId + ": Timed out " + timeout + " for " + request, e); - sendOnError(requestObserver, Status.DEADLINE_EXCEEDED, ioe.getMessage()); - throw ioe; - } catch (ExecutionException e) { - if (LOG.isTraceEnabled()) { - LOG.trace("{} : failed {}", clientId, request, e); - } - throw IOUtils.toIOException(e); - } - } - - private void sendOnError(StreamObserver requestObserver, Status status, String message) { - try { - requestObserver.onError(status.withDescription(message).asException()); - } catch (Exception ignored) { - // the stream already closed. - } - } - - private TimeDuration getTimeoutDuration(RaftClientRequest request) { - final long timeoutMs = request.getTimeoutMs(); - if (timeoutMs > 0) { - return TimeDuration.valueOf(timeoutMs, TimeUnit.MILLISECONDS); - } - return request.is(RaftClientRequestProto.TypeCase.WATCH) ? watchRequestTimeoutDuration : requestTimeoutDuration; + return replyFuture.thenApply(ClientProtoUtils::toRaftClientReply); } private RaftClientRequestProto toRaftClientRequestProto(RaftClientRequest request) throws IOException { diff --git a/ratis-grpc/src/main/java/org/apache/ratis/grpc/metrics/MessageMetrics.java b/ratis-grpc/src/main/java/org/apache/ratis/grpc/metrics/MessageMetrics.java index b152c67098..2a211aae80 100644 --- a/ratis-grpc/src/main/java/org/apache/ratis/grpc/metrics/MessageMetrics.java +++ b/ratis-grpc/src/main/java/org/apache/ratis/grpc/metrics/MessageMetrics.java @@ -61,6 +61,14 @@ private void inc(String metricNamePrefix, Type t) { types.get(t) .computeIfAbsent(metricNamePrefix, prefix -> getRegistry().counter(prefix + t.getSuffix())) .inc(); + final Map counters = types.get(t); + LongCounter c = counters.get(metricNamePrefix); + if (c == null) { + synchronized (counters) { + c = counters.computeIfAbsent(metricNamePrefix, prefix -> getRegistry().counter(prefix + t.getSuffix())); + } + } + c.inc(); } /** diff --git a/ratis-grpc/src/main/java/org/apache/ratis/grpc/server/GrpcLogAppender.java b/ratis-grpc/src/main/java/org/apache/ratis/grpc/server/GrpcLogAppender.java index b3d43808c1..c016dabfe4 100644 --- a/ratis-grpc/src/main/java/org/apache/ratis/grpc/server/GrpcLogAppender.java +++ b/ratis-grpc/src/main/java/org/apache/ratis/grpc/server/GrpcLogAppender.java @@ -24,6 +24,7 @@ import org.apache.ratis.metrics.Timekeeper; import org.apache.ratis.proto.RaftProtos.InstallSnapshotResult; import org.apache.ratis.protocol.RaftPeerId; +import org.apache.ratis.retry.MultipleLinearRandomRetry; import org.apache.ratis.retry.RetryPolicy; import org.apache.ratis.server.RaftServer; import org.apache.ratis.server.RaftServerConfigKeys; @@ -63,8 +64,6 @@ import java.util.concurrent.TimeUnit; import java.util.concurrent.atomic.AtomicLong; -import static org.apache.ratis.server.raftlog.LogProtoUtils.toLogEntryTermIndexString; - /** * A new log appender implementation using grpc bi-directional stream API. */ @@ -195,9 +194,8 @@ public GrpcLogAppender(RaftServer.Division server, LeaderState leaderState, Foll lock = new AutoCloseableReadWriteLock(this); caller = LOG.isTraceEnabled()? JavaUtils.getCallerStackTraceElement(): null; - errorRetryWaitPolicy = RetryPolicy.parse( - RaftServerConfigKeys.Log.Appender.retryPolicy(properties), - RaftServerConfigKeys.Log.Appender.RETRY_POLICY_KEY); + errorRetryWaitPolicy = MultipleLinearRandomRetry.parseCommaSeparated( + RaftServerConfigKeys.Log.Appender.retryPolicy(properties)); } @Override @@ -236,7 +234,7 @@ private void resetClient(AppendEntriesRequest request, Event event) { } getFollower().computeNextIndex(getNextIndexForError(nextIndex)); } catch (IOException ie) { - LOG.warn("{}: Failed to resetClient for {}", this, getFollowerId(), ie); + LOG.warn(this + ": Failed to getClient for " + getFollowerId(), ie); } } @@ -299,8 +297,8 @@ private void mayWait() { getEventAwaitForSignal().await(getWaitTimeMs() + errorWaitTimeMs(), TimeUnit.MILLISECONDS); } catch (InterruptedException ie) { + LOG.warn(this + ": Wait interrupted by " + ie); Thread.currentThread().interrupt(); - LOG.warn("{} is interrupted: {}", this, ie.toString()); } } @@ -517,8 +515,8 @@ public void onNext(AppendEntriesReplyProto reply) { try { onNextImpl(request, reply); } catch(Exception t) { - LOG.error("Failed onNext(reply), request={}, reply={}", - request, ServerStringUtils.toAppendEntriesReplyString(reply), t); + LOG.error("Failed onNext request=" + request + + ", reply=" + ServerStringUtils.toAppendEntriesReplyString(reply), t); } } @@ -593,8 +591,8 @@ private void updateNextIndex(long replyNextIndex) { } private class InstallSnapshotResponseHandler implements StreamObserver { - private final String name; - private final Queue pending = new LinkedList<>(); + private final String name = getFollower().getName() + "-" + JavaUtils.getClassSimpleName(getClass()); + private final Queue pending; private final CompletableFuture done = new CompletableFuture<>(); private final boolean isNotificationOnly; @@ -603,8 +601,8 @@ private class InstallSnapshotResponseHandler implements StreamObserver(); this.isNotificationOnly = notifyOnly; - this.name = getFollower().getName() + "-InstallSnapshot" + (isNotificationOnly ? "Notification" : ""); } void addPending(InstallSnapshotRequestProto request) { @@ -632,11 +630,11 @@ void removePending(InstallSnapshotReplyProto reply) { if (isNotificationOnly) { Preconditions.assertSame(InstallSnapshotReplyBodyCase.SNAPSHOTINDEX, reply.getInstallSnapshotReplyBodyCase(), "reply case"); - Preconditions.assertSame(INSTALL_SNAPSHOT_NOTIFICATION_INDEX, index, "poll index"); + Preconditions.assertSame(INSTALL_SNAPSHOT_NOTIFICATION_INDEX, (int) index, "poll index"); } else { Preconditions.assertSame(InstallSnapshotReplyBodyCase.REQUESTINDEX, reply.getInstallSnapshotReplyBodyCase(), "reply case"); - Preconditions.assertSame(reply.getRequestIndex(), index, "poll index"); + Preconditions.assertSame(reply.getRequestIndex(), (int) index, "poll index"); } } } @@ -646,8 +644,8 @@ void onFollowerCatchup(long followerSnapshotIndex) { final long leaderStartIndex = getRaftLog().getStartIndex(); final long followerNextIndex = followerSnapshotIndex + 1; if (followerNextIndex >= leaderStartIndex) { - LOG.info("{}: follower nextIndex = {} >= leader startIndex = {}", - this, followerNextIndex, leaderStartIndex); + LOG.info("{}: Follower can catch up leader after install the snapshot, as leader's start index is {}", + this, followerNextIndex); notifyInstallSnapshotFinished(InstallSnapshotResult.SUCCESS, followerSnapshotIndex); } } @@ -679,10 +677,10 @@ boolean hasAllResponse() { @Override public void onNext(InstallSnapshotReplyProto reply) { - BatchLogger.print(BatchLogKey.INSTALL_SNAPSHOT_REPLY, name, - suffix -> LOG.info("{}: received {} reply {} {}", this, - replyState.isFirstReplyReceived() ? "a" : "the first", - ServerStringUtils.toInstallSnapshotReplyString(reply), suffix)); + if (LOG.isInfoEnabled()) { + LOG.info("{}: received {} reply {}", this, replyState.isFirstReplyReceived()? "a" : "the first", + ServerStringUtils.toInstallSnapshotReplyString(reply)); + } // update the last rpc time getFollower().updateLastRpcResponseTime(); @@ -691,13 +689,12 @@ public void onNext(InstallSnapshotReplyProto reply) { final long followerSnapshotIndex; switch (reply.getResult()) { case SUCCESS: - LOG.info("{}: Completed", this); + LOG.info("{}: Completed InstallSnapshot. Reply: {}", this, reply); getFollower().setAttemptedToInstallSnapshot(); removePending(reply); break; case IN_PROGRESS: - BatchLogger.print(BatchLogKey.INSTALL_SNAPSHOT_IN_PROGRESS, name, - suffix -> LOG.info("{}: in progress, {}", this, suffix)); + LOG.info("{}: InstallSnapshot in progress.", this); removePending(reply); break; case ALREADY_INSTALLED: @@ -713,7 +710,7 @@ public void onNext(InstallSnapshotReplyProto reply) { onFollowerTerm(reply.getTerm()); break; case CONF_MISMATCH: - LOG.error("{}: CONF_MISMATCH ({}): Leader {} has it set to {} but follower {} has it set to {}", + LOG.error("{}: Configuration Mismatch ({}): Leader {} has it set to {} but follower {} has it set to {}", this, RaftServerConfigKeys.Log.Appender.INSTALL_SNAPSHOT_ENABLED_KEY, getServer().getId(), installSnapshotEnabled, getFollowerId(), !installSnapshotEnabled); break; @@ -728,19 +725,17 @@ public void onNext(InstallSnapshotReplyProto reply) { removePending(reply); break; case SNAPSHOT_UNAVAILABLE: - BatchLogger.print(BatchLogKey.SNAPSHOT_UNAVAILABLE, name, - suffix -> LOG.info("{}: Follower failed since the snapshot is unavailable {}", this, suffix)); + LOG.info("{}: Follower could not install snapshot as it is not available.", this); getFollower().setAttemptedToInstallSnapshot(); notifyInstallSnapshotFinished(InstallSnapshotResult.SNAPSHOT_UNAVAILABLE, RaftLog.INVALID_LOG_INDEX); removePending(reply); break; case UNRECOGNIZED: - LOG.error("{}: Reply result {}, {}", - name, reply.getResult(), ServerStringUtils.toInstallSnapshotReplyString(reply)); + LOG.error("Unrecognized the reply result {}: Leader is {}, follower is {}", + reply.getResult(), getServer().getId(), getFollowerId()); break; case SNAPSHOT_EXPIRED: - LOG.warn("{}: Follower failed since the request expired, {}", - name, ServerStringUtils.toInstallSnapshotReplyString(reply)); + LOG.warn("{}: Follower could not install snapshot as it is expired.", this); default: break; } @@ -819,9 +814,8 @@ private void installSnapshot(SnapshotInfo snapshot) { * @param firstAvailable the first available log's index on the Leader */ private void notifyInstallSnapshot(TermIndex firstAvailable) { - BatchLogger.print(BatchLogKey.INSTALL_SNAPSHOT_NOTIFY, getFollower().getName(), - suffix -> LOG.info("{}: notifyInstallSnapshot with firstAvailable={}, followerNextIndex={} {}", - this, firstAvailable, getFollower().getNextIndex(), suffix)); + LOG.info("{}: notifyInstallSnapshot with firstAvailable={}, followerNextIndex={}", + this, firstAvailable, getFollower().getNextIndex()); final InstallSnapshotResponseHandler responseHandler = new InstallSnapshotResponseHandler(true); StreamObserver snapshotRequestObserver = null; @@ -848,6 +842,44 @@ private void notifyInstallSnapshot(TermIndex firstAvailable) { responseHandler.waitForResponse(); } + /** + * Should the Leader notify the Follower to install the snapshot through + * its own State Machine. + * @return the first available log's start term index + */ + private TermIndex shouldNotifyToInstallSnapshot() { + final FollowerInfo follower = getFollower(); + final long leaderNextIndex = getRaftLog().getNextIndex(); + final boolean isFollowerBootstrapping = getLeaderState().isFollowerBootstrapping(follower); + final long leaderStartIndex = getRaftLog().getStartIndex(); + final TermIndex firstAvailable = Optional.ofNullable(getRaftLog().getTermIndex(leaderStartIndex)) + .orElseGet(() -> TermIndex.valueOf(getServer().getInfo().getCurrentTerm(), leaderNextIndex)); + if (isFollowerBootstrapping && !follower.hasAttemptedToInstallSnapshot()) { + // If the follower is bootstrapping and has not yet installed any snapshot from leader, then the follower should + // be notified to install a snapshot. Every follower should try to install at least one snapshot during + // bootstrapping, if available. + LOG.debug("{}: follower is bootstrapping, notify to install snapshot to {}.", this, firstAvailable); + return firstAvailable; + } + + final long followerNextIndex = follower.getNextIndex(); + if (followerNextIndex >= leaderNextIndex) { + return null; + } + + if (followerNextIndex < leaderStartIndex) { + // The Leader does not have the logs from the Follower's last log + // index onwards. And install snapshot is disabled. So the Follower + // should be notified to install the latest snapshot through its + // State Machine. + return firstAvailable; + } else if (leaderStartIndex == RaftLog.INVALID_LOG_INDEX) { + // Leader has no logs to check from, hence return next index. + return firstAvailable; + } + + return null; + } static class AppendEntriesRequest { private final Timekeeper timer; @@ -905,9 +937,13 @@ boolean isHeartbeat() { @Override public String toString() { + final String entries = entriesCount == 0? "" + : entriesCount == 1? ",entry=" + firstEntry + : ",entries=" + firstEntry + "..." + lastEntry; return JavaUtils.getClassSimpleName(getClass()) + ":cid=" + callId - + ":" + toLogEntryTermIndexString(entriesCount, firstEntry, lastEntry); + + ",entriesCount=" + entriesCount + + entries; } } diff --git a/ratis-grpc/src/main/java/org/apache/ratis/grpc/server/GrpcServerProtocolClient.java b/ratis-grpc/src/main/java/org/apache/ratis/grpc/server/GrpcServerProtocolClient.java index d2748c7be2..4a280ab335 100644 --- a/ratis-grpc/src/main/java/org/apache/ratis/grpc/server/GrpcServerProtocolClient.java +++ b/ratis-grpc/src/main/java/org/apache/ratis/grpc/server/GrpcServerProtocolClient.java @@ -1,4 +1,4 @@ -/* +/** * 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 @@ -17,11 +17,13 @@ */ package org.apache.ratis.grpc.server; +import org.apache.ratis.grpc.GrpcTlsConfig; import org.apache.ratis.grpc.GrpcUtil; import org.apache.ratis.grpc.util.StreamObserverWithTimeout; import org.apache.ratis.protocol.RaftPeerId; import org.apache.ratis.server.util.ServerStringUtils; import org.apache.ratis.thirdparty.io.grpc.ManagedChannel; +import org.apache.ratis.thirdparty.io.grpc.netty.GrpcSslContexts; import org.apache.ratis.thirdparty.io.grpc.netty.NegotiationType; import org.apache.ratis.thirdparty.io.grpc.netty.NettyChannelBuilder; import org.apache.ratis.thirdparty.io.grpc.stub.CallStreamObserver; @@ -31,7 +33,7 @@ import org.apache.ratis.proto.grpc.RaftServerProtocolServiceGrpc.RaftServerProtocolServiceBlockingStub; import org.apache.ratis.proto.grpc.RaftServerProtocolServiceGrpc.RaftServerProtocolServiceStub; import org.apache.ratis.protocol.RaftPeer; -import org.apache.ratis.thirdparty.io.netty.handler.ssl.SslContext; +import org.apache.ratis.thirdparty.io.netty.handler.ssl.SslContextBuilder; import org.apache.ratis.util.TimeDuration; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -42,10 +44,9 @@ * This is a RaftClient implementation that supports streaming data to the raft * ring. The stream implementation utilizes gRPC. */ -class GrpcServerProtocolClient implements Closeable { +public class GrpcServerProtocolClient implements Closeable { // Common channel private final ManagedChannel channel; - private final GrpcStubPool pool; // Channel and stub for heartbeat private ManagedChannel hbChannel; private RaftServerProtocolServiceStub hbAsyncStub; @@ -58,34 +59,40 @@ class GrpcServerProtocolClient implements Closeable { //visible for using in log / error messages AND to use in instrumented tests private final RaftPeerId raftPeerId; - GrpcServerProtocolClient(RaftPeer target, int connections, int flowControlWindow, - TimeDuration requestTimeout, SslContext sslContext, boolean separateHBChannel) { + public GrpcServerProtocolClient(RaftPeer target, int flowControlWindow, + TimeDuration requestTimeout, GrpcTlsConfig tlsConfig, boolean separateHBChannel) { raftPeerId = target.getId(); LOG.info("Build channel for {}", target); useSeparateHBChannel = separateHBChannel; - channel = buildChannel(target, flowControlWindow, sslContext); + channel = buildChannel(target, flowControlWindow, tlsConfig); blockingStub = RaftServerProtocolServiceGrpc.newBlockingStub(channel); asyncStub = RaftServerProtocolServiceGrpc.newStub(channel); if (useSeparateHBChannel) { - hbChannel = buildChannel(target, flowControlWindow, sslContext); + hbChannel = buildChannel(target, flowControlWindow, tlsConfig); hbAsyncStub = RaftServerProtocolServiceGrpc.newStub(hbChannel); } requestTimeoutDuration = requestTimeout; - this.pool = connections == 1? null : newGrpcStubPool(target.getAddress(), sslContext, connections); - } - - GrpcStubPool newGrpcStubPool(String address, SslContext sslContext, int connections) { - return new GrpcStubPool<>(connections, address, sslContext, RaftServerProtocolServiceGrpc::newStub, 16); } - private ManagedChannel buildChannel(RaftPeer target, int flowControlWindow, SslContext sslContext) { + private ManagedChannel buildChannel(RaftPeer target, int flowControlWindow, + GrpcTlsConfig tlsConfig) { NettyChannelBuilder channelBuilder = NettyChannelBuilder.forTarget(target.getAddress()); // ignore any http proxy for grpc channelBuilder.proxyDetector(uri -> null); - if (sslContext != null) { - channelBuilder.useTransportSecurity().sslContext(sslContext); + if (tlsConfig!= null) { + SslContextBuilder sslContextBuilder = GrpcSslContexts.forClient(); + GrpcUtil.setTrustManager(sslContextBuilder, tlsConfig.getTrustManager()); + if (tlsConfig.getMtlsEnabled()) { + GrpcUtil.setKeyManager(sslContextBuilder, tlsConfig.getKeyManager()); + } + try { + channelBuilder.useTransportSecurity().sslContext(sslContextBuilder.build()); + } catch (Exception ex) { + throw new IllegalArgumentException("Failed to build SslContext, peerId=" + raftPeerId + + ", tlsConfig=" + tlsConfig, ex); + } } else { channelBuilder.negotiationType(NegotiationType.PLAINTEXT); } @@ -100,9 +107,6 @@ public void close() { GrpcUtil.shutdownManagedChannel(hbChannel); } GrpcUtil.shutdownManagedChannel(channel); - if (pool != null) { - pool.close(); - } } public RequestVoteReplyProto requestVote(RequestVoteRequestProto request) { @@ -121,44 +125,8 @@ public StartLeaderElectionReplyProto startLeaderElection(StartLeaderElectionRequ } void readIndex(ReadIndexRequestProto request, StreamObserver s) { - if (pool == null) { - asyncStub.withDeadlineAfter(requestTimeoutDuration.getDuration(), requestTimeoutDuration.getUnit()) - .readIndex(request, s); - } else { - GrpcStubPool.Stub p; - try { - p = pool.acquire(); - } catch (InterruptedException e) { - Thread.currentThread().interrupt(); - s.onError(e); - return; - } - p.getStub().withDeadlineAfter(requestTimeoutDuration.getDuration(), requestTimeoutDuration.getUnit()) - .readIndex(request, new StreamObserver() { - @Override - public void onNext(ReadIndexReplyProto v) { - s.onNext(v); - } - - @Override - public void onError(Throwable t) { - try { - s.onError(t); - } finally { - p.release(); - } - } - - @Override - public void onCompleted() { - try { - s.onCompleted(); - } finally { - p.release(); - } - } - }); - } + asyncStub.withDeadlineAfter(requestTimeoutDuration.getDuration(), requestTimeoutDuration.getUnit()) + .readIndex(request, s); } CallStreamObserver appendEntries( diff --git a/ratis-grpc/src/main/java/org/apache/ratis/grpc/server/GrpcServerProtocolService.java b/ratis-grpc/src/main/java/org/apache/ratis/grpc/server/GrpcServerProtocolService.java index 4b5097fb6a..7e17cb3cf4 100644 --- a/ratis-grpc/src/main/java/org/apache/ratis/grpc/server/GrpcServerProtocolService.java +++ b/ratis-grpc/src/main/java/org/apache/ratis/grpc/server/GrpcServerProtocolService.java @@ -81,8 +81,7 @@ void release() { } } - abstract class ServerRequestStreamObserver - implements StreamObserver { + abstract class ServerRequestStreamObserver implements StreamObserver { private final RaftServer.Op op; private final Supplier nameSupplier; private final StreamObserver responseObserver; diff --git a/ratis-grpc/src/main/java/org/apache/ratis/grpc/server/GrpcServicesImpl.java b/ratis-grpc/src/main/java/org/apache/ratis/grpc/server/GrpcServicesImpl.java index f864e31308..d6f6a0c866 100644 --- a/ratis-grpc/src/main/java/org/apache/ratis/grpc/server/GrpcServicesImpl.java +++ b/ratis-grpc/src/main/java/org/apache/ratis/grpc/server/GrpcServicesImpl.java @@ -72,8 +72,7 @@ public final class GrpcServicesImpl class AsyncService implements RaftServerAsynchronousProtocol { @Override - public CompletableFuture appendEntriesAsync(AppendEntriesRequestProto request) - throws IOException { + public CompletableFuture appendEntriesAsync(AppendEntriesRequestProto request) { throw new UnsupportedOperationException("This method is not supported"); } @@ -257,11 +256,6 @@ public GrpcServicesImpl build() { return new GrpcServicesImpl(this); } - public Builder setTlsConfig(GrpcTlsConfig tlsConfig) { - this.tlsConfig = tlsConfig; - return this; - } - public Builder setAdminTlsConfig(GrpcTlsConfig config) { this.adminTlsConfig = config; return this; @@ -276,10 +270,6 @@ public Builder setServerTlsConfig(GrpcTlsConfig config) { this.serverTlsConfig = config; return this; } - - public GrpcTlsConfig getTlsConfig() { - return tlsConfig; - } } public static Builder newBuilder() { diff --git a/ratis-metrics-api/pom.xml b/ratis-metrics-api/pom.xml index 6731d06602..db368269e2 100644 --- a/ratis-metrics-api/pom.xml +++ b/ratis-metrics-api/pom.xml @@ -18,7 +18,7 @@ ratis org.apache.ratis - 3.2.0-SNAPSHOT + 3.3.0-SNAPSHOT ratis-metrics-api diff --git a/ratis-metrics-default/pom.xml b/ratis-metrics-default/pom.xml index bcd6020915..b561cbe924 100644 --- a/ratis-metrics-default/pom.xml +++ b/ratis-metrics-default/pom.xml @@ -18,7 +18,7 @@ ratis org.apache.ratis - 3.2.0-SNAPSHOT + 3.3.0-SNAPSHOT ratis-metrics-default diff --git a/ratis-metrics-dropwizard3/pom.xml b/ratis-metrics-dropwizard3/pom.xml index c77bf2a290..2dd329858e 100644 --- a/ratis-metrics-dropwizard3/pom.xml +++ b/ratis-metrics-dropwizard3/pom.xml @@ -18,7 +18,7 @@ ratis org.apache.ratis - 3.2.0-SNAPSHOT + 3.3.0-SNAPSHOT ratis-metrics-dropwizard3 diff --git a/ratis-netty/pom.xml b/ratis-netty/pom.xml index cb7430451c..5688a1fa4a 100644 --- a/ratis-netty/pom.xml +++ b/ratis-netty/pom.xml @@ -17,7 +17,7 @@ ratis org.apache.ratis - 3.2.0-SNAPSHOT + 3.3.0-SNAPSHOT ratis-netty diff --git a/ratis-proto/pom.xml b/ratis-proto/pom.xml index 611e3861f0..322bf56778 100644 --- a/ratis-proto/pom.xml +++ b/ratis-proto/pom.xml @@ -17,7 +17,7 @@ ratis org.apache.ratis - 3.2.0-SNAPSHOT + 3.3.0-SNAPSHOT ratis-proto @@ -171,8 +171,14 @@ ratis-thirdparty-misc - jakarta.annotation - jakarta.annotation-api + org.junit.jupiter + junit-jupiter-engine + test + + + org.junit.platform + junit-platform-launcher + test diff --git a/ratis-resource-bundle/pom.xml b/ratis-resource-bundle/pom.xml index c000677c1e..edecf1b633 100644 --- a/ratis-resource-bundle/pom.xml +++ b/ratis-resource-bundle/pom.xml @@ -23,7 +23,7 @@ ratis org.apache.ratis - 3.2.0-SNAPSHOT + 3.3.0-SNAPSHOT .. diff --git a/ratis-server-api/pom.xml b/ratis-server-api/pom.xml index 36e77fa9a6..feed49190c 100644 --- a/ratis-server-api/pom.xml +++ b/ratis-server-api/pom.xml @@ -17,7 +17,7 @@ ratis org.apache.ratis - 3.2.0-SNAPSHOT + 3.3.0-SNAPSHOT ratis-server-api diff --git a/ratis-server-api/src/main/java/org/apache/ratis/server/leader/LogAppender.java b/ratis-server-api/src/main/java/org/apache/ratis/server/leader/LogAppender.java index dc189a14aa..a333b8393a 100644 --- a/ratis-server-api/src/main/java/org/apache/ratis/server/leader/LogAppender.java +++ b/ratis-server-api/src/main/java/org/apache/ratis/server/leader/LogAppender.java @@ -136,89 +136,38 @@ default RaftPeerId getFollowerId() { /** @return an {@link Iterable} of {@link InstallSnapshotRequestProto} for sending the given snapshot. */ Iterable newInstallSnapshotRequests(String requestId, SnapshotInfo snapshot); - /** - * Get the previous {@link TermIndex} for the given next index. - * This is used to set the previous log entry in AppendEntries requests. - * - * @return the previous {@link TermIndex}, or null if unavailable - * (e.g. the entry has been purged and the snapshot does not cover it). - */ - default TermIndex getPrevious(long nextIndex) { - if (nextIndex == RaftLog.LEAST_VALID_LOG_INDEX) { - return null; - } - - final long previousIndex = nextIndex - 1; - final TermIndex previous = getRaftLog().getTermIndex(previousIndex); - if (previous != null) { - return previous; - } - - final SnapshotInfo snapshot = getServer().getStateMachine().getLatestSnapshot(); - if (snapshot != null) { - final TermIndex snapshotTermIndex = snapshot.getTermIndex(); - if (snapshotTermIndex.getIndex() == previousIndex) { - return snapshotTermIndex; - } - } - - return null; - } - /** * Should this {@link LogAppender} send a snapshot to the follower? * * @return the snapshot if it should install a snapshot; otherwise, return null. */ default SnapshotInfo shouldInstallSnapshot() { + // we should install snapshot if the follower needs to catch up and: + // 1. there is no local log entry but there is snapshot + // 2. or the follower's next index is smaller than the log start index + // 3. or the follower is bootstrapping (i.e. not yet caught up) and has not installed any snapshot yet + final FollowerInfo follower = getFollower(); + final boolean isFollowerBootstrapping = getLeaderState().isFollowerBootstrapping(follower); final SnapshotInfo snapshot = getServer().getStateMachine().getLatestSnapshot(); - return shouldInstallSnapshot(snapshot != null) ? snapshot : null; - } - - /** - * Should this {@link LogAppender} send a snapshot notification to the follower? - * - * @return the first available log {@link TermIndex} if it should install a snapshot; otherwise, return null. - */ - default TermIndex shouldNotifyToInstallSnapshot() { - if (!shouldInstallSnapshot(true)) { - return null; - } - final TermIndex start = getRaftLog().getTermIndex(getRaftLog().getStartIndex()); - if (start != null) { - return start; - } - // No log is currently available; return the next, which will become available in the future. - return TermIndex.valueOf(getServer().getInfo().getCurrentTerm(), getRaftLog().getNextIndex()); - } - default boolean shouldInstallSnapshot(boolean hasSnapshot) { - final FollowerInfo follower = getFollower(); - if (getLeaderState().isFollowerBootstrapping(follower) - && !follower.hasAttemptedToInstallSnapshot()) { - if (!hasSnapshot) { + if (isFollowerBootstrapping && !follower.hasAttemptedToInstallSnapshot()) { + if (snapshot == null) { // Leader cannot send null snapshot to follower. Hence, acknowledge InstallSnapshot attempt (even though it // was not attempted) so that follower can come out of staging state after appending log entries. follower.setAttemptedToInstallSnapshot(); + } else { + return snapshot; } - return true; } - final long leaderNextIndex = getRaftLog().getNextIndex(); final long followerNextIndex = getFollower().getNextIndex(); - if (followerNextIndex >= leaderNextIndex) { - // follower caught up already - return false; - } - final long leaderStartIndex = getRaftLog().getStartIndex(); - if (followerNextIndex < leaderStartIndex || leaderStartIndex == RaftLog.INVALID_LOG_INDEX) { - // leader does not have follower's next log - return true; + if (followerNextIndex < getRaftLog().getNextIndex()) { + final long logStartIndex = getRaftLog().getStartIndex(); + if (followerNextIndex < logStartIndex || (logStartIndex == RaftLog.INVALID_LOG_INDEX && snapshot != null)) { + return snapshot; + } } - // leader does not have the previous log for appendEntries - return followerNextIndex == leaderStartIndex && - followerNextIndex > RaftLog.LEAST_VALID_LOG_INDEX && - getPrevious(followerNextIndex) == null; + return null; } /** Define how this {@link LogAppender} should run. */ diff --git a/ratis-server-api/src/main/java/org/apache/ratis/server/protocol/TermIndex.java b/ratis-server-api/src/main/java/org/apache/ratis/server/protocol/TermIndex.java index 745f366d13..369aefc85f 100644 --- a/ratis-server-api/src/main/java/org/apache/ratis/server/protocol/TermIndex.java +++ b/ratis-server-api/src/main/java/org/apache/ratis/server/protocol/TermIndex.java @@ -21,6 +21,7 @@ import org.apache.ratis.proto.RaftProtos.TermIndexProto; import org.apache.ratis.server.raftlog.RaftLog; import org.apache.ratis.util.BiWeakValueCache; +import org.apache.ratis.util.MemoizedSupplier; import java.util.Comparator; import java.util.Optional; @@ -95,6 +96,8 @@ static BiWeakValueCache getCache() { private static TermIndex newTermIndex(long term, long index) { return new TermIndex() { + private final Supplier protoSupplier = MemoizedSupplier.valueOf(TermIndex.super::toProto); + @Override public long getTerm() { return term; @@ -123,12 +126,22 @@ public int hashCode() { return Long.hashCode(term) ^ Long.hashCode(index); } + @Override + public TermIndexProto toProto() { + return protoSupplier.get(); + } + private String longToString(long n) { return n >= 0L ? String.valueOf(n) : "~"; } @Override public String toString() { + if (this.equals(INITIAL_VALUE)) { + return ""; + } else if (this.equals(PROTO_DEFAULT)) { + return ""; + } return String.format("(t:%s, i:%s)", longToString(term), longToString(index)); } }; diff --git a/ratis-server-api/src/main/java/org/apache/ratis/server/raftlog/RaftLog.java b/ratis-server-api/src/main/java/org/apache/ratis/server/raftlog/RaftLog.java index 07446282e7..ca785a4a6d 100644 --- a/ratis-server-api/src/main/java/org/apache/ratis/server/raftlog/RaftLog.java +++ b/ratis-server-api/src/main/java/org/apache/ratis/server/raftlog/RaftLog.java @@ -44,9 +44,6 @@ public interface RaftLog extends RaftLogSequentialOps, Closeable { /** Invalid log index is used to indicate that the log index is missing. */ long INVALID_LOG_INDEX = LEAST_VALID_LOG_INDEX - 1; - /** Is this log already opened but not yet closed? */ - boolean isOpened(); - /** Does this log contains the given {@link TermIndex}? */ default boolean contains(TermIndex ti) { Objects.requireNonNull(ti, "ti == null"); diff --git a/ratis-server-api/src/main/java/org/apache/ratis/statemachine/TransactionContext.java b/ratis-server-api/src/main/java/org/apache/ratis/statemachine/TransactionContext.java index 1a33bcaa59..d6ff9a0833 100644 --- a/ratis-server-api/src/main/java/org/apache/ratis/statemachine/TransactionContext.java +++ b/ratis-server-api/src/main/java/org/apache/ratis/statemachine/TransactionContext.java @@ -23,6 +23,7 @@ import org.apache.ratis.protocol.RaftClientRequest; import org.apache.ratis.thirdparty.com.google.protobuf.ByteString; import org.apache.ratis.util.Preconditions; +import org.apache.ratis.util.ReferenceCountedObject; import org.apache.ratis.util.ReflectionUtils; import java.io.IOException; diff --git a/ratis-server/pom.xml b/ratis-server/pom.xml index c8f55eff73..2c0bc93a25 100644 --- a/ratis-server/pom.xml +++ b/ratis-server/pom.xml @@ -17,7 +17,7 @@ ratis org.apache.ratis - 3.2.0-SNAPSHOT + 3.3.0-SNAPSHOT ratis-server diff --git a/ratis-server/src/main/java/org/apache/ratis/server/impl/ConfigurationManager.java b/ratis-server/src/main/java/org/apache/ratis/server/impl/ConfigurationManager.java index 5cf3535a61..2ba8107837 100644 --- a/ratis-server/src/main/java/org/apache/ratis/server/impl/ConfigurationManager.java +++ b/ratis-server/src/main/java/org/apache/ratis/server/impl/ConfigurationManager.java @@ -44,11 +44,9 @@ public class ConfigurationManager { * The current raft configuration. If configurations is not empty, should be * the last entry of the map. Otherwise is initialConf. */ - @SuppressWarnings({"squid:S3077"}) // Suppress volatile for generic type - private volatile RaftConfigurationImpl currentConf; + private RaftConfigurationImpl currentConf; /** Cache the peer corresponding to {@link #id}. */ - @SuppressWarnings({"squid:S3077"}) // Suppress volatile for generic type - private volatile RaftPeer currentPeer; + private RaftPeer currentPeer; ConfigurationManager(RaftPeerId id, RaftConfigurationImpl initialConf) { this.id = id; diff --git a/ratis-server/src/main/java/org/apache/ratis/server/impl/FollowerState.java b/ratis-server/src/main/java/org/apache/ratis/server/impl/FollowerState.java index 7a8414ca2d..b01abcddc0 100644 --- a/ratis-server/src/main/java/org/apache/ratis/server/impl/FollowerState.java +++ b/ratis-server/src/main/java/org/apache/ratis/server/impl/FollowerState.java @@ -158,8 +158,8 @@ private void runImpl() { } synchronized (server) { if (roleChangeChecking(electionTimeout)) { - LOG.info("{}: change to CANDIDATE, lastRpcElapsedTime:{}ms, electionTimeout:{}", - this, lastRpcTime.elapsedTimeMs(), electionTimeout); + LOG.info("{}: change to CANDIDATE, lastRpcElapsedTime:{}, electionTimeout:{}", + this, lastRpcTime.elapsedTime(), electionTimeout); server.getLeaderElectionMetrics().onLeaderElectionTimeout(); // Update timeout metric counters. // election timeout, should become a candidate server.changeToCandidate(false); diff --git a/ratis-server/src/main/java/org/apache/ratis/server/impl/LeaderElection.java b/ratis-server/src/main/java/org/apache/ratis/server/impl/LeaderElection.java index e6c3ce5f0b..6ee55b59cf 100644 --- a/ratis-server/src/main/java/org/apache/ratis/server/impl/LeaderElection.java +++ b/ratis-server/src/main/java/org/apache/ratis/server/impl/LeaderElection.java @@ -260,7 +260,7 @@ static class Executor { } void shutdown() { - executor.shutdown(); + executor.shutdownNow(); } void submit(Callable task) { @@ -370,6 +370,7 @@ public void run() { } } + @SuppressWarnings("try") private void runImpl() { if (!lifeCycle.compareAndTransition(STARTING, RUNNING)) { final LifeCycle.State state = lifeCycle.getCurrentState(); @@ -425,7 +426,7 @@ private ResultAndTerm submitRequestAndWaitResult(Phase phase, RaftConfigurationI if (others.isEmpty()) { r = new ResultAndTerm(Result.PASSED, electionTerm); } else { - final TermIndex lastEntry = server.getState().getLastEntry(); + final TermIndex lastEntry = server.getLastEntry(); final Executor voteExecutor = new Executor(this, others.size()); try { final int submitted = submitRequests(phase, electionTerm, lastEntry, others, voteExecutor); @@ -488,7 +489,7 @@ private int submitRequests(Phase phase, long electionTerm, TermIndex lastEntry, for (final RaftPeer peer : others) { final RequestVoteRequestProto r = ServerProtoUtils.toRequestVoteRequestProto( server.getMemberId(), peer.getId(), electionTerm, lastEntry, phase == Phase.PRE_VOTE); - voteExecutor.submit(() -> server.getServerRpc().requestVote(r)); + voteExecutor.submit(() -> server.requestVote(r)); submitted++; } return submitted; @@ -514,6 +515,9 @@ private ResultAndTerm waitForResults(Phase phase, long electionTerm, int submitt Set higherPriorityPeers = getHigherPriorityPeers(conf); final boolean singleMode = conf.isSingleMode(server.getId()); + // true iff this server does not have any commits + final boolean emptyCommit = server.getLastCommittedIndex() < RaftLog.LEAST_VALID_LOG_INDEX; + while (waitForNum > 0 && shouldRun(electionTerm)) { final TimeDuration waitTime = timeout.elapsedTime().apply(n -> -n); if (waitTime.isNonPositive()) { diff --git a/ratis-server/src/main/java/org/apache/ratis/server/impl/LeaderStateImpl.java b/ratis-server/src/main/java/org/apache/ratis/server/impl/LeaderStateImpl.java index 622db7bf69..dab660fc05 100644 --- a/ratis-server/src/main/java/org/apache/ratis/server/impl/LeaderStateImpl.java +++ b/ratis-server/src/main/java/org/apache/ratis/server/impl/LeaderStateImpl.java @@ -39,7 +39,6 @@ import org.apache.ratis.protocol.exceptions.ReadIndexException; import org.apache.ratis.protocol.exceptions.ReconfigurationTimeoutException; import org.apache.ratis.server.RaftServerConfigKeys; -import org.apache.ratis.server.RaftServerConfigKeys.Read.ReadIndex.Type; import org.apache.ratis.server.impl.ReadIndexHeartbeats.AppendEntriesListener; import org.apache.ratis.server.leader.FollowerInfo; import org.apache.ratis.server.leader.LeaderState; @@ -737,7 +736,8 @@ void submitStepDownEvent(long term, StepDownReason reason) { private void stepDown(long term, StepDownReason reason) { try { lease.getAndSetEnabled(false); - server.changeToFollowerAndPersistMetadata(term, false, reason).join(); + server.changeToFollowerAndPersistMetadata(term, false, reason) + .get(5, TimeUnit.SECONDS); pendingStepDown.complete(server::newSuccessReply); } catch(Exception e) { pendingStepDown.completeExceptionally(e); @@ -1242,22 +1242,8 @@ private boolean checkLeaderLease() { && (server.getRaftConf().isSingleton() || lease.isValid()); } - void replyPendingRequest(TermIndex termIndex, RaftClientReply reply, RetryCacheImpl.CacheEntry cacheEntry) { - final PendingRequest pending = pendingRequests.remove(termIndex); - - final LongSupplier replyMethod = () -> { - cacheEntry.updateResult(reply); - if (pending != null) { - pending.setReply(reply); - } - return termIndex.getIndex(); - }; - - if (readIndexType == Type.REPLIED_INDEX) { - replyFlusher.hold(replyMethod); - } else { - replyMethod.getAsLong(); - } + void replyPendingRequest(TermIndex termIndex, RaftClientReply reply) { + pendingRequests.replyPendingRequest(termIndex, reply); } TransactionContext getTransactionContext(TermIndex termIndex) { @@ -1352,6 +1338,7 @@ private static boolean isCaughtUp(FollowerInfo follower) { } @Override + @SuppressWarnings("deprecation") public void checkHealth(FollowerInfo follower) { final TimeDuration elapsedTime = follower.getLastRpcResponseTime().elapsedTime(); if (elapsedTime.compareTo(server.properties().rpcSlownessTimeout()) > 0) { diff --git a/ratis-server/src/main/java/org/apache/ratis/server/impl/PendingRequests.java b/ratis-server/src/main/java/org/apache/ratis/server/impl/PendingRequests.java index f89d354e6a..259695d5ed 100644 --- a/ratis-server/src/main/java/org/apache/ratis/server/impl/PendingRequests.java +++ b/ratis-server/src/main/java/org/apache/ratis/server/impl/PendingRequests.java @@ -117,7 +117,7 @@ private static class RequestMap { raftServerMetrics.addNumPendingRequestsMegaByteSize(resource::getMegaByteSize); } - synchronized Permit tryAcquire(Message message) { + Permit tryAcquire(Message message) { final int messageSize = Message.getSize(message); final int messageSizeMb = roundUpMb(messageSize ); final Acquired acquired = resource.tryAcquire(messageSizeMb); @@ -139,7 +139,13 @@ synchronized Permit tryAcquire(Message message) { if (messageSizeMb > diffMb) { resource.releaseExtraMb(messageSizeMb - diffMb); } + return putPermit(); + } + private synchronized Permit putPermit() { + if (resource.isClosed()) { + return null; + } final Permit permit = new Permit(); permits.put(permit, permit); return permit; @@ -151,9 +157,9 @@ synchronized PendingRequest put(Permit permit, PendingRequest p) { if (removed == null) { return null; } - Preconditions.assertSame(permit, removed, "permit"); + Preconditions.assertTrue(removed == permit); final PendingRequest previous = map.put(p.getTermIndex(), p); - Preconditions.assertNull(previous, "previous"); + Preconditions.assertTrue(previous == null); return p; } @@ -264,13 +270,12 @@ TransactionContext getTransactionContext(TermIndex termIndex) { return pendingRequest != null ? pendingRequest.getEntry() : null; } - /** @return the removed the {@link PendingRequest} for the given {@link TermIndex}. */ - PendingRequest remove(TermIndex termIndex) { + void replyPendingRequest(TermIndex termIndex, RaftClientReply reply) { final PendingRequest pending = pendingRequests.remove(termIndex); if (pending != null) { Preconditions.assertEquals(termIndex, pending.getTermIndex(), "termIndex"); + pending.setReply(reply); } - return pending; } /** diff --git a/ratis-server/src/main/java/org/apache/ratis/server/impl/RaftServerImpl.java b/ratis-server/src/main/java/org/apache/ratis/server/impl/RaftServerImpl.java index b28961b816..043ba1ee71 100644 --- a/ratis-server/src/main/java/org/apache/ratis/server/impl/RaftServerImpl.java +++ b/ratis-server/src/main/java/org/apache/ratis/server/impl/RaftServerImpl.java @@ -174,7 +174,7 @@ public RaftPeerRole getCurrentRole() { @Override public boolean isLeaderReady() { - return getRole().isLeaderReady(); + return isLeader() && getRole().isLeaderReady(); } @Override @@ -239,17 +239,19 @@ public long[] getFollowerMatchIndices() { private final RetryCacheImpl retryCache; private final CommitInfoCache commitInfoCache = new CommitInfoCache(); private final WriteIndexCache writeIndexCache; - private final NavigableIndices appendLogTermIndices; private final RaftServerJmxAdapter jmxAdapter = new RaftServerJmxAdapter(this); private final LeaderElectionMetrics leaderElectionMetrics; private final RaftServerMetricsImpl raftServerMetrics; - - // Disallow appendEntries before start() complete; otherwise, it could fail with illegal lifeCycle transition - private final AtomicBoolean startComplete = new AtomicBoolean(false); - private final AtomicBoolean firstElectionSinceStartup = new AtomicBoolean(true); private final CountDownLatch closeFinishedLatch = new CountDownLatch(1); + // To avoid append entry before complete start() method + // For example, if thread1 start(), but before thread1 startAsFollower(), thread2 receive append entry + // request, and change state to RUNNING by lifeCycle.compareAndTransition(STARTING, RUNNING), + // then thread1 execute lifeCycle.transition(RUNNING) in startAsFollower(), + // So happens IllegalStateException: ILLEGAL TRANSITION: RUNNING -> RUNNING, + private final AtomicBoolean startComplete; + private final TransferLeadership transferLeadership; private final SnapshotManagementRequestHandler snapshotRequestHandler; private final SnapshotInstallationHandler snapshotInstallationHandler; @@ -284,13 +286,15 @@ public long[] getFollowerMatchIndices() { this.readOption = RaftServerConfigKeys.Read.option(properties); this.writeIndexCache = new WriteIndexCache(properties); this.transactionManager = new TransactionManager(id); - TraceUtils.setTracerWhenEnabled(properties); this.leaderElectionMetrics = LeaderElectionMetrics.getLeaderElectionMetrics( getMemberId(), state::getLastLeaderElapsedTimeMs); this.raftServerMetrics = RaftServerMetricsImpl.computeIfAbsentRaftServerMetrics( getMemberId(), this::getCommitIndex, retryCache::getStatistics); + this.startComplete = new AtomicBoolean(false); + this.threadGroup = new ThreadGroup(proxy.getThreadGroup(), getMemberId().toString()); + this.transferLeadership = new TransferLeadership(this, properties); this.snapshotRequestHandler = new SnapshotManagementRequestHandler(this); this.snapshotInstallationHandler = new SnapshotInstallationHandler(this, properties); @@ -408,7 +412,7 @@ boolean start() throws IOException { startAsPeer(RaftPeerRole.LISTENER); } else { LOG.info("{}: start with initializing state, conf={}", getMemberId(), conf); - setRole(RaftPeerRole.FOLLOWER, NOT_IN_CONF); + setRole(RaftPeerRole.FOLLOWER, "start"); } jmxAdapter.registerMBean(); @@ -589,7 +593,7 @@ private synchronized CompletableFuture changeToFollower( throw new IllegalStateException("Unexpected role " + old); } CompletableFuture future = CompletableFuture.completedFuture(null); - if (shouldSetFollower(old, force)) { + if ((old != RaftPeerRole.FOLLOWER || force) && old != RaftPeerRole.LISTENER) { setRole(RaftPeerRole.FOLLOWER, reason); if (old == RaftPeerRole.LEADER) { future = role.shutdownLeaderState(false) @@ -605,7 +609,7 @@ private synchronized CompletableFuture changeToFollower( state.setLeader(null, reason); } else if (old == RaftPeerRole.CANDIDATE) { future = role.shutdownLeaderElection(); - } else if (old == RaftPeerRole.FOLLOWER || old == RaftPeerRole.LISTENER) { + } else if (old == RaftPeerRole.FOLLOWER) { future = role.shutdownFollowerState(); } @@ -760,12 +764,7 @@ RaftClientReply newExceptionReply(RaftClientRequest request, RaftException excep } private CompletableFuture checkLeaderState(RaftClientRequest request) { - try { - assertGroup(getMemberId(), request); - } catch (GroupMismatchException e) { - return JavaUtils.completeExceptionally(e); - } - return checkLeaderState(request, null, null); + return checkLeaderState(request, null); } /** @@ -781,7 +780,7 @@ private CompletableFuture checkLeaderState(RaftClientRequest re if (!getInfo().isLeader()) { NotLeaderException exception = generateNotLeaderException(); final RaftClientReply reply = newExceptionReply(request, exception); - return failWithReply(reply, entry, context); + return RetryCacheImpl.failWithReply(reply, entry); } if (!getInfo().isLeaderReady()) { final CacheEntry cacheEntry = retryCache.getIfPresent(ClientInvocationId.valueOf(request)); @@ -790,13 +789,13 @@ private CompletableFuture checkLeaderState(RaftClientRequest re } final LeaderNotReadyException lnre = new LeaderNotReadyException(getMemberId()); final RaftClientReply reply = newExceptionReply(request, lnre); - return failWithReply(reply, entry, context); + return RetryCacheImpl.failWithReply(reply, entry); } if (!request.isReadOnly() && isSteppingDown()) { final LeaderSteppingDownException lsde = new LeaderSteppingDownException(getMemberId() + " is stepping down"); final RaftClientReply reply = newExceptionReply(request, lsde); - return failWithReply(reply, entry, context); + return RetryCacheImpl.failWithReply(reply, entry); } return null; @@ -823,63 +822,66 @@ void assertLifeCycleState(Set expected) throws ServerNotReadyEx } /** - * Handle a normal update request from client. + * Append a transaction to the log for processing a client request. + * Note that the given request could be different from {@link TransactionContext#getClientRequest()} + * since the request could be converted; see {@link #convertRaftClientRequest(RaftClientRequest)}. + * + * @param request The client request. + * @param context The context of the transaction. + * @param cacheEntry the entry in the retry cache. + * @return a future of the reply. */ private CompletableFuture appendTransaction( - RaftClientRequest request, TransactionContextImpl context, CacheEntry cacheEntry) throws IOException { + RaftClientRequest request, TransactionContextImpl context, CacheEntry cacheEntry) { + Objects.requireNonNull(request, "request == null"); CodeInjectionForTesting.execute(APPEND_TRANSACTION, getId(), request.getClientId(), request, context, cacheEntry); - assertLifeCycleState(LifeCycle.States.RUNNING); - - final LeaderStateImpl unsyncedLeaderState = role.getLeaderState().orElse(null); - if (unsyncedLeaderState == null) { - final NotLeaderException nle = generateNotLeaderException(); - final RaftClientReply reply = newExceptionReply(request, nle); - return failWithReply(reply, cacheEntry, context); - } - final PendingRequests.Permit unsyncedPermit = unsyncedLeaderState.tryAcquirePendingRequest(request.getMessage()); - if (unsyncedPermit == null) { - return getResourceUnavailableReply("acquire a pending write request", request, cacheEntry, context); - } - - final LeaderStateImpl leaderState; final PendingRequest pending; synchronized (this) { - final CompletableFuture reply = checkLeaderState(request, cacheEntry, context); + final CompletableFuture reply = checkLeaderState(request, cacheEntry); if (reply != null) { return reply; } - leaderState = role.getLeaderStateNonNull(); - final PendingRequests.Permit permit = leaderState == unsyncedLeaderState ? unsyncedPermit - : leaderState.tryAcquirePendingRequest(request.getMessage()); - if (permit == null) { - return getResourceUnavailableReply("acquire a pending write request", request, cacheEntry, context); - } - // append the message to its local log + final LeaderStateImpl leaderState = role.getLeaderStateNonNull(); writeIndexCache.add(request.getClientId(), context.getLogIndexFuture()); + + final PendingRequests.Permit permit = leaderState.tryAcquirePendingRequest(request.getMessage()); + if (permit == null) { + cacheEntry.failWithException(new ResourceUnavailableException( + getMemberId() + ": Failed to acquire a pending write request for " + request)); + return cacheEntry.getReplyFuture(); + } try { + assertLifeCycleState(LifeCycle.States.RUNNING); state.appendLog(context); } catch (StateMachineException e) { // the StateMachineException is thrown by the SM in the preAppend stage. // Return the exception in a RaftClientReply. - final RaftClientReply exceptionReply = newExceptionReply(request, e); + RaftClientReply exceptionReply = newExceptionReply(request, e); + cacheEntry.failWithReply(exceptionReply); // leader will step down here if (e.leaderShouldStepDown() && getInfo().isLeader()) { - leaderState.submitStepDownEvent(StepDownReason.STATE_MACHINE_EXCEPTION); + leaderState.submitStepDownEvent(LeaderState.StepDownReason.STATE_MACHINE_EXCEPTION); } - return failWithReply(exceptionReply, cacheEntry, null); + return CompletableFuture.completedFuture(exceptionReply); + } catch (ServerNotReadyException e) { + final RaftClientReply exceptionReply = newExceptionReply(request, e); + return CompletableFuture.completedFuture(exceptionReply); } // put the request into the pending queue pending = leaderState.addPendingRequest(permit, request, context); if (pending == null) { - return getResourceUnavailableReply("add a pending write request", request, cacheEntry, context); + cacheEntry.failWithException(new ResourceUnavailableException( + getMemberId() + ": Failed to add a pending write request for " + request)); + return cacheEntry.getReplyFuture(); } + leaderState.notifySenders(); } - leaderState.notifySenders(); + return pending.getFuture(); } @@ -917,14 +919,16 @@ private RaftClientReply combineReplies(RaftClientReply reply, RaftClientReply wa } void stepDownOnJvmPause() { - role.getLeaderState().ifPresent(leader -> leader.submitStepDownEvent(StepDownReason.JVM_PAUSE)); + role.getLeaderState().ifPresent(leader -> leader.submitStepDownEvent(LeaderState.StepDownReason.JVM_PAUSE)); } - private RaftClientRequest filterDataStreamRaftClientRequest(RaftClientRequest request) - throws InvalidProtocolBufferException { - return !request.is(TypeCase.FORWARD) ? request : ClientProtoUtils.toRaftClientRequest( - RaftClientRequestProto.parseFrom( - request.getMessage().getContent().asReadOnlyByteBuffer())); + /** If the given request is {@link TypeCase#FORWARD}, convert it. */ + static RaftClientRequest convertRaftClientRequest(RaftClientRequest request) throws InvalidProtocolBufferException { + if (!request.is(TypeCase.FORWARD)) { + return request; + } + return ClientProtoUtils.toRaftClientRequest(RaftClientRequestProto.parseFrom( + request.getMessage().getContent().asReadOnlyByteBuffer())); } CompletableFuture executeSubmitServerRequestAsync( @@ -934,10 +938,9 @@ CompletableFuture executeSubmitServerRequestAsync( serverExecutor).join(); } - CompletableFuture executeSubmitClientRequestAsync(RaftClientRequest request) { - return CompletableFuture.supplyAsync( - () -> JavaUtils.callAsUnchecked(() -> submitClientRequestAsync(request), CompletionException::new), - clientExecutor).join(); + CompletableFuture executeSubmitClientRequestAsync( + ReferenceCountedObject request) { + return CompletableFuture.supplyAsync(() -> submitClientRequestAsync(request), clientExecutor).join(); } @Override @@ -967,7 +970,8 @@ public CompletableFuture submitClientRequestAsync( } } - private CompletableFuture replyFuture(RaftClientRequest request) throws IOException { + private CompletableFuture replyFuture(ReferenceCountedObject requestRef) { + final RaftClientRequest request = requestRef.get(); retryCache.invalidateRepliedRequests(request); final TypeCase type = request.getType().getTypeCase(); @@ -979,17 +983,18 @@ private CompletableFuture replyFuture(RaftClientRequest request case WATCH: return watchAsync(request); case MESSAGESTREAM: - return messageStreamAsync(request); + return messageStreamAsync(requestRef); case WRITE: case FORWARD: - return writeAsync(request); + return writeAsync(requestRef); default: throw new IllegalStateException("Unexpected request type: " + type + ", request=" + request); } } - private CompletableFuture writeAsync(RaftClientRequest request) throws IOException { - final CompletableFuture future = writeAsyncImpl(request); + private CompletableFuture writeAsync(ReferenceCountedObject requestRef) { + final RaftClientRequest request = requestRef.get(); + final CompletableFuture future = writeAsyncImpl(requestRef); if (request.is(TypeCase.WRITE)) { // check replication final ReplicationLevel replication = request.getType().getWrite().getReplication(); @@ -1000,7 +1005,8 @@ private CompletableFuture writeAsync(RaftClientRequest request) return future; } - private CompletableFuture writeAsyncImpl(RaftClientRequest request) throws IOException { + private CompletableFuture writeAsyncImpl(ReferenceCountedObject requestRef) { + final RaftClientRequest request = requestRef.get(); final CompletableFuture reply = checkLeaderState(request); if (reply != null) { return reply; @@ -1013,30 +1019,30 @@ private CompletableFuture writeAsyncImpl(RaftClientRequest requ // return the cached future. return cacheEntry.getReplyFuture(); } - // This request will be added to pending requests later in appendTransaction. - // Any failure in between must invoke cancelTransaction. - final TransactionContextImpl context = (TransactionContextImpl) stateMachine.startTransaction( - filterDataStreamRaftClientRequest(request)); + // TODO: this client request will not be added to pending requests until + // later which means that any failure in between will leave partial state in + // the state machine. We should call cancelTransaction() for failed requests + final TransactionContextImpl context; + try { + context = (TransactionContextImpl) stateMachine.startTransaction(convertRaftClientRequest(request)); + } catch (IOException e) { + final RaftClientReply exceptionReply = newExceptionReply(request, + new RaftException("Failed to startTransaction for " + request, e)); + cacheEntry.failWithReply(exceptionReply); + return CompletableFuture.completedFuture(exceptionReply); + } if (context.getException() != null) { - final Exception exception = context.getException(); - final StateMachineException e = new StateMachineException(getMemberId(), exception); + final StateMachineException e = new StateMachineException(getMemberId(), context.getException()); final RaftClientReply exceptionReply = newExceptionReply(request, e); - return failWithReply(exceptionReply, cacheEntry, context); + cacheEntry.failWithReply(exceptionReply); + return CompletableFuture.completedFuture(exceptionReply); } - try { - return appendTransaction(request, context, cacheEntry); - } catch (Exception e) { - cancelTransaction(context, e); - throw e; - } + context.setDelegatedRef(requestRef); + return appendTransaction(request, context, cacheEntry); } private CompletableFuture watchAsync(RaftClientRequest request) { - if (OrderedAsync.DUMMY.getContent().equals(request.getMessage().getContent())) { - return CompletableFuture.completedFuture(RaftClientReply.newBuilder().setRequest(request).build()); - } - final CompletableFuture reply = checkLeaderState(request); if (reply != null) { return reply; @@ -1051,7 +1057,7 @@ private CompletableFuture watchAsync(RaftClientRequest request) private CompletableFuture staleReadAsync(RaftClientRequest request) { final long minIndex = request.getType().getStaleRead().getMinIndex(); final long commitIndex = state.getLog().getLastCommittedIndex(); - LOG.debug("{}: minIndex={}, commitIndex={} from {}", getMemberId(), minIndex, commitIndex, request.getClientId()); + LOG.debug("{}: minIndex={}, commitIndex={}", getMemberId(), minIndex, commitIndex); if (commitIndex < minIndex) { final StaleReadException e = new StaleReadException( "Unable to serve stale-read due to server commit index = " + commitIndex + " < min = " + minIndex); @@ -1135,7 +1141,8 @@ private RaftClientReply readException2Reply(RaftClientRequest request, Throwable } } - private CompletableFuture messageStreamAsync(RaftClientRequest request) throws IOException { + private CompletableFuture messageStreamAsync(ReferenceCountedObject requestRef) { + final RaftClientRequest request = requestRef.get(); final CompletableFuture reply = checkLeaderState(request); if (reply != null) { return reply; @@ -1457,13 +1464,12 @@ public RequestVoteReplyProto requestVote(RequestVoteRequestProto r) throws IOExc RaftPeerId.valueOf(request.getRequestorId()), ProtoUtils.toRaftGroupId(request.getRaftGroupId()), r.getCandidateTerm(), - TermIndex.valueOf(r.getCandidateLastEntry()), - request.getCallId()); + TermIndex.valueOf(r.getCandidateLastEntry())); } private RequestVoteReplyProto requestVote(Phase phase, RaftPeerId candidateId, RaftGroupId candidateGroupId, - long candidateTerm, TermIndex candidateLastEntry, long callId) throws IOException { + long candidateTerm, TermIndex candidateLastEntry) throws IOException { CodeInjectionForTesting.execute(REQUEST_VOTE, getId(), candidateId, candidateTerm, candidateLastEntry); LOG.info("{}: receive requestVote({}, {}, {}, {}, {})", @@ -1498,7 +1504,7 @@ private RequestVoteReplyProto requestVote(Phase phase, shouldShutdown = true; } reply = toRequestVoteReplyProto(candidateId, getMemberId(), - voteGranted, state.getCurrentTerm(), shouldShutdown); + voteGranted, state.getCurrentTerm(), shouldShutdown, state.getLastEntry()); if (LOG.isInfoEnabled()) { LOG.info("{} replies to {} vote request: {}. Peer's state: {}", getMemberId(), phase, toRequestVoteReplyString(reply), state); @@ -1568,7 +1574,7 @@ public CompletableFuture readIndexAsync(ReadIndexRequestPro static void logAppendEntries(boolean isHeartbeat, Supplier message) { if (isHeartbeat) { if (LOG.isTraceEnabled()) { - LOG.trace("HEARTBEAT: {}", message.get()); + LOG.trace("HEARTBEAT: " + message.get()); } } else { if (LOG.isDebugEnabled()) { @@ -1624,7 +1630,7 @@ leaderId, getMemberId(), currentTerm, followerCommit, state.getNextIndex(), } catch (IOException e) { return JavaUtils.completeExceptionally(e); } - state.setLeader(leaderId, Op.APPEND_ENTRIES); + state.setLeader(leaderId, "appendEntries"); if (!proto.getInitializing() && lifeCycle.compareAndTransition(State.STARTING, State.RUNNING)) { role.startFollowerState(this, Op.APPEND_ENTRIES); @@ -1729,7 +1735,7 @@ private long checkInconsistentAppendEntries(TermIndex previous, List replyPendingRequest( } // update pending request - final LeaderStateImpl leader = role.getLeaderState().orElse(null); - if (leader != null) { - leader.replyPendingRequest(termIndex, r, cacheEntry); - } else { - cacheEntry.updateResult(r); - } + role.getLeaderState().ifPresent(leader -> leader.replyPendingRequest(termIndex, r)); + cacheEntry.updateResult(r); }); } diff --git a/ratis-server/src/main/java/org/apache/ratis/server/impl/RaftServerProxy.java b/ratis-server/src/main/java/org/apache/ratis/server/impl/RaftServerProxy.java index 4bcd31080b..2914c434f1 100644 --- a/ratis-server/src/main/java/org/apache/ratis/server/impl/RaftServerProxy.java +++ b/ratis-server/src/main/java/org/apache/ratis/server/impl/RaftServerProxy.java @@ -52,8 +52,8 @@ import org.apache.ratis.util.MemoizedSupplier; import org.apache.ratis.util.Preconditions; import org.apache.ratis.util.ProtoUtils; +import org.apache.ratis.util.ReferenceCountedObject; import org.apache.ratis.util.TimeDuration; -import org.apache.ratis.util.VersionInfo; import java.io.Closeable; import java.io.File; @@ -208,8 +208,6 @@ String toString(RaftGroupId groupId, CompletableFuture f) { RaftServerProxy(RaftPeerId id, StateMachine.Registry stateMachineRegistry, RaftProperties properties, Parameters parameters, ThreadGroup threadGroup) { - VersionInfo.load(getClass()).printStartupMessages(id, LOG::info); - this.properties = properties; this.stateMachineRegistry = stateMachineRegistry; @@ -454,9 +452,15 @@ public void close() { } @Override - public CompletableFuture submitClientRequestAsync(RaftClientRequest request) { - return getImplFuture(request.getRaftGroupId()) - .thenCompose(impl -> impl.executeSubmitClientRequestAsync(request)); + public CompletableFuture submitClientRequestAsync( + ReferenceCountedObject requestRef) { + final RaftClientRequest request = requestRef.retain(); + try { + return getImplFuture(request.getRaftGroupId()) + .thenCompose(impl -> impl.executeSubmitClientRequestAsync(requestRef)); + } finally { + requestRef.release(); + } } @Override diff --git a/ratis-server/src/main/java/org/apache/ratis/server/impl/ReadRequests.java b/ratis-server/src/main/java/org/apache/ratis/server/impl/ReadRequests.java index 6112a46009..e63a23a0b8 100644 --- a/ratis-server/src/main/java/org/apache/ratis/server/impl/ReadRequests.java +++ b/ratis-server/src/main/java/org/apache/ratis/server/impl/ReadRequests.java @@ -20,7 +20,7 @@ import org.apache.ratis.conf.RaftProperties; import org.apache.ratis.protocol.exceptions.ReadException; import org.apache.ratis.server.RaftServerConfigKeys; -import org.apache.ratis.util.Preconditions; +import org.apache.ratis.statemachine.StateMachine; import org.apache.ratis.util.TimeDuration; import org.apache.ratis.util.TimeoutExecutor; import org.slf4j.Logger; @@ -29,7 +29,7 @@ import java.util.NavigableMap; import java.util.TreeMap; import java.util.concurrent.CompletableFuture; -import java.util.function.LongConsumer; +import java.util.function.Consumer; /** For supporting linearizable read. */ class ReadRequests { @@ -37,18 +37,10 @@ class ReadRequests { static class ReadIndexQueue { private final TimeoutExecutor scheduler = TimeoutExecutor.getInstance(); - /** The log index known to be applied. */ - private long lastAppliedIndex; - /** - * Map : readIndex -> appliedIndexFuture (when completes, readIndex <= appliedIndex). - * Invariant: all keys > lastAppliedIndex. - */ private final NavigableMap> sorted = new TreeMap<>(); - private final TimeDuration readTimeout; - ReadIndexQueue(long lastAppliedIndex, TimeDuration readTimeout) { - this.lastAppliedIndex = lastAppliedIndex; + ReadIndexQueue(TimeDuration readTimeout) { this.readTimeout = readTimeout; } @@ -56,9 +48,6 @@ CompletableFuture add(long readIndex) { final CompletableFuture returned; final boolean create; synchronized (this) { - if (readIndex <= lastAppliedIndex) { - return CompletableFuture.completedFuture(lastAppliedIndex); - } // The same as computeIfAbsent except that it also tells if a new value is created. final CompletableFuture existing = sorted.get(readIndex); create = existing == null; @@ -90,19 +79,7 @@ private void handleTimeout(long readIndex) { /** Complete all the entries less than or equal to the given applied index. */ - synchronized void complete(long appliedIndex) { - if (appliedIndex > lastAppliedIndex) { - lastAppliedIndex = appliedIndex; - } else { - // appliedIndex <= lastAppliedIndex: nothing to do - if (!sorted.isEmpty()) { - // Assert: all keys > lastAppliedIndex. - final long first = sorted.firstKey(); - Preconditions.assertTrue(first > lastAppliedIndex, - () -> "first = " + first + " <= lastAppliedIndex = " + lastAppliedIndex); - } - return; - } + synchronized void complete(Long appliedIndex) { final NavigableMap> headMap = sorted.headMap(appliedIndex, true); headMap.values().forEach(f -> f.complete(appliedIndex)); headMap.clear(); @@ -110,16 +87,27 @@ synchronized void complete(long appliedIndex) { } private final ReadIndexQueue readIndexQueue; + private final StateMachine stateMachine; - ReadRequests(long appliedIndex, RaftProperties properties) { - this.readIndexQueue = new ReadIndexQueue(appliedIndex, RaftServerConfigKeys.Read.timeout(properties)); + ReadRequests(RaftProperties properties, StateMachine stateMachine) { + this.readIndexQueue = new ReadIndexQueue(RaftServerConfigKeys.Read.timeout(properties)); + this.stateMachine = stateMachine; } - LongConsumer getAppliedIndexConsumer() { + Consumer getAppliedIndexConsumer() { return readIndexQueue::complete; } CompletableFuture waitToAdvance(long readIndex) { - return readIndexQueue.add(readIndex); + final long lastApplied = stateMachine.getLastAppliedTermIndex().getIndex(); + if (lastApplied >= readIndex) { + return CompletableFuture.completedFuture(lastApplied); + } + final CompletableFuture f = readIndexQueue.add(readIndex); + final long current = stateMachine.getLastAppliedTermIndex().getIndex(); + if (current > lastApplied) { + readIndexQueue.complete(current); + } + return f; } } diff --git a/ratis-server/src/main/java/org/apache/ratis/server/impl/RetryCacheImpl.java b/ratis-server/src/main/java/org/apache/ratis/server/impl/RetryCacheImpl.java index 96ad62a531..50d238b07a 100644 --- a/ratis-server/src/main/java/org/apache/ratis/server/impl/RetryCacheImpl.java +++ b/ratis-server/src/main/java/org/apache/ratis/server/impl/RetryCacheImpl.java @@ -84,10 +84,9 @@ void failWithReply(RaftClientReply reply) { replyFuture.complete(reply); } - CompletableFuture failWithException(Throwable t) { + void failWithException(Throwable t) { failed = true; replyFuture.completeExceptionally(t); - return replyFuture; } @Override @@ -257,4 +256,24 @@ public synchronized void close() { cache.invalidateAll(); statistics.set(null); } + + static CompletableFuture failWithReply( + RaftClientReply reply, CacheEntry entry) { + if (entry != null) { + entry.failWithReply(reply); + return entry.getReplyFuture(); + } else { + return CompletableFuture.completedFuture(reply); + } + } + + static CompletableFuture failWithException( + Throwable t, CacheEntry entry) { + if (entry != null) { + entry.failWithException(t); + return entry.getReplyFuture(); + } else { + return JavaUtils.completeExceptionally(t); + } + } } diff --git a/ratis-server/src/main/java/org/apache/ratis/server/impl/ServerImplUtils.java b/ratis-server/src/main/java/org/apache/ratis/server/impl/ServerImplUtils.java index ae6a7ee005..864b402a23 100644 --- a/ratis-server/src/main/java/org/apache/ratis/server/impl/ServerImplUtils.java +++ b/ratis-server/src/main/java/org/apache/ratis/server/impl/ServerImplUtils.java @@ -48,8 +48,6 @@ import java.util.TreeMap; import java.util.Objects; import java.util.concurrent.TimeUnit; -import java.util.concurrent.atomic.AtomicReference; -import java.util.function.Function; /** Server utilities for internal use. */ public final class ServerImplUtils { @@ -172,7 +170,6 @@ public static RaftServerProxy newRaftServer( RaftPeerId id, RaftGroup group, RaftStorage.StartupOption option, StateMachine.Registry stateMachineRegistry, ThreadGroup threadGroup, RaftProperties properties, Parameters parameters) throws IOException { RaftServer.LOG.debug("newRaftServer: {}, {}", id, group); - Objects.requireNonNull(id, "id == null"); if (group != null && !group.getPeers().isEmpty()) { Objects.requireNonNull(id, () -> "RaftPeerId " + id + " is not in RaftGroup " + group); Objects.requireNonNull(group.getPeer(id), () -> "RaftPeerId " + id + " is not in RaftGroup " + group); diff --git a/ratis-server/src/main/java/org/apache/ratis/server/impl/ServerProtoUtils.java b/ratis-server/src/main/java/org/apache/ratis/server/impl/ServerProtoUtils.java index 19d4ce6a75..494037f373 100644 --- a/ratis-server/src/main/java/org/apache/ratis/server/impl/ServerProtoUtils.java +++ b/ratis-server/src/main/java/org/apache/ratis/server/impl/ServerProtoUtils.java @@ -24,7 +24,6 @@ import org.apache.ratis.protocol.RaftGroupMemberId; import org.apache.ratis.protocol.RaftPeer; import org.apache.ratis.protocol.RaftPeerId; -import org.apache.ratis.rpc.CallId; import org.apache.ratis.server.protocol.TermIndex; import org.apache.ratis.server.raftlog.RaftLog; import org.apache.ratis.util.Preconditions; @@ -43,6 +42,17 @@ private static RaftRpcReplyProto.Builder toRaftRpcReplyProtoBuilder( requestorId.toByteString(), replyId.getPeerId().toByteString(), replyId.getGroupId(), null, success); } + static RequestVoteReplyProto toRequestVoteReplyProto( + RaftPeerId requestorId, RaftGroupMemberId replyId, boolean success, long term, boolean shouldShutdown, + TermIndex lastEntry) { + return RequestVoteReplyProto.newBuilder() + .setServerReply(toRaftRpcReplyProtoBuilder(requestorId, replyId, success)) + .setTerm(term) + .setShouldShutdown(shouldShutdown) + .setLastEntry((lastEntry != null? lastEntry : TermIndex.INITIAL_VALUE).toProto()) + .build(); + } + static RequestVoteReplyProto toRequestVoteReplyProto( RaftPeerId requestorId, RaftGroupMemberId replyId, boolean success, long term, boolean shouldShutdown, TermIndex lastEntry, long callId) { @@ -57,8 +67,7 @@ static RequestVoteReplyProto toRequestVoteReplyProto( static RequestVoteRequestProto toRequestVoteRequestProto( RaftGroupMemberId requestorId, RaftPeerId replyId, long term, TermIndex lastEntry, boolean preVote) { final RequestVoteRequestProto.Builder b = RequestVoteRequestProto.newBuilder() - .setServerRequest(ClientProtoUtils.toRaftRpcRequestProtoBuilder(requestorId, replyId) - .setCallId(CallId.getAndIncrement())) + .setServerRequest(ClientProtoUtils.toRaftRpcRequestProtoBuilder(requestorId, replyId)) .setCandidateTerm(term) .setPreVote(preVote); Optional.ofNullable(lastEntry).map(TermIndex::toProto).ifPresent(b::setCandidateLastEntry); @@ -180,12 +189,8 @@ static ServerRpcProto toServerRpcProto(RaftPeer peer, long delay) { // if no peer information return empty return ServerRpcProto.getDefaultInstance(); } - return toServerRpcProto(peer.getRaftPeerProto(), delay); - } - - static ServerRpcProto toServerRpcProto(RaftPeerProto peer, long delay) { return ServerRpcProto.newBuilder() - .setId(peer) + .setId(peer.getRaftPeerProto()) .setLastRpcElapsedTimeMs(delay) .build(); } diff --git a/ratis-server/src/main/java/org/apache/ratis/server/impl/ServerState.java b/ratis-server/src/main/java/org/apache/ratis/server/impl/ServerState.java index cef0af7304..c49e9554f0 100644 --- a/ratis-server/src/main/java/org/apache/ratis/server/impl/ServerState.java +++ b/ratis-server/src/main/java/org/apache/ratis/server/impl/ServerState.java @@ -24,7 +24,6 @@ import org.apache.ratis.server.RaftConfiguration; import org.apache.ratis.server.RaftServerConfigKeys; import org.apache.ratis.server.impl.LeaderElection.Phase; -import org.apache.ratis.server.protocol.RaftServerProtocol.Op; import org.apache.ratis.server.protocol.TermIndex; import org.apache.ratis.server.raftlog.LogProtoUtils; import org.apache.ratis.server.raftlog.RaftLog; @@ -124,8 +123,13 @@ class ServerState { // On start the leader is null, start the clock now this.lastNoLeaderTime = new AtomicReference<>(Timestamp.currentTime()); this.noLeaderTimeout = RaftServerConfigKeys.Notification.noLeaderTimeout(prop); - this.log = JavaUtils.memoize(() -> initRaftLog(() -> getSnapshotIndexFromStateMachine(stateMachine), prop)); - this.readRequests = new ReadRequests(stateMachine.getLastAppliedTermIndex().getIndex(), prop); + + final LongSupplier getSnapshotIndexFromStateMachine = () -> Optional.ofNullable(stateMachine.getLatestSnapshot()) + .map(SnapshotInfo::getIndex) + .filter(i -> i >= 0) + .orElse(RaftLog.INVALID_LOG_INDEX); + this.log = JavaUtils.memoize(() -> initRaftLog(getSnapshotIndexFromStateMachine, prop)); + this.readRequests = new ReadRequests(prop, stateMachine); this.stateMachineUpdater = JavaUtils.memoize(() -> new StateMachineUpdater( stateMachine, server, this, getLog().getSnapshotIndex(), prop, this.readRequests.getAppliedIndexConsumer())); @@ -150,16 +154,6 @@ RaftGroupMemberId getMemberId() { return memberId; } - private long getSnapshotIndexFromStateMachine(StateMachine stateMachine) { - final SnapshotInfo latest = stateMachine.getLatestSnapshot(); - LOG.info("{}: getLatestSnapshot({}) returns {}", getMemberId(), stateMachine, latest); - if (latest == null) { - return RaftLog.INVALID_LOG_INDEX; - } - final long index = latest.getIndex(); - return index >= 0 ? index : RaftLog.INVALID_LOG_INDEX; - } - void writeRaftConfiguration(LogEntryProto conf) { getStorage().writeRaftConfiguration(conf); } @@ -258,7 +252,7 @@ RaftPeerId getVotedFor() { */ void grantVote(RaftPeerId candidateId) { votedFor = candidateId; - setLeader(null, Op.REQUEST_VOTE); + setLeader(null, "grantVote"); } void setLeader(RaftPeerId newLeaderId, Object op) { @@ -271,7 +265,7 @@ void setLeader(RaftPeerId newLeaderId, Object op) { suffix = ""; } else { final Timestamp previous = lastNoLeaderTime.getAndSet(null); - suffix = ", leader elected after " + (previous != null ? previous.elapsedTimeMs() : 0) + "ms"; + suffix = ", leader elected after " + previous.elapsedTimeMs() + "ms"; server.setFirstElection(op); server.getStateMachine().event().notifyLeaderChanged(getMemberId(), newLeaderId); } @@ -376,12 +370,10 @@ boolean isConfCommitted() { return getLog().getLastCommittedIndex() >= getRaftConf().getLogEntryIndex(); } - private boolean setRaftConf(LogEntryProto entry) { + void setRaftConf(LogEntryProto entry) { if (entry.hasConfigurationEntry()) { setRaftConf(LogProtoUtils.toRaftConfiguration(entry)); - return true; } - return false; } void setRaftConf(RaftConfiguration conf) { @@ -399,19 +391,10 @@ void truncate(long logIndex) { configurationManager.removeConfigurations(logIndex); } - void updateConfiguration(List entries) throws IOException { - if (entries == null || entries.isEmpty()) { - return; - } - configurationManager.removeConfigurations(entries.get(0).getIndex()); - - boolean changed = false; - for(LogEntryProto entry : entries) { - changed |= setRaftConf(entry); - } - - if (changed && server.getRole().getCurrentRole() == RaftPeerRole.LISTENER) { - server.changeToFollowerAndPersistMetadata(getCurrentTerm(), true, "setRaftConf").join(); + void updateConfiguration(List entries) { + if (entries != null && !entries.isEmpty()) { + configurationManager.removeConfigurations(entries.get(0).getIndex()); + entries.forEach(this::setRaftConf); } } @@ -443,7 +426,7 @@ void close() { if (e instanceof InterruptedException) { Thread.currentThread().interrupt(); } - LOG.warn("{}: Failed to join {}", getMemberId(), getStateMachineUpdater(), e); + LOG.warn(getMemberId() + ": Failed to join " + getStateMachineUpdater(), e); } try { @@ -451,7 +434,7 @@ void close() { getLog().close(); } } catch (Throwable e) { - LOG.warn("{}: Failed to close raft log {}", getMemberId(), getLog(), e); + LOG.warn(getMemberId() + ": Failed to close raft log " + getLog(), e); } try { @@ -459,7 +442,7 @@ void close() { getStorage().close(); } } catch (Throwable e) { - LOG.warn("{}: Failed to close raft storage {}", getMemberId(), getStorage(), e); + LOG.warn(getMemberId() + ": Failed to close raft storage " + getStorage(), e); } } diff --git a/ratis-server/src/main/java/org/apache/ratis/server/impl/SnapshotInstallationHandler.java b/ratis-server/src/main/java/org/apache/ratis/server/impl/SnapshotInstallationHandler.java index 7f191e54dc..4f1ac4177f 100644 --- a/ratis-server/src/main/java/org/apache/ratis/server/impl/SnapshotInstallationHandler.java +++ b/ratis-server/src/main/java/org/apache/ratis/server/impl/SnapshotInstallationHandler.java @@ -29,6 +29,7 @@ import org.apache.ratis.proto.RaftProtos.ServerRpcProto; import org.apache.ratis.protocol.RaftGroupId; import org.apache.ratis.protocol.RaftGroupMemberId; +import org.apache.ratis.protocol.RaftPeer; import org.apache.ratis.protocol.RaftPeerId; import org.apache.ratis.server.RaftServerConfigKeys; import org.apache.ratis.server.protocol.RaftServerProtocol; @@ -45,7 +46,6 @@ import org.slf4j.LoggerFactory; import java.io.IOException; -import java.util.Collections; import java.util.Optional; import java.util.concurrent.CompletableFuture; import java.util.concurrent.atomic.AtomicBoolean; @@ -144,8 +144,8 @@ private InstallSnapshotReplyProto installSnapshotImpl(InstallSnapshotRequestProt final LogEntryProto proto = request.getLastRaftConfigurationLogEntryProto(); state.truncate(proto.getIndex()); if (!state.getRaftConf().equals(LogProtoUtils.toRaftConfiguration(proto))) { - LOG.info("{}: set new configuration {} from snapshot", getMemberId(), ProtoUtils.shortDebugString(proto)); - state.updateConfiguration(Collections.singletonList(proto)); + LOG.info("{}: set new configuration {} from snapshot", getMemberId(), proto); + state.setRaftConf(proto); state.writeRaftConfiguration(proto); server.getStateMachine().event().notifyConfigurationChanged( proto.getTerm(), proto.getIndex(), proto.getConfigurationEntry()); @@ -181,7 +181,7 @@ private CompletableFuture checkAndInstallSnapshot(Ins future = server.changeToFollowerAndPersistMetadata(leaderTerm, true, "installSnapshot"); state.setLeader(leaderId, "installSnapshot"); - server.updateLastRpcTime(UpdateType.INSTALL_SNAPSHOT_START); + server.updateLastRpcTime(FollowerState.UpdateType.INSTALL_SNAPSHOT_START); long callId = chunk0CallId.get(); // 1. leaderTerm < currentTerm will never come here // 2. leaderTerm == currentTerm && callId == request.getCallId() @@ -228,7 +228,7 @@ private CompletableFuture checkAndInstallSnapshot(Ins chunk0CallId.set(-1); } } finally { - server.updateLastRpcTime(UpdateType.INSTALL_SNAPSHOT_COMPLETE); + server.updateLastRpcTime(FollowerState.UpdateType.INSTALL_SNAPSHOT_COMPLETE); } } if (snapshotChunkRequest.getDone()) { @@ -291,7 +291,7 @@ private CompletableFuture notifyStateMachineToInstall // For the cases where RaftConf is empty on newly started peer with empty peer list, // we retrieve leader info from installSnapShotRequestProto. final RoleInfoProto proto = leaderProto == null || server.getRaftConf().getPeer(state.getLeaderId()) != null? - server.getRoleInfoProto(): getRoleInfoProto(leaderProto); + server.getRoleInfoProto(): getRoleInfoProto(ProtoUtils.toRaftPeer(leaderProto)); // This is the first installSnapshot notify request for this term and // index. Notify the state machine to install the snapshot. LOG.info("{}: notifyInstallSnapshot: nextIndex is {} but the leader's first available index is {}.", @@ -385,7 +385,7 @@ private CompletableFuture notifyStateMachineToInstall } } - private RoleInfoProto getRoleInfoProto(RaftPeerProto leader) { + private RoleInfoProto getRoleInfoProto(RaftPeer leader) { final RoleInfo role = server.getRole(); final Optional fs = role.getFollowerState(); final ServerRpcProto leaderInfo = toServerRpcProto(leader, diff --git a/ratis-server/src/main/java/org/apache/ratis/server/impl/StateMachineUpdater.java b/ratis-server/src/main/java/org/apache/ratis/server/impl/StateMachineUpdater.java index 8c7cb83621..9c5290efe4 100644 --- a/ratis-server/src/main/java/org/apache/ratis/server/impl/StateMachineUpdater.java +++ b/ratis-server/src/main/java/org/apache/ratis/server/impl/StateMachineUpdater.java @@ -45,7 +45,6 @@ import java.util.concurrent.TimeUnit; import java.util.concurrent.atomic.AtomicReference; import java.util.function.Consumer; -import java.util.function.LongConsumer; import java.util.stream.LongStream; /** @@ -91,12 +90,12 @@ enum State { private final MemoizedSupplier stateMachineMetrics; - private final LongConsumer appliedIndexConsumer; + private final Consumer appliedIndexConsumer; private volatile boolean isRemoving; StateMachineUpdater(StateMachine stateMachine, RaftServerImpl server, - ServerState serverState, long lastAppliedIndex, RaftProperties properties, LongConsumer appliedIndexConsumer) { + ServerState serverState, long lastAppliedIndex, RaftProperties properties, Consumer appliedIndexConsumer) { this.name = ServerStringUtils.generateUnifiedName(serverState.getMemberId(), getClass()); this.appliedIndexConsumer = appliedIndexConsumer; this.infoIndexChange = s -> LOG.info("{}: {}", name, s); @@ -116,7 +115,8 @@ enum State { final int numSnapshotFilesRetained = RaftServerConfigKeys.Snapshot.retentionFileNum(properties); this.snapshotRetentionPolicy = new SnapshotRetentionPolicy() { @Override - public int getNumSnapshotsRetained() { + @SuppressWarnings({"deprecation", "try"}) +public int getNumSnapshotsRetained() { return numSnapshotFilesRetained; } }; @@ -288,6 +288,7 @@ private void checkAndTakeSnapshot(CompletableFuture futures) } } + @SuppressWarnings("try") private void takeSnapshot(CompletableFuture applyLogFutures) throws ExecutionException, InterruptedException { final long i; applyLogFutures.get(); diff --git a/ratis-server/src/main/java/org/apache/ratis/server/impl/WriteIndexCache.java b/ratis-server/src/main/java/org/apache/ratis/server/impl/WriteIndexCache.java index 98250ca229..df4448622f 100644 --- a/ratis-server/src/main/java/org/apache/ratis/server/impl/WriteIndexCache.java +++ b/ratis-server/src/main/java/org/apache/ratis/server/impl/WriteIndexCache.java @@ -46,15 +46,14 @@ class WriteIndexCache { .build(); } - void add(ClientId key, CompletableFuture current) { + void add(ClientId key, CompletableFuture future) { final AtomicReference> ref; try { ref = cache.get(key, AtomicReference::new); } catch (ExecutionException e) { throw new IllegalStateException(e); } - ref.updateAndGet(previous -> previous == null ? current - : previous.thenCombine(current, Math::max)); + ref.set(future); } CompletableFuture getWriteIndexFuture(RaftClientRequest request) { diff --git a/ratis-server/src/main/java/org/apache/ratis/server/leader/InstallSnapshotRequests.java b/ratis-server/src/main/java/org/apache/ratis/server/leader/InstallSnapshotRequests.java index 218c864e69..6300ea483c 100644 --- a/ratis-server/src/main/java/org/apache/ratis/server/leader/InstallSnapshotRequests.java +++ b/ratis-server/src/main/java/org/apache/ratis/server/leader/InstallSnapshotRequests.java @@ -119,7 +119,7 @@ public InstallSnapshotRequestProto next() { final FileInfo info = snapshot.getFiles().get(fileIndex); try { if (current == null) { - current = FileChunkReader.newInstance(info, getRelativePath.apply(info)); + current = new FileChunkReader(info, getRelativePath.apply(info)); } final FileChunkProto chunk = current.readFileChunk(snapshotChunkMaxSize); if (chunk.getDone()) { diff --git a/ratis-server/src/main/java/org/apache/ratis/server/leader/LogAppenderBase.java b/ratis-server/src/main/java/org/apache/ratis/server/leader/LogAppenderBase.java index ad579235b7..d7e82f4a3f 100644 --- a/ratis-server/src/main/java/org/apache/ratis/server/leader/LogAppenderBase.java +++ b/ratis-server/src/main/java/org/apache/ratis/server/leader/LogAppenderBase.java @@ -50,6 +50,7 @@ /** * An abstract implementation of {@link LogAppender}. */ +@SuppressWarnings({"deprecation", "try"}) public abstract class LogAppenderBase implements LogAppender { /** For buffering log entries to create an {@link EntryList}. */ private static class EntryBuffer { @@ -202,14 +203,7 @@ public void start() { @Override public boolean isRunning() { - return daemon.isWorking() - && isLeaderAlive(); - } - - private boolean isLeaderAlive() { - return server.getInfo().isAlive() - && server.getInfo().isLeader() - && getRaftLog().isOpened(); + return daemon.isWorking() && server.getInfo().isLeader(); } @Override @@ -218,12 +212,8 @@ public CompletableFuture stopAsync() { } void restart() { - if (daemon.isClosingOrClosed()) { - LOG.warn("{}: daemon is closing or closed, skipping restart", this); - return; - } - if (!isLeaderAlive()) { - LOG.warn("{}: leader is not ready, skipping restart", this); + if (!server.getInfo().isAlive()) { + LOG.warn("Failed to restart {}: server {} is not alive", this, server.getMemberId()); return; } getLeaderState().restart(this); @@ -251,6 +241,27 @@ public boolean hasPendingDataRequests() { return false; } + private TermIndex getPrevious(long nextIndex) { + if (nextIndex == RaftLog.LEAST_VALID_LOG_INDEX) { + return null; + } + + final long previousIndex = nextIndex - 1; + final TermIndex previous = getRaftLog().getTermIndex(previousIndex); + if (previous != null) { + return previous; + } + + final SnapshotInfo snapshot = server.getStateMachine().getLatestSnapshot(); + if (snapshot != null) { + final TermIndex snapshotTermIndex = snapshot.getTermIndex(); + if (snapshotTermIndex.getIndex() == previousIndex) { + return snapshotTermIndex; + } + } + + return null; + } protected long getNextIndexForInconsistency(long requestFirstIndex, long replyNextIndex) { long next = replyNextIndex; diff --git a/ratis-server/src/main/java/org/apache/ratis/server/leader/LogAppenderDaemon.java b/ratis-server/src/main/java/org/apache/ratis/server/leader/LogAppenderDaemon.java index c779d007ba..5de3f3b4da 100644 --- a/ratis-server/src/main/java/org/apache/ratis/server/leader/LogAppenderDaemon.java +++ b/ratis-server/src/main/java/org/apache/ratis/server/leader/LogAppenderDaemon.java @@ -58,10 +58,6 @@ public boolean isWorking() { return !LifeCycle.States.CLOSING_OR_CLOSED_OR_EXCEPTION.contains(lifeCycle.getCurrentState()); } - public boolean isClosingOrClosed() { - return LifeCycle.States.CLOSING_OR_CLOSED.contains(lifeCycle.getCurrentState()); - } - public void tryToStart() { if (lifeCycle.compareAndTransition(NEW, STARTING)) { daemon.start(); diff --git a/ratis-server/src/main/java/org/apache/ratis/server/raftlog/RaftLogBase.java b/ratis-server/src/main/java/org/apache/ratis/server/raftlog/RaftLogBase.java index 2fd301915c..8042f5e21c 100644 --- a/ratis-server/src/main/java/org/apache/ratis/server/raftlog/RaftLogBase.java +++ b/ratis-server/src/main/java/org/apache/ratis/server/raftlog/RaftLogBase.java @@ -89,8 +89,7 @@ protected RaftLogBase(RaftGroupMemberId memberId, RaftProperties properties) { this.name = memberId + "-" + JavaUtils.getClassSimpleName(getClass()); this.memberId = memberId; - final long index = getSnapshotIndexFromStateMachine.getAsLong(); - LOG.info("{}: snapshotIndexFromStateMachine = {}", name, index); + long index = getSnapshotIndexFromStateMachine.getAsLong(); this.commitIndex = new RaftLogIndex("commitIndex", index); this.snapshotIndex = new RaftLogIndex("snapshotIndex", index); this.purgeIndex = new RaftLogIndex("purgeIndex", LEAST_VALID_LOG_INDEX - 1); @@ -116,12 +115,13 @@ public void checkLogState() { state.assertOpen(); } - @Override + /** Is this log already opened? */ public boolean isOpened() { return state.isOpened(); } @Override + @SuppressWarnings("try") public boolean updateCommitIndex(long majorityIndex, long currentTerm, boolean isLeader) { try(AutoCloseableLock writeLock = tryWriteLock(TimeDuration.ONE_SECOND)) { final long oldCommittedIndex = getLastCommittedIndex(); @@ -155,6 +155,7 @@ protected void updateSnapshotIndexFromStateMachine() { } @Override + @SuppressWarnings("try") public void updateSnapshotIndex(long newSnapshotIndex) { try(AutoCloseableLock writeLock = writeLock()) { final long oldSnapshotIndex = getSnapshotIndex(); @@ -173,6 +174,7 @@ public final long append(long term, TransactionContext transaction) throws State return runner.runSequentially(() -> appendImpl(term, transaction)); } + @SuppressWarnings("try") private long appendImpl(long term, TransactionContext operation) throws StateMachineException { checkLogState(); try(AutoCloseableLock writeLock = writeLock()) { @@ -222,6 +224,7 @@ public final long appendMetadata(long term, long newCommitIndex) { return runner.runSequentially(() -> appendMetadataImpl(term, newCommitIndex)); } + @SuppressWarnings("try") private long appendMetadataImpl(long term, long newCommitIndex) { checkLogState(); if (!shouldAppendMetadata(newCommitIndex)) { @@ -254,6 +257,7 @@ public final long append(long term, RaftConfiguration configuration) { return runner.runSequentially(() -> appendImpl(term, configuration)); } + @SuppressWarnings("try") private long appendImpl(long term, RaftConfiguration newConf) { checkLogState(); try(AutoCloseableLock writeLock = writeLock()) { diff --git a/ratis-server/src/main/java/org/apache/ratis/server/raftlog/memory/MemoryRaftLog.java b/ratis-server/src/main/java/org/apache/ratis/server/raftlog/memory/MemoryRaftLog.java index f4b6dc452e..3579bb1f37 100644 --- a/ratis-server/src/main/java/org/apache/ratis/server/raftlog/memory/MemoryRaftLog.java +++ b/ratis-server/src/main/java/org/apache/ratis/server/raftlog/memory/MemoryRaftLog.java @@ -43,6 +43,7 @@ /** * A simple RaftLog implementation in memory. Used only for testing. */ +@SuppressWarnings({"deprecation", "try"}) public class MemoryRaftLog extends RaftLogBase { static class EntryList { private final List entries = new ArrayList<>(); diff --git a/ratis-server/src/main/java/org/apache/ratis/server/raftlog/segmented/LogSegment.java b/ratis-server/src/main/java/org/apache/ratis/server/raftlog/segmented/LogSegment.java index d0b356c6fd..6b1696b960 100644 --- a/ratis-server/src/main/java/org/apache/ratis/server/raftlog/segmented/LogSegment.java +++ b/ratis-server/src/main/java/org/apache/ratis/server/raftlog/segmented/LogSegment.java @@ -27,9 +27,11 @@ import org.apache.ratis.server.storage.RaftStorage; import org.apache.ratis.thirdparty.com.google.common.annotations.VisibleForTesting; import org.apache.ratis.thirdparty.com.google.protobuf.CodedOutputStream; +import org.apache.ratis.util.CodeInjectionForTesting; import org.apache.ratis.util.FileUtils; import org.apache.ratis.util.JavaUtils; import org.apache.ratis.util.Preconditions; +import org.apache.ratis.util.ReferenceCountedObject; import org.apache.ratis.util.SizeInBytes; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -48,7 +50,6 @@ import java.util.concurrent.atomic.AtomicLong; import java.util.concurrent.atomic.AtomicReference; import java.util.function.Consumer; -import org.apache.ratis.util.CodeInjectionForTesting; /** @@ -61,6 +62,8 @@ public final class LogSegment { static final Logger LOG = LoggerFactory.getLogger(LogSegment.class); + public static final String APPEND_RECORD = LogSegment.class.getSimpleName() + ".append"; + enum Op { LOAD_SEGMENT_FILE, REMOVE_CACHE, @@ -83,7 +86,7 @@ static long getEntrySize(LogEntryProto entry, Op op) { default: throw new IllegalStateException("Unexpected op " + op + ", entry=" + entry); } - final int serialized = e.getSerializedSize(); + final int serialized = entry.getSerializedSize(); return serialized + CodedOutputStream.computeUInt32SizeNoTag(serialized) + 4L; } @@ -168,7 +171,8 @@ static LogSegment newLogSegment(RaftStorage storage, LogSegmentStartEnd startEnd } public static int readSegmentFile(File file, LogSegmentStartEnd startEnd, SizeInBytes maxOpSize, - CorruptionPolicy corruptionPolicy, SegmentedRaftLogMetrics raftLogMetrics, Consumer entryConsumer) + CorruptionPolicy corruptionPolicy, SegmentedRaftLogMetrics raftLogMetrics, + Consumer> entryConsumer) throws IOException { int count = 0; try(SegmentedRaftLogInputStream in = new SegmentedRaftLogInputStream(file, startEnd, maxOpSize, raftLogMetrics)) { @@ -179,7 +183,8 @@ public static int readSegmentFile(File file, LogSegmentStartEnd startEnd, SizeIn } if (entryConsumer != null) { - entryConsumer.accept(next); + // TODO: use reference count to support zero buffer copying for readSegmentFile + entryConsumer.accept(ReferenceCountedObject.wrap(next)); } count++; } @@ -206,10 +211,7 @@ static LogSegment loadSegment(RaftStorage storage, File file, LogSegmentStartEnd final CorruptionPolicy corruptionPolicy = CorruptionPolicy.get(storage, RaftStorage::getLogCorruptionPolicy); final boolean isOpen = startEnd.isOpen(); final int entryCount = readSegmentFile(file, startEnd, maxOpSize, corruptionPolicy, raftLogMetrics, entry -> { - segment.append(keepEntryInCache || isOpen, entry, Op.LOAD_SEGMENT_FILE, true); - if (logConsumer != null) { - logConsumer.accept(entry); - } + segment.append(Op.LOAD_SEGMENT_FILE, entry, keepEntryInCache || isOpen, logConsumer); }); LOG.info("Successfully read {} entries from segment file {}", entryCount, file); @@ -397,7 +399,6 @@ File getFile() { private volatile boolean isOpen; private long totalFileSize = SegmentedRaftLogFormat.getHeaderLength(); - private AtomicLong totalCacheSize = new AtomicLong(0); /** Segment start index, inclusive. */ private final long startIndex; /** Segment end index, inclusive. */ @@ -415,7 +416,7 @@ File getFile() { /** * the entryCache caches the content of log entries. */ - private final Map entryCache = new ConcurrentHashMap<>(); + private final EntryCache entryCache = new EntryCache(); private LogSegment(RaftStorage storage, boolean isOpen, long start, long end, SizeInBytes maxOpSize, SegmentedRaftLogMetrics raftLogMetrics) { @@ -432,11 +433,7 @@ long getStartIndex() { } long getEndIndex() { - if (!isOpen) { - return endIndex; - } - final LogRecord last = records.getLast(); - return last == null ? getStartIndex() - 1 : last.getTermIndex().getIndex(); + return endIndex; } boolean isOpen() { @@ -444,16 +441,16 @@ boolean isOpen() { } int numOfEntries() { - return Math.toIntExact(getEndIndex() - startIndex + 1); + return Math.toIntExact(endIndex - startIndex + 1); } CorruptionPolicy getLogCorruptionPolicy() { return CorruptionPolicy.get(storage, RaftStorage::getLogCorruptionPolicy); } - void appendToOpenSegment(LogEntryProto entry, Op op, boolean verifyEntryIndex) { + void appendToOpenSegment(Op op, ReferenceCountedObject entryRef) { Preconditions.assertTrue(isOpen(), "The log segment %s is not open for append", this); - append(true, entry, op, verifyEntryIndex); + append(op, entryRef, true, null); } private void append(Op op, ReferenceCountedObject entryRef, @@ -463,6 +460,7 @@ private void append(Op op, ReferenceCountedObject entryRef, final LogRecord record = new LogRecord(totalFileSize, entry); if (keepEntryInCache) { putEntryCache(record.getTermIndex(), entryRef, op); + CodeInjectionForTesting.execute(APPEND_RECORD, this, record.getTermIndex()); } appendLogRecord(op, record); totalFileSize += getEntrySize(entry, op); @@ -534,7 +532,7 @@ long getTotalFileSize() { } long getTotalCacheSize() { - return totalCacheSize.get(); + return entryCache.size(); } /** @@ -592,29 +590,20 @@ void evictCache() { entryCache.evict(); } - void putEntryCache(TermIndex key, LogEntryProto value, Op op) { - final LogEntryProto previous = entryCache.put(key, value); - long previousSize = 0; - if (previous != null) { - // Different threads maybe load LogSegment file into cache at the same time, so duplicate maybe happen - previousSize = getEntrySize(value, Op.REMOVE_CACHE); - } - totalCacheSize.getAndAdd(getEntrySize(value, op) - previousSize); + void putEntryCache(TermIndex key, ReferenceCountedObject valueRef, Op op) { + entryCache.put(key, valueRef, op); } - void removeEntryCache(TermIndex key, Op op) { - LogEntryProto value = entryCache.remove(key); - if (value != null) { - totalCacheSize.getAndAdd(-getEntrySize(value, op)); - } + void removeEntryCache(TermIndex key) { + entryCache.remove(key); } boolean hasCache() { - return isOpen || !entryCache.isEmpty(); // open segment always has cache. + return isOpen || entryCache.size() > 0; // open segment always has cache. } boolean containsIndex(long index) { - return startIndex <= index && getEndIndex() >= index; + return startIndex <= index && endIndex >= index; } boolean hasEntries() { diff --git a/ratis-server/src/main/java/org/apache/ratis/server/raftlog/segmented/SegmentedRaftLog.java b/ratis-server/src/main/java/org/apache/ratis/server/raftlog/segmented/SegmentedRaftLog.java index 2c7fedbe0c..895531c897 100644 --- a/ratis-server/src/main/java/org/apache/ratis/server/raftlog/segmented/SegmentedRaftLog.java +++ b/ratis-server/src/main/java/org/apache/ratis/server/raftlog/segmented/SegmentedRaftLog.java @@ -42,6 +42,7 @@ import org.apache.ratis.util.AwaitToRun; import org.apache.ratis.util.JavaUtils; import org.apache.ratis.util.Preconditions; +import org.apache.ratis.util.ReferenceCountedObject; import org.apache.ratis.util.StringUtils; import java.io.File; @@ -212,7 +213,6 @@ public TransactionContext getTransactionContext(LogEntryProto entry, boolean cre private final long segmentMaxSize; private final boolean stateMachineCachingEnabled; private final SegmentedRaftLogMetrics metrics; - private final boolean readLockEnabled; @SuppressWarnings({"squid:S2095"}) // Suppress closeable warning private SegmentedRaftLog(Builder b) { @@ -228,12 +228,6 @@ private SegmentedRaftLog(Builder b) { this.fileLogWorker = new SegmentedRaftLogWorker(b.memberId, stateMachine, b.submitUpdateCommitEvent, b.server, storage, b.properties, getRaftLogMetrics()); stateMachineCachingEnabled = RaftServerConfigKeys.Log.StateMachineData.cachingEnabled(b.properties); - this.readLockEnabled = RaftServerConfigKeys.Log.readLockEnabled(b.properties); - } - - @Override - public AutoCloseableLock readLock() { - return readLockEnabled ? super.readLock() : null; } @Override @@ -287,6 +281,7 @@ private void loadLogSegments(long lastIndexInSnapshot, } @Override + @SuppressWarnings("deprecation") public LogEntryProto get(long index) throws RaftLogIOException { final ReferenceCountedObject ref = retainLog(index); if (ref == null) { @@ -330,6 +325,7 @@ public ReferenceCountedObject retainLog(long index) throws RaftLo } @Override + @SuppressWarnings("deprecation") public EntryWithData getEntryWithData(long index) throws RaftLogIOException { throw new UnsupportedOperationException("Use retainEntryWithData(" + index + ") instead."); } @@ -430,6 +426,7 @@ protected CompletableFuture appendEntryImpl(ReferenceCountedObject appendEntryImpl(ReferenceCountedObject appendEntryImpl(ReferenceCountedObject appendEntryTimerContext.stop()); - return writeFuture; + return write.getFuture().whenComplete((clientReply, exception) -> appendEntryTimerContext.stop()); } catch (Exception e) { LOG.error("{}: Failed to append {}", getName(), toLogEntryString(entry), e); throw e; diff --git a/ratis-server/src/main/java/org/apache/ratis/server/raftlog/segmented/SegmentedRaftLogCache.java b/ratis-server/src/main/java/org/apache/ratis/server/raftlog/segmented/SegmentedRaftLogCache.java index d9c9fde337..8d79c58d37 100644 --- a/ratis-server/src/main/java/org/apache/ratis/server/raftlog/segmented/SegmentedRaftLogCache.java +++ b/ratis-server/src/main/java/org/apache/ratis/server/raftlog/segmented/SegmentedRaftLogCache.java @@ -32,6 +32,7 @@ import org.apache.ratis.util.AutoCloseableReadWriteLock; import org.apache.ratis.util.JavaUtils; import org.apache.ratis.util.Preconditions; +import org.apache.ratis.util.ReferenceCountedObject; import org.apache.ratis.util.SizeInBytes; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -617,21 +618,14 @@ long getLastIndexInClosedSegments() { TermIndex getLastTermIndex() { try (AutoCloseableLock readLock = closedSegments.readLock()) { - LogSegment tmpSegment = openSegment; - return (tmpSegment != null && tmpSegment.getLastTermIndex() != null) ? - tmpSegment.getLastTermIndex() : + return (openSegment != null && openSegment.numOfEntries() > 0) ? + openSegment.getLastTermIndex() : (closedSegments.isEmpty() ? null : closedSegments.get(closedSegments.size() - 1).getLastTermIndex()); } } - void verifyAppendEntryIndex(LogEntryProto entry) { - // SegmentedRaftLog does the segment creation/rolling work. - Objects.requireNonNull(openSegment, "openSegment == null"); - openSegment.verifyEntryIndex(entry.getIndex()); - } - - void appendEntry(LogEntryProto entry, LogSegment.Op op) { + void appendEntry(LogSegment.Op op, ReferenceCountedObject entry) { // SegmentedRaftLog does the segment creation/rolling work. Here we just // simply append the entry into the open segment. Objects.requireNonNull(openSegment, "openSegment == null"); diff --git a/ratis-server/src/main/java/org/apache/ratis/server/raftlog/segmented/SegmentedRaftLogInputStream.java b/ratis-server/src/main/java/org/apache/ratis/server/raftlog/segmented/SegmentedRaftLogInputStream.java index c302d1f9aa..3cc8767fa4 100644 --- a/ratis-server/src/main/java/org/apache/ratis/server/raftlog/segmented/SegmentedRaftLogInputStream.java +++ b/ratis-server/src/main/java/org/apache/ratis/server/raftlog/segmented/SegmentedRaftLogInputStream.java @@ -83,7 +83,7 @@ private void init() throws IOException { state.open(); boolean initSuccess = false; try { - reader = SegmentedRaftLogReader.newInstance(logFile, maxOpSize, raftLogMetrics); + reader = new SegmentedRaftLogReader(logFile, maxOpSize, raftLogMetrics); initSuccess = reader.verifyHeader(); } finally { if (!initSuccess) { diff --git a/ratis-server/src/main/java/org/apache/ratis/server/raftlog/segmented/SegmentedRaftLogReader.java b/ratis-server/src/main/java/org/apache/ratis/server/raftlog/segmented/SegmentedRaftLogReader.java index b8c906366a..57baffb2fd 100644 --- a/ratis-server/src/main/java/org/apache/ratis/server/raftlog/segmented/SegmentedRaftLogReader.java +++ b/ratis-server/src/main/java/org/apache/ratis/server/raftlog/segmented/SegmentedRaftLogReader.java @@ -46,7 +46,7 @@ import java.util.Optional; import java.util.zip.Checksum; -final class SegmentedRaftLogReader implements Closeable { +class SegmentedRaftLogReader implements Closeable { static final Logger LOG = LoggerFactory.getLogger(SegmentedRaftLogReader.class); /** * InputStream wrapper that keeps track of the current stream position. @@ -150,18 +150,10 @@ public long skip(long amt) throws IOException { private final SegmentedRaftLogMetrics raftLogMetrics; private final SizeInBytes maxOpSize; - static SegmentedRaftLogReader newInstance(File file, SizeInBytes maxOpSize, SegmentedRaftLogMetrics raftLogMetrics) - throws IOException { - final LimitedInputStream limiter = new LimitedInputStream(new BufferedInputStream(FileUtils.newInputStream(file))); - final DataInputStream in = new DataInputStream(limiter); - return new SegmentedRaftLogReader(file, maxOpSize, raftLogMetrics, limiter, in); - } - - private SegmentedRaftLogReader(File file, SizeInBytes maxOpSize, SegmentedRaftLogMetrics raftLogMetrics, - LimitedInputStream limiter, DataInputStream in) { + SegmentedRaftLogReader(File file, SizeInBytes maxOpSize, SegmentedRaftLogMetrics raftLogMetrics) throws IOException { this.file = file; - this.limiter = limiter; - this.in = in; + this.limiter = new LimitedInputStream(new BufferedInputStream(FileUtils.newInputStream(file))); + in = new DataInputStream(limiter); checksum = new PureJavaCrc32C(); this.maxOpSize = maxOpSize; this.raftLogMetrics = raftLogMetrics; diff --git a/ratis-server/src/main/java/org/apache/ratis/server/raftlog/segmented/SegmentedRaftLogWorker.java b/ratis-server/src/main/java/org/apache/ratis/server/raftlog/segmented/SegmentedRaftLogWorker.java index e13d7105a1..3382eb9fae 100644 --- a/ratis-server/src/main/java/org/apache/ratis/server/raftlog/segmented/SegmentedRaftLogWorker.java +++ b/ratis-server/src/main/java/org/apache/ratis/server/raftlog/segmented/SegmentedRaftLogWorker.java @@ -484,9 +484,10 @@ private PurgeLog(TruncationSegments segments) { void execute() throws IOException { if (segments.getToDelete() != null) { try(UncheckedAutoCloseable ignored = raftLogMetrics.startPurgeTimer()) { - for (SegmentFileInfo fileInfo : segments.getToDelete()) { - final Path deleted = FileUtils.deleteFile(fileInfo.getFile(storage)); - LOG.info("{}: Purged RaftLog segment: info={}, path={}", name, fileInfo, deleted); + SegmentFileInfo[] toDeletes = segments.getToDelete(); + for (int i = toDeletes.length - 1; i >= 0; i--) { + final Path deleted = FileUtils.deleteFile(toDeletes[i].getFile(storage)); + LOG.info("{}: Purged RaftLog segment: info={}, path={}", name, toDeletes[i], deleted); } } } diff --git a/ratis-server/src/main/java/org/apache/ratis/server/storage/FileChunkReader.java b/ratis-server/src/main/java/org/apache/ratis/server/storage/FileChunkReader.java index 6c45412096..b80924eef1 100644 --- a/ratis-server/src/main/java/org/apache/ratis/server/storage/FileChunkReader.java +++ b/ratis-server/src/main/java/org/apache/ratis/server/storage/FileChunkReader.java @@ -34,7 +34,7 @@ import java.security.MessageDigest; /** Read {@link FileChunkProto}s from a file. */ -public final class FileChunkReader implements Closeable { +public class FileChunkReader implements Closeable { private final FileInfo info; private final Path relativePath; private final InputStream in; @@ -51,27 +51,17 @@ public final class FileChunkReader implements Closeable { * @param relativePath the relative path of the file. * @throws IOException if it failed to open the file. */ - public static FileChunkReader newInstance(FileInfo info, Path relativePath) throws IOException { + public FileChunkReader(FileInfo info, Path relativePath) throws IOException { + this.info = info; + this.relativePath = relativePath; final File f = info.getPath().toFile(); - final InputStream in; - final MessageDigest digester; - if (info.getFileDigest() == null) { digester = MD5FileUtil.newMD5(); - in = new DigestInputStream(FileUtils.newInputStream(f), digester); + this.in = new DigestInputStream(FileUtils.newInputStream(f), digester); } else { digester = null; - in = FileUtils.newInputStream(f); + this.in = FileUtils.newInputStream(f); } - - return new FileChunkReader(info, relativePath, in, digester); - } - - private FileChunkReader(FileInfo info, Path relativePath, InputStream in, MessageDigest digester) { - this.info = info; - this.relativePath = relativePath; - this.in = in; - this.digester = digester; } static ByteString readFileChunk(int chunkLength, InputStream in) throws IOException { diff --git a/ratis-server/src/main/java/org/apache/ratis/server/storage/SnapshotManager.java b/ratis-server/src/main/java/org/apache/ratis/server/storage/SnapshotManager.java index a96001b598..e4afdff03e 100644 --- a/ratis-server/src/main/java/org/apache/ratis/server/storage/SnapshotManager.java +++ b/ratis-server/src/main/java/org/apache/ratis/server/storage/SnapshotManager.java @@ -179,8 +179,8 @@ private static void rename(File tmpDir, File stateMachineDir) throws IOException try { moved = FileUtils.move(stateMachineDir, TMP + StringUtils.currentDateTime()); } catch(IOException e) { - LOG.warn("Failed to rename state machine directory {} to a " + TMP + " directory. Try deleting it directly.", - stateMachineDir.getAbsolutePath(), e); + LOG.warn("Failed to rename state machine directory " + stateMachineDir.getAbsolutePath() + + " to a " + TMP + " directory. Try deleting it directly.", e); FileUtils.deleteFully(stateMachineDir); } existingDir = moved; @@ -201,7 +201,7 @@ private static void rename(File tmpDir, File stateMachineDir) throws IOException try { FileUtils.deleteFully(existingDir); } catch (IOException e) { - LOG.warn("Failed to delete existing directory {}", existingDir.getAbsolutePath(), e); + LOG.warn("Failed to delete existing directory " + existingDir.getAbsolutePath(), e); } } } diff --git a/ratis-server/src/main/java/org/apache/ratis/server/util/ServerStringUtils.java b/ratis-server/src/main/java/org/apache/ratis/server/util/ServerStringUtils.java index d3c7a4bc70..3a5db62859 100644 --- a/ratis-server/src/main/java/org/apache/ratis/server/util/ServerStringUtils.java +++ b/ratis-server/src/main/java/org/apache/ratis/server/util/ServerStringUtils.java @@ -24,7 +24,9 @@ import org.apache.ratis.proto.RaftProtos.LogEntryProto; import org.apache.ratis.proto.RaftProtos.RequestVoteReplyProto; import org.apache.ratis.proto.RaftProtos.StateMachineLogEntryProto; +import org.apache.ratis.protocol.RaftGroupMemberId; import org.apache.ratis.server.protocol.TermIndex; +import org.apache.ratis.server.raftlog.LogProtoUtils; import org.apache.ratis.util.JavaUtils; import org.apache.ratis.util.ProtoUtils; @@ -85,7 +87,7 @@ public static String toInstallSnapshotRequestString(InstallSnapshotRequestProto s = "notify:" + TermIndex.valueOf(notification.getFirstAvailableTermIndex()); break; default: - throw new IllegalStateException("Unexpected InstallSnapshotRequestBodyCase in " + request); + throw new IllegalStateException("Unexpected body case in " + request); } return ProtoUtils.toString(request.getServerRequest()) + "-t" + request.getLeaderTerm() @@ -120,7 +122,11 @@ public static String toRequestVoteReplyString(RequestVoteReplyProto proto) { + "-last:" + TermIndex.valueOf(proto.getLastEntry()); } - /** Generate the unified name for the given member and class. */ + /** + * Used to generate the necessary unified name in the submodules under + * {@link org.apache.ratis.server.impl.RaftServerImpl}, which consists + * of {@link org.apache.ratis.server.impl.ServerState#memberId} and the specific class. + */ public static String generateUnifiedName(RaftGroupMemberId memberId, Class clazz) { return memberId + "-" + JavaUtils.getClassSimpleName(clazz); } diff --git a/ratis-server/src/main/java/org/apache/ratis/statemachine/impl/BaseStateMachine.java b/ratis-server/src/main/java/org/apache/ratis/statemachine/impl/BaseStateMachine.java index b97749f262..7ed1124295 100644 --- a/ratis-server/src/main/java/org/apache/ratis/statemachine/impl/BaseStateMachine.java +++ b/ratis-server/src/main/java/org/apache/ratis/statemachine/impl/BaseStateMachine.java @@ -42,7 +42,6 @@ import java.util.SortedMap; import java.util.TreeMap; import java.util.concurrent.CompletableFuture; -import java.util.concurrent.atomic.AtomicInteger; import java.util.concurrent.atomic.AtomicReference; /** @@ -50,9 +49,6 @@ */ public class BaseStateMachine implements StateMachine, StateMachine.DataApi, StateMachine.EventApi, StateMachine.LeaderEventApi, StateMachine.FollowerEventApi { - private static final AtomicInteger ID_GENERATOR = new AtomicInteger(); - - private final int id = ID_GENERATOR.incrementAndGet(); private final CompletableFuture server = new CompletableFuture<>(); @SuppressWarnings({"squid:S3077"}) // Suppress volatile for generic type private volatile RaftGroupId groupId; @@ -230,7 +226,7 @@ public void close() throws IOException { @Override public String toString() { - return JavaUtils.getClassSimpleName(getClass()) + "-" + id + ":" + return JavaUtils.getClassSimpleName(getClass()) + ":" + (!server.isDone()? "uninitialized": getId() + ":" + groupId); } } diff --git a/ratis-server/src/main/java/org/apache/ratis/statemachine/impl/SimpleStateMachineStorage.java b/ratis-server/src/main/java/org/apache/ratis/statemachine/impl/SimpleStateMachineStorage.java index 5322778995..7e8afbaa85 100644 --- a/ratis-server/src/main/java/org/apache/ratis/statemachine/impl/SimpleStateMachineStorage.java +++ b/ratis-server/src/main/java/org/apache/ratis/statemachine/impl/SimpleStateMachineStorage.java @@ -39,6 +39,7 @@ import java.nio.file.Path; import java.util.ArrayList; import java.util.Comparator; +import java.util.Iterator; import java.util.List; import java.util.Objects; import java.util.Optional; @@ -81,9 +82,6 @@ public void format() throws IOException { // TODO } - /** - * Fetch all the snapshot files irrespective of whether they have an MD5 file or not - */ static List getSingleFileSnapshotInfos(Path dir) throws IOException { final List infos = new ArrayList<>(); try (DirectoryStream stream = Files.newDirectoryStream(dir)) { @@ -94,8 +92,7 @@ static List getSingleFileSnapshotInfos(Path dir) throws if (matcher.matches()) { final long term = Long.parseLong(matcher.group(1)); final long index = Long.parseLong(matcher.group(2)); - final MD5Hash md5 = MD5FileUtil.readStoredMd5ForFile(path.toFile()); - final FileInfo fileInfo = new FileInfo(path, md5); + final FileInfo fileInfo = new FileInfo(path, null); //No FileDigest here. infos.add(new SingleFileSnapshotInfo(fileInfo, term, index)); } } @@ -118,25 +115,10 @@ public void cleanupOldSnapshots(SnapshotRetentionPolicy snapshotRetentionPolicy) } final List allSnapshotFiles = getSingleFileSnapshotInfos(stateMachineDir.toPath()); - allSnapshotFiles.sort(Comparator.comparing(SingleFileSnapshotInfo::getIndex).reversed()); - int numSnapshotsWithMd5 = 0; - int deleteIdx = -1; - for (int i = 0; i < allSnapshotFiles.size(); i++) { - final SingleFileSnapshotInfo snapshot = allSnapshotFiles.get(i); - if (snapshot.hasMd5()) { - if (++numSnapshotsWithMd5 == numSnapshotsRetained) { - // We have found the last snapshot with an MD5 file that needs to be retained - deleteIdx = i + 1; - break; - } - } else { - LOG.warn("Snapshot file {} has missing MD5 file.", snapshot); - } - } - - if (deleteIdx > 0) { - allSnapshotFiles.subList(deleteIdx, allSnapshotFiles.size()) + if (allSnapshotFiles.size() > numSnapshotsRetained) { + allSnapshotFiles.sort(Comparator.comparing(SingleFileSnapshotInfo::getIndex).reversed()); + allSnapshotFiles.subList(numSnapshotsRetained, allSnapshotFiles.size()) .stream() .map(SingleFileSnapshotInfo::getFile) .map(FileInfo::getPath) @@ -144,21 +126,20 @@ public void cleanupOldSnapshots(SnapshotRetentionPolicy snapshotRetentionPolicy) LOG.info("Deleting old snapshot at {}", snapshotPath.toAbsolutePath()); FileUtils.deletePathQuietly(snapshotPath); }); - } - - // clean up the md5 files if the corresponding snapshot file does not exist - try (DirectoryStream stream = Files.newDirectoryStream(stateMachineDir.toPath(), - SNAPSHOT_MD5_FILTER)) { - for (Path md5path : stream) { - Path md5FileNamePath = md5path.getFileName(); - if (md5FileNamePath == null) { - continue; - } - final String md5FileName = md5FileNamePath.toString(); - final File snapshotFile = new File(stateMachineDir, - md5FileName.substring(0, md5FileName.length() - MD5_SUFFIX.length())); - if (!snapshotFile.exists()) { - FileUtils.deletePathQuietly(md5path); + // clean up the md5 files if the corresponding snapshot file does not exist + try (DirectoryStream stream = Files.newDirectoryStream(stateMachineDir.toPath(), + SNAPSHOT_MD5_FILTER)) { + for (Path md5path : stream) { + Path md5FileNamePath = md5path.getFileName(); + if (md5FileNamePath == null) { + continue; + } + final String md5FileName = md5FileNamePath.toString(); + final File snapshotFile = new File(stateMachineDir, + md5FileName.substring(0, md5FileName.length() - MD5_SUFFIX.length())); + if (!snapshotFile.exists()) { + FileUtils.deletePathQuietly(md5path); + } } } } @@ -201,19 +182,24 @@ protected File getCorruptSnapshotFile(long term, long endIndex) { } static SingleFileSnapshotInfo findLatestSnapshot(Path dir) throws IOException { - final List infos = getSingleFileSnapshotInfos(dir); - if (infos.isEmpty()) { + final Iterator i = getSingleFileSnapshotInfos(dir).iterator(); + if (!i.hasNext()) { return null; } - infos.sort(Comparator.comparing(SingleFileSnapshotInfo::getIndex).reversed()); - for (SingleFileSnapshotInfo latest : infos) { - if (latest.hasMd5()) { - return latest; + SingleFileSnapshotInfo latest = i.next(); + for(; i.hasNext(); ) { + final SingleFileSnapshotInfo info = i.next(); + if (info.getIndex() > latest.getIndex()) { + latest = info; } } - return infos.get(0); // all snapshots do not have MD5 + // read md5 + final Path path = latest.getFile().getPath(); + final MD5Hash md5 = MD5FileUtil.readStoredMd5ForFile(path.toFile()); + final FileInfo info = new FileInfo(path, md5); + return new SingleFileSnapshotInfo(info, latest.getTerm(), latest.getIndex()); } public SingleFileSnapshotInfo updateLatestSnapshot(SingleFileSnapshotInfo info) { @@ -240,12 +226,8 @@ public SingleFileSnapshotInfo loadLatestSnapshot() { return null; } try { - final SingleFileSnapshotInfo latest = updateLatestSnapshot(findLatestSnapshot(dir.toPath())); - LOG.debug("Latest snapshot is {} in {}", latest, dir); - return latest; - } catch (IOException e) { - LOG.warn("Failed to updateLatestSnapshot from {}", dir, e); - FileUtils.listDir(dir, s -> LOG.warn(" {}", s), LOG::error); + return updateLatestSnapshot(findLatestSnapshot(dir.toPath())); + } catch (IOException ignored) { return null; } } diff --git a/ratis-server/src/main/java/org/apache/ratis/statemachine/impl/SingleFileSnapshotInfo.java b/ratis-server/src/main/java/org/apache/ratis/statemachine/impl/SingleFileSnapshotInfo.java index 5ecc59a102..14d501a4af 100644 --- a/ratis-server/src/main/java/org/apache/ratis/statemachine/impl/SingleFileSnapshotInfo.java +++ b/ratis-server/src/main/java/org/apache/ratis/statemachine/impl/SingleFileSnapshotInfo.java @@ -36,11 +36,6 @@ public SingleFileSnapshotInfo(FileInfo fileInfo, long term, long endIndex) { this(fileInfo, TermIndex.valueOf(term, endIndex)); } - /** @return true iff the MD5 exists. */ - public boolean hasMd5() { - return getFile().getFileDigest() != null; - } - /** @return the file associated with the snapshot. */ public FileInfo getFile() { return getFiles().get(0); diff --git a/ratis-server/src/main/java/org/apache/ratis/statemachine/impl/TransactionContextImpl.java b/ratis-server/src/main/java/org/apache/ratis/statemachine/impl/TransactionContextImpl.java index 18ed297f87..58869f5edc 100644 --- a/ratis-server/src/main/java/org/apache/ratis/statemachine/impl/TransactionContextImpl.java +++ b/ratis-server/src/main/java/org/apache/ratis/statemachine/impl/TransactionContextImpl.java @@ -27,6 +27,7 @@ import org.apache.ratis.thirdparty.com.google.protobuf.ByteString; import org.apache.ratis.util.MemoizedSupplier; import org.apache.ratis.util.Preconditions; +import org.apache.ratis.util.ReferenceCountedObject; import java.io.IOException; import java.util.Objects; @@ -152,6 +153,7 @@ public ReferenceCountedObject wrap(LogEntryProto entry) { } @Override + @SuppressWarnings("deprecation") public StateMachineLogEntryProto getStateMachineLogEntry() { return stateMachineLogEntry; } @@ -194,6 +196,7 @@ private LogEntryProto setLogEntry(LogEntryProto entry) { @Override + @SuppressWarnings("deprecation") public LogEntryProto getLogEntry() { return logEntryCopy == null ? null : logEntryCopy.get(); } @@ -229,6 +232,8 @@ public TransactionContext preAppendTransaction() throws IOException { @Override public TransactionContext cancelTransaction() throws IOException { + // TODO: This is not called from Raft server / log yet. When an IOException happens, we should + // call this to let the SM know that Transaction cannot be synced return stateMachine.cancelTransaction(this); } diff --git a/ratis-server/src/test/java/org/apache/ratis/InstallSnapshotFromLeaderTests.java b/ratis-server/src/test/java/org/apache/ratis/InstallSnapshotFromLeaderTests.java index f873e5dfdc..e7683a3991 100644 --- a/ratis-server/src/test/java/org/apache/ratis/InstallSnapshotFromLeaderTests.java +++ b/ratis-server/src/test/java/org/apache/ratis/InstallSnapshotFromLeaderTests.java @@ -102,18 +102,15 @@ private void testMultiFileInstallSnapshot(CLUSTER cluster) throws Exception { RaftTestUtil.waitForLeader(cluster); final RaftPeerId leaderId = cluster.getLeader().getId(); - try (final RaftClient client = cluster.createClient(leaderId)) { - for (; i < SNAPSHOT_TRIGGER_THRESHOLD * 2 - 1; i++) { - RaftClientReply - reply = client.io().send(new RaftTestUtil.SimpleMessage("m" + i)); - Assertions.assertTrue(reply.isSuccess()); - } - - client.getSnapshotManagementApi(leaderId).create(3000); + try (final RaftClient client = cluster.createClient(leaderId)) { + for (; i < SNAPSHOT_TRIGGER_THRESHOLD * 2 - 1; i++) { + RaftClientReply + reply = client.io().send(new RaftTestUtil.SimpleMessage("m" + i)); + Assertions.assertTrue(reply.isSuccess()); } - final SnapshotInfo snapshot = cluster.getLeader().getStateMachine().getLatestSnapshot(); - Assertions.assertEquals(3, snapshot.getFiles().size()); + client.getSnapshotManagementApi(leaderId).create(3000); + } final SnapshotInfo snapshot = cluster.getLeader().getStateMachine().getLatestSnapshot(); Assertions.assertEquals(3, snapshot.getFiles().size()); @@ -123,17 +120,75 @@ private void testMultiFileInstallSnapshot(CLUSTER cluster) throws Exception { // trigger setConfiguration cluster.setConfiguration(change.getPeersInNewConf()); - // Check the installed snapshot file number on each Follower matches with the - // leader snapshot. - JavaUtils.attempt(() -> { - for (RaftServer.Division follower : cluster.getFollowers()) { - final SnapshotInfo info = follower.getStateMachine().getLatestSnapshot(); - Assertions.assertNotNull(info); - Assertions.assertEquals(3, info.getFiles().size()); - } - }, 10, ONE_SECOND, "check snapshot", LOG); - } finally { - cluster.shutdown(); + RaftServerTestUtil.waitAndCheckNewConf(cluster, change.getPeersInNewConf(), 0, null); + + // Check the installed snapshot file number on each Follower matches with the + // leader snapshot. + JavaUtils.attempt(() -> { + for (RaftServer.Division follower : cluster.getFollowers()) { + final SnapshotInfo info = follower.getStateMachine().getLatestSnapshot(); + Assertions.assertNotNull(info); + Assertions.assertEquals(3, info.getFiles().size()); + } + }, 10, ONE_SECOND, "check snapshot", LOG); + } + + private void testInstallSnapshotDuringLeaderSwitch(CLUSTER cluster) throws Exception { + RaftTestUtil.waitForLeader(cluster); + final RaftPeerId leaderId = cluster.getLeader().getId(); + + // perform operations and force all peers to take snapshot + try (final RaftClient client = cluster.createClient(leaderId)) { + for (int i = 0; i < SNAPSHOT_TRIGGER_THRESHOLD * 2; i++) { + final RaftClientReply + reply = client.io().send(new RaftTestUtil.SimpleMessage("m" + i)); + Assertions.assertTrue(reply.isSuccess()); + } + + for (final RaftPeer peer: cluster.getPeers()) { + final RaftClientReply snapshotReply = client.getSnapshotManagementApi(leaderId).create(3000); + Assertions.assertTrue(snapshotReply.isSuccess()); + } + } + final SnapshotInfo snapshot = cluster.getLeader().getStateMachine().getLatestSnapshot(); + Assertions.assertNotNull(snapshot); + + // isolate two followers (majority) in old configuration + final List oldFollowers = cluster.getFollowers(); + for (RaftServer.Division f: oldFollowers) { + RaftTestUtil.isolate(cluster, f.getId()); + } + + // add two more peers and install snapshot from leaders + final PeerChanges change = cluster.addNewPeers(2, true); + try (final RaftClient client = cluster.createClient(leaderId, RetryPolicies.noRetry())) { + final RaftException e = Assertions.assertThrows(RaftException.class, + () -> client.admin().setConfiguration(change.getPeersInNewConf())); + Assertions.assertTrue( e instanceof RaftRetryFailureException + || e instanceof ReconfigurationTimeoutException, + () -> "Unexpected exception: " + e); + } + + final SnapshotInfo snapshotInfo = cluster.getDivision(change.getAddedPeers().get(0).getId()) + .getStateMachine().getLatestSnapshot(); + Assertions.assertNotNull(snapshotInfo); + + // recover the old followers and isolate the leader to force leader switch + RaftTestUtil.isolate(cluster, leaderId); + for (RaftServer.Division f: oldFollowers) { + RaftTestUtil.deIsolate(cluster, f.getId()); + } + RaftTestUtil.waitForLeader(cluster); + + try (final RaftClient client = cluster.createClient(cluster.getLeader().getId())) { + // successfully setConfiguration during leader switch + final RaftClientReply setConf = client.admin().setConfiguration(change.getPeersInNewConf()); + Assertions.assertTrue(setConf.isSuccess()); + + RaftTestUtil.deIsolate(cluster, leaderId); + final RaftClientReply + reply = client.io().send(new RaftTestUtil.SimpleMessage("final")); + Assertions.assertTrue(reply.isSuccess()); } } diff --git a/ratis-server/src/test/java/org/apache/ratis/InstallSnapshotNotificationTests.java b/ratis-server/src/test/java/org/apache/ratis/InstallSnapshotNotificationTests.java index a8558287ff..931bf6317f 100644 --- a/ratis-server/src/test/java/org/apache/ratis/InstallSnapshotNotificationTests.java +++ b/ratis-server/src/test/java/org/apache/ratis/InstallSnapshotNotificationTests.java @@ -408,7 +408,7 @@ private void testInstallSnapshotNotificationCount(CLUSTER cluster) throws Except } // Make sure each new peer got one snapshot notification. - Assertions.assertEquals(2, numSnapshotRequests.get()); + Assertions.assertEquals(numNewPeers, numSnapshotRequests.get()); } finally { cluster.shutdown(); @@ -565,7 +565,7 @@ private void testInstallSnapshotDuringBootstrap(CLUSTER cluster) throws Exceptio } // Make sure each new peer got at least one snapshot notification. - Assertions.assertTrue(2 <= numSnapshotRequests.get()); + Assertions.assertTrue(numNewPeers <= numSnapshotRequests.get()); } finally { cluster.shutdown(); } diff --git a/ratis-server/src/test/java/org/apache/ratis/LogAppenderTests.java b/ratis-server/src/test/java/org/apache/ratis/LogAppenderTests.java index e5d0ee0ef8..8a8731daf4 100644 --- a/ratis-server/src/test/java/org/apache/ratis/LogAppenderTests.java +++ b/ratis-server/src/test/java/org/apache/ratis/LogAppenderTests.java @@ -170,7 +170,6 @@ public void testFollowerHeartbeatMetric() throws IOException, InterruptedExcepti assertTrue(t.getTimer().getMeanRate() > 0.0d); assertTrue(t.getTimer().getCount() > 0L); } - cluster.shutdown(); } cluster.shutdown(); } @@ -226,4 +225,94 @@ void runTest(CLUSTER cluster) throws Exception { Assertions.assertNotNull(last); Assertions.assertTrue(last.getIndex() <= leader.getInfo().getLastAppliedIndex()); } + + @Test + public void testNewAppendEntriesRequestAfterPurgeFollowerBehindStartIndex() throws Exception { + final RaftProperties prop = getProperties(); + RaftServerConfigKeys.Log.setPurgeGap(prop, 1); + RaftServerConfigKeys.Log.setSegmentSizeMax(prop, SizeInBytes.valueOf("1KB")); + runWithNewCluster(3, cluster -> { + final long startIndexAfterPurge = setupPurgedLeaderLog(cluster); + // Test when followerNextIndex < leader's logStartIndex + runTestNewAppendEntriesRequestAfterPurge(cluster, startIndexAfterPurge - 1); + }); + } + + @Test + public void testNewAppendEntriesRequestAfterPurgeFollowerAtStartIndex() throws Exception { + final RaftProperties prop = getProperties(); + RaftServerConfigKeys.Log.setPurgeGap(prop, 1); + RaftServerConfigKeys.Log.setSegmentSizeMax(prop, SizeInBytes.valueOf("1KB")); + runWithNewCluster(3, cluster -> { + final long startIndexAfterPurge = setupPurgedLeaderLog(cluster); + // Test when followerNextIndex == leader's logStartIndex, but the previous index is already purged + runTestNewAppendEntriesRequestAfterPurge(cluster, startIndexAfterPurge); + }); + } + + private long setupPurgedLeaderLog(CLUSTER cluster) throws Exception { + final RaftServer.Division leader = waitForLeader(cluster); + final RaftLog leaderLog = leader.getRaftLog(); + + try (RaftClient client = cluster.createClient(leader.getId())) { + for (SimpleMessage msg : generateMsgs(5)) { + client.io().send(msg); + } + } + + final long lastLogIndex = leaderLog.getLastEntryTermIndex().getIndex(); + LOG.info("Leader log lastIndex={}, startIndex={}", lastLogIndex, leaderLog.getStartIndex()); + Assertions.assertTrue(lastLogIndex > 5, "Need enough log entries for the test"); + + // Take a snapshot so that shouldInstallSnapshot() can return it + final long snapshotIndex = SimpleStateMachine4Testing.get(leader).takeSnapshot(); + LOG.info("Snapshot taken at index {}", snapshotIndex); + Assertions.assertTrue(snapshotIndex > 0, "Snapshot should have been taken"); + + final long purgeUpTo = lastLogIndex - 2; + LOG.info("Purging leader log up to index {}", purgeUpTo); + leaderLog.purge(purgeUpTo).get(); + + final long startIndexAfterPurge = leaderLog.getStartIndex(); + LOG.info("Leader log after purge: startIndex={}", startIndexAfterPurge); + Assertions.assertTrue(startIndexAfterPurge > 1, + "Purge should have advanced startIndex, but got " + startIndexAfterPurge); + + return startIndexAfterPurge; + } + + void runTestNewAppendEntriesRequestAfterPurge(CLUSTER cluster, + long targetNextIndex) throws Exception { + final RaftServer.Division leader = waitForLeader(cluster); + final RaftLog leaderLog = leader.getRaftLog(); + final long startIndexAfterPurge = leaderLog.getStartIndex(); + + final Stream appenders = RaftServerTestUtil.getLogAppenders(leader); + Assertions.assertNotNull(appenders, "Leader should have log appenders"); + final LogAppender appender = appenders.findFirst().orElseThrow( + () -> new AssertionError("No log appender found")); + + Assertions.assertTrue(targetNextIndex > RaftLog.LEAST_VALID_LOG_INDEX, + "targetNextIndex should be > LEAST_VALID_LOG_INDEX"); + appender.getFollower().setNextIndex(targetNextIndex); + + LOG.info("Set follower nextIndex={}, startIndexAfterPurge={}, snapshotIndex={}", + targetNextIndex, startIndexAfterPurge, appender.getFollower().getSnapshotIndex()); + Assertions.assertEquals(0, appender.getFollower().getSnapshotIndex(), + "Follower snapshotIndex should be 0 (default, never installed snapshot)"); + + Assertions.assertNull(leaderLog.getTermIndex(targetNextIndex - 1), + "Entry at previousIndex=" + (targetNextIndex - 1) + " should have been purged"); + + // Should return null instead of throwing NPE + Assertions.assertNull(appender.newAppendEntriesRequest(0, false), + "newAppendEntriesRequest should return null when previous TermIndex is not found"); + + Assertions.assertEquals(targetNextIndex, appender.getFollower().getNextIndex(), + "Follower nextIndex should remain unchanged"); + + Assertions.assertNotNull(appender.shouldInstallSnapshot(), + "shouldInstallSnapshot should return non-null when followerNextIndex (" + + targetNextIndex + ") and previous entry has been purged"); + } } diff --git a/ratis-server/src/test/java/org/apache/ratis/OutputStreamBaseTest.java b/ratis-server/src/test/java/org/apache/ratis/OutputStreamBaseTest.java index 9ce54b6f54..9821126ce6 100644 --- a/ratis-server/src/test/java/org/apache/ratis/OutputStreamBaseTest.java +++ b/ratis-server/src/test/java/org/apache/ratis/OutputStreamBaseTest.java @@ -303,6 +303,7 @@ private void runTestKillLeader(CLUSTER cluster) throws Exception { final boolean latchCompleted = latch.await(5, TimeUnit.SECONDS); Assertions.assertTrue(latchCompleted, "Writer thread did not finish within the timeout"); LOG.info("Writer success? " + success.get()); + Assertions.assertNotNull(success.get(), "Writer thread completed but success was not set"); Assertions.assertTrue(success.get()); // total number of tx should be >= result + 2, where 2 means two NoOp from // leaders. It may be larger than result+2 because the client may resend diff --git a/ratis-server/src/test/java/org/apache/ratis/RaftBasicTests.java b/ratis-server/src/test/java/org/apache/ratis/RaftBasicTests.java index 045667759c..c71b57e826 100644 --- a/ratis-server/src/test/java/org/apache/ratis/RaftBasicTests.java +++ b/ratis-server/src/test/java/org/apache/ratis/RaftBasicTests.java @@ -34,17 +34,21 @@ import org.apache.ratis.server.RaftServerConfigKeys; import org.apache.ratis.server.impl.BlockRequestHandlingInjection; import org.apache.ratis.server.impl.MiniRaftCluster; +import org.apache.ratis.server.impl.RaftServerTestUtil; import org.apache.ratis.server.impl.RetryCacheTestUtil; import org.apache.ratis.server.metrics.ServerMetricsTestUtils; import org.apache.ratis.server.raftlog.RaftLog; import org.apache.ratis.util.ExitUtils; import org.apache.ratis.util.JavaUtils; +import org.apache.ratis.util.Slf4jUtils; import org.apache.ratis.util.TimeDuration; import org.apache.ratis.util.Timestamp; import org.junit.jupiter.api.Assertions; import org.junit.jupiter.api.Assumptions; import org.junit.jupiter.api.Test; +import org.junit.jupiter.api.Timeout; import org.slf4j.Logger; +import org.slf4j.event.Level; import java.io.IOException; import java.util.List; @@ -72,6 +76,9 @@ public abstract class RaftBasicTests extends BaseTest implements MiniRaftCluster.Factory.Get { { + Slf4jUtils.setLogLevel(RaftServer.Division.LOG, Level.DEBUG); + RaftServerTestUtil.setStateMachineUpdaterLogLevel(Level.DEBUG); + RaftServerConfigKeys.RetryCache.setExpiryTime(getProperties(), TimeDuration.valueOf(5, TimeUnit.SECONDS)); } diff --git a/ratis-server/src/test/java/org/apache/ratis/RaftExceptionBaseTest.java b/ratis-server/src/test/java/org/apache/ratis/RaftExceptionBaseTest.java index 53222cd6b8..b06d6e904a 100644 --- a/ratis-server/src/test/java/org/apache/ratis/RaftExceptionBaseTest.java +++ b/ratis-server/src/test/java/org/apache/ratis/RaftExceptionBaseTest.java @@ -114,7 +114,7 @@ void runTestNotLeaderExceptionWithReconf(CLUSTER cluster) throws Exception { // trigger setConfiguration LOG.info("Start changing the configuration: {}", change.getPeersInNewConf()); try (final RaftClient c2 = cluster.createClient(newLeader)) { - RaftClientReply reply = c2.admin().setConfiguration(change.allPeersInNewConf); + RaftClientReply reply = c2.admin().setConfiguration(change.getPeersInNewConf()); Assertions.assertTrue(reply.isSuccess()); } LOG.info(cluster.printServers()); @@ -138,7 +138,7 @@ void runTestNotLeaderExceptionWithReconf(CLUSTER cluster) throws Exception { @Test public void testGroupMismatchException() throws Exception { - runWithNewCluster(NUM_PEERS, this::runTestGroupMismatchException); + runWithSameCluster(NUM_PEERS, this::runTestGroupMismatchException); } void runTestGroupMismatchException(CLUSTER cluster) throws Exception { @@ -171,7 +171,7 @@ void runTestGroupMismatchException(CLUSTER cluster) throws Exception { @Test public void testStaleReadException() throws Exception { - runWithNewCluster(NUM_PEERS, this::runTestStaleReadException); + runWithSameCluster(NUM_PEERS, this::runTestStaleReadException); } void runTestStaleReadException(CLUSTER cluster) throws Exception { @@ -186,7 +186,7 @@ void runTestStaleReadException(CLUSTER cluster) throws Exception { @Test public void testLogAppenderBufferCapacity() throws Exception { - runWithNewCluster(NUM_PEERS, this::runTestLogAppenderBufferCapacity); + runWithSameCluster(NUM_PEERS, this::runTestLogAppenderBufferCapacity); } void runTestLogAppenderBufferCapacity(CLUSTER cluster) throws Exception { diff --git a/ratis-server/src/test/java/org/apache/ratis/ReadOnlyRequestTests.java b/ratis-server/src/test/java/org/apache/ratis/ReadOnlyRequestTests.java index 573434a1fb..94e9433b15 100644 --- a/ratis-server/src/test/java/org/apache/ratis/ReadOnlyRequestTests.java +++ b/ratis-server/src/test/java/org/apache/ratis/ReadOnlyRequestTests.java @@ -19,7 +19,7 @@ import org.apache.ratis.client.RaftClient; import org.apache.ratis.conf.RaftProperties; -import org.apache.ratis.proto.RaftProtos; +import org.apache.ratis.proto.RaftProtos.LogEntryProto; import org.apache.ratis.protocol.Message; import org.apache.ratis.protocol.RaftClientReply; import org.apache.ratis.protocol.RaftPeerId; @@ -35,7 +35,6 @@ import org.apache.ratis.statemachine.TransactionContext; import org.apache.ratis.statemachine.impl.BaseStateMachine; import org.apache.ratis.util.Slf4jUtils; -import org.apache.ratis.util.TimeDuration; import org.junit.jupiter.api.Assertions; import org.junit.jupiter.api.BeforeEach; import org.junit.jupiter.api.Test; @@ -59,6 +58,10 @@ public abstract class ReadOnlyRequestTests static final String WAIT_AND_INCREMENT_STRING = "WAIT_AND_INCREMENT"; static final String QUERY_STRING = "QUERY"; + public static final Message INCREMENT = new RaftTestUtil.SimpleMessage(INCREMENT_STRING); + public static final Message WAIT_AND_INCREMENT = new RaftTestUtil.SimpleMessage(WAIT_AND_INCREMENT_STRING); + public static final Message QUERY = new RaftTestUtil.SimpleMessage(QUERY_STRING); + @BeforeEach public void setup() { final RaftProperties p = getProperties(); @@ -83,10 +86,8 @@ static void runTestReadOnly(C cluster) throws Except try (final RaftClient client = cluster.createClient(leaderId)) { for (int i = 1; i <= 10; i++) { - RaftClientReply reply = client.io().send(incrementMessage); - Assertions.assertTrue(reply.isSuccess()); - reply = client.io().sendReadOnly(queryMessage); - Assertions.assertEquals(i, retrieve(reply)); + assertReplyExact(i, client.io().send(INCREMENT)); + assertReplyExact(i, client.io().sendReadOnly(QUERY)); } } } finally { @@ -108,18 +109,12 @@ static void runTestReadTimeout(Class result = client.async().send(incrementMessage); - client.admin().transferLeadership(null, 200); - - Assertions.assertThrows(ReadIndexException.class, () -> { - RaftClientReply timeoutReply = noRetry.io().sendReadOnly(queryMessage); - Assertions.assertNotNull(timeoutReply.getException()); - Assertions.assertTrue(timeoutReply.getException() instanceof ReadException); - }); - } - - } finally { - cluster.shutdown(); + Assertions.assertThrows(exceptionClass, () -> { + final RaftClientReply timeoutReply = noRetry.io().sendReadOnly(QUERY); + Assertions.assertFalse(timeoutReply.isSuccess()); + Assertions.assertNotNull(timeoutReply.getException()); + Assertions.assertInstanceOf(ReadException.class, timeoutReply.getException()); + }); } } @@ -132,171 +127,15 @@ static void runTestReadOnlyRetryWhenLeaderDown(Retry throws Exception { final RaftPeerId leaderId = RaftTestUtil.waitForLeader(cluster).getId(); - private void testFollowerReadOnlyImpl(CLUSTER cluster) throws Exception { - try { - RaftTestUtil.waitForLeader(cluster); - - List followers = cluster.getFollowers(); - Assertions.assertEquals(2, followers.size()); - - final RaftPeerId f0 = followers.get(0).getId(); - final RaftPeerId f1 = followers.get(1).getId(); - try (RaftClient client = cluster.createClient(cluster.getLeader().getId())) { - for (int i = 1; i <= 10; i++) { - final RaftClientReply reply = client.io().send(incrementMessage); - Assertions.assertTrue(reply.isSuccess()); - final RaftClientReply read1 = client.io().sendReadOnly(queryMessage, f0); - Assertions.assertEquals(i, retrieve(read1)); - final CompletableFuture read2 = client.async().sendReadOnly(queryMessage, f1); - Assertions.assertEquals(i, retrieve(read2.get(1, TimeUnit.SECONDS))); - } - } - } finally { - cluster.shutdown(); - } - } - - @Test - public void testFollowerLinearizableReadParallel() throws Exception { - getProperties().setEnum(RaftServerConfigKeys.Read.OPTION_KEY, RaftServerConfigKeys.Read.Option.LINEARIZABLE); - runWithNewCluster(NUM_SERVERS, this::testFollowerReadOnlyParallelImpl); - } - - @Test - public void testFollowerLeaseReadParallel() throws Exception { - getProperties().setBoolean(RaftServerConfigKeys.Read.LEADER_LEASE_ENABLED_KEY, true); - runWithNewCluster(NUM_SERVERS, this::testFollowerReadOnlyParallelImpl); - } - - private void testFollowerReadOnlyParallelImpl(CLUSTER cluster) throws Exception { - try { - RaftTestUtil.waitForLeader(cluster); - - List followers = cluster.getFollowers(); - Assertions.assertEquals(2, followers.size()); - - try (RaftClient leaderClient = cluster.createClient(cluster.getLeader().getId()); - RaftClient followerClient1 = cluster.createClient(followers.get(0).getId())) { - - leaderClient.io().send(incrementMessage); - leaderClient.async().send(waitAndIncrementMessage); - Thread.sleep(100); - - RaftClientReply clientReply = followerClient1.io().sendReadOnly(queryMessage, followers.get(0).getId()); - Assertions.assertEquals(2, retrieve(clientReply)); - } - - } finally { - cluster.shutdown(); - } - } - - @Test - public void testFollowerLinearizableReadFailWhenLeaderDown() throws Exception { - getProperties().setEnum(RaftServerConfigKeys.Read.OPTION_KEY, RaftServerConfigKeys.Read.Option.LINEARIZABLE); - runWithNewCluster(NUM_SERVERS, this::testFollowerReadOnlyFailWhenLeaderDownImpl); - } - - @Test - public void testFollowerLeaseReadWhenLeaderDown() throws Exception { - getProperties().setBoolean(RaftServerConfigKeys.Read.LEADER_LEASE_ENABLED_KEY, true); - runWithNewCluster(NUM_SERVERS, this::testFollowerReadOnlyFailWhenLeaderDownImpl); - } - - private void testFollowerReadOnlyFailWhenLeaderDownImpl(CLUSTER cluster) throws Exception { - try { - RaftTestUtil.waitForLeader(cluster); - - List followers = cluster.getFollowers(); - Assertions.assertEquals(2, followers.size()); - - try (RaftClient leaderClient = cluster.createClient(cluster.getLeader().getId()); - RaftClient followerClient1 = cluster.createClient(followers.get(0).getId(), RetryPolicies.noRetry())) { - leaderClient.io().send(incrementMessage); - - RaftClientReply clientReply = followerClient1.io().sendReadOnly(queryMessage); - Assertions.assertEquals(1, retrieve(clientReply)); - - // kill the leader - // read timeout quicker than election timeout - leaderClient.admin().transferLeadership(null, 200); - - Assertions.assertThrows(ReadIndexException.class, () -> { - followerClient1.io().sendReadOnly(queryMessage, followers.get(0).getId()); - }); - } - - } finally { - cluster.shutdown(); - } - } - - @Test - public void testFollowerReadOnlyRetryWhenLeaderDown() throws Exception { - getProperties().setEnum(RaftServerConfigKeys.Read.OPTION_KEY, RaftServerConfigKeys.Read.Option.LINEARIZABLE); - runWithNewCluster(NUM_SERVERS, this::testFollowerReadOnlyRetryWhenLeaderDown); - } - - @Test - public void testFollowerLeaseReadRetryWhenLeaderDown() throws Exception { - getProperties().setBoolean(RaftServerConfigKeys.Read.LEADER_LEASE_ENABLED_KEY, true); - runWithNewCluster(NUM_SERVERS, this::testFollowerReadOnlyRetryWhenLeaderDown); - } - - private void testFollowerReadOnlyRetryWhenLeaderDown(CLUSTER cluster) throws Exception { - // only retry on readIndexException - final RetryPolicy retryPolicy = ExceptionDependentRetry - .newBuilder() - .setDefaultPolicy(RetryPolicies.noRetry()) - .setExceptionToPolicy(ReadIndexException.class, - RetryPolicies.retryForeverWithSleep(TimeDuration.valueOf(500, TimeUnit.MILLISECONDS))) - .build(); - - RaftTestUtil.waitForLeader(cluster); - - try (RaftClient client = cluster.createClient(cluster.getLeader().getId(), retryPolicy)) { - client.io().send(incrementMessage); - - final RaftClientReply clientReply = client.io().sendReadOnly(queryMessage); - Assertions.assertEquals(1, retrieve(clientReply)); + try (RaftClient client = cluster.createClient(leaderId, retryPolicy)) { + assertReplyExact(1, client.io().send(INCREMENT)); + assertReplyExact(1, client.io().sendReadOnly(QUERY)); // kill the leader client.admin().transferLeadership(null, 200); // readOnly will success after re-election - final RaftClientReply replySuccess = client.io().sendReadOnly(queryMessage); - Assertions.assertEquals(1, retrieve(clientReply)); - } - } - - @Test - public void testReadAfterWrite() throws Exception { - runWithNewCluster(NUM_SERVERS, this::testReadAfterWriteImpl); - } - - private void testReadAfterWriteImpl(CLUSTER cluster) throws Exception { - RaftTestUtil.waitForLeader(cluster); - try (RaftClient client = cluster.createClient()) { - // test blocking read-after-write - client.io().send(incrementMessage); - final RaftClientReply blockReply = client.io().sendReadAfterWrite(queryMessage); - Assertions.assertEquals(1, retrieve(blockReply)); - - // test asynchronous read-after-write - client.async().send(incrementMessage); - client.async().sendReadAfterWrite(queryMessage).thenAccept(reply -> { - Assertions.assertEquals(2, retrieve(reply)); - }); - - for (int i = 0; i < 20; i++) { - client.async().send(incrementMessage); - } - final CompletableFuture linearizable = client.async().sendReadOnly(queryMessage); - final CompletableFuture readAfterWrite = client.async().sendReadAfterWrite(queryMessage); - - CompletableFuture.allOf(linearizable, readAfterWrite).get(); - // read-after-write is more consistent than linearizable read - Assertions.assertTrue(retrieve(readAfterWrite.get()) >= retrieve(linearizable.get())); + assertReplyExact(1, client.io().sendReadOnly(QUERY)); } } @@ -375,12 +214,12 @@ private long timeoutIncrement() { @Override public CompletableFuture applyTransaction(TransactionContext trx) { - final RaftProtos.LogEntryProto logEntry = trx.getLogEntryUnsafe(); - LOG.debug("apply trx with index=" + logEntry.getIndex()); - updateLastAppliedTermIndex(logEntry.getTerm(), logEntry.getIndex()); + final LogEntryProto logEntry = trx.getLogEntry(); + final TermIndex ti = TermIndex.valueOf(logEntry); + updateLastAppliedTermIndex(ti); + LOG.info("{}: updateLastAppliedTermIndex {}", getId(), ti); - String command = logEntry.getStateMachineLogEntry() - .getLogData().toString(StandardCharsets.UTF_8); + final String command = logEntry.getStateMachineLogEntry().getLogData().toString(StandardCharsets.UTF_8); final long updatedCount; if (command.equals(INCREMENT_STRING)) { diff --git a/ratis-server/src/test/java/org/apache/ratis/server/impl/LeaderElectionTests.java b/ratis-server/src/test/java/org/apache/ratis/server/impl/LeaderElectionTests.java index c39b8a2c3c..3a47d127c5 100644 --- a/ratis-server/src/test/java/org/apache/ratis/server/impl/LeaderElectionTests.java +++ b/ratis-server/src/test/java/org/apache/ratis/server/impl/LeaderElectionTests.java @@ -21,8 +21,8 @@ import org.apache.ratis.RaftTestUtil; import org.apache.ratis.client.RaftClient; import org.apache.ratis.conf.RaftProperties; -import org.apache.ratis.metrics.impl.DefaultTimekeeperImpl; import org.apache.ratis.metrics.impl.RatisMetricRegistryImpl; +import org.apache.ratis.metrics.impl.DefaultTimekeeperImpl; import org.apache.ratis.proto.RaftProtos; import org.apache.ratis.protocol.RaftClientReply; import org.apache.ratis.protocol.RaftGroupId; @@ -37,6 +37,7 @@ import org.apache.ratis.server.RaftServerConfigKeys; import org.apache.ratis.server.metrics.LeaderElectionMetrics; import org.apache.ratis.server.protocol.TermIndex; +import org.apache.ratis.server.raftlog.segmented.SegmentedRaftLogTestUtils; import org.apache.ratis.test.tag.Flaky; import org.apache.ratis.thirdparty.com.codahale.metrics.Timer; import org.apache.ratis.util.CodeInjectionForTesting; @@ -49,7 +50,6 @@ import org.junit.jupiter.api.Assertions; import org.junit.jupiter.api.Test; import org.slf4j.Logger; -import org.slf4j.event.Level; import java.io.IOException; import java.util.ArrayList; @@ -66,8 +66,8 @@ import static org.apache.ratis.RaftTestUtil.waitForLeader; import static org.apache.ratis.server.metrics.LeaderElectionMetrics.LAST_LEADER_ELECTION_ELAPSED_TIME; import static org.apache.ratis.server.metrics.LeaderElectionMetrics.LEADER_ELECTION_COUNT_METRIC; -import static org.apache.ratis.server.metrics.LeaderElectionMetrics.LEADER_ELECTION_TIMEOUT_COUNT_METRIC; import static org.apache.ratis.server.metrics.LeaderElectionMetrics.LEADER_ELECTION_TIME_TAKEN; +import static org.apache.ratis.server.metrics.LeaderElectionMetrics.LEADER_ELECTION_TIMEOUT_COUNT_METRIC; import static org.junit.jupiter.api.Assertions.assertEquals; import static org.junit.jupiter.api.Assertions.assertNotNull; import static org.junit.jupiter.api.Assertions.assertTrue; @@ -75,6 +75,8 @@ import static org.mockito.Mockito.mock; import static org.mockito.Mockito.when; +import org.slf4j.event.Level; + public abstract class LeaderElectionTests extends BaseTest implements MiniRaftCluster.Factory.Get { @@ -86,16 +88,15 @@ public abstract class LeaderElectionTests @Test public void testBasicLeaderElection() throws Exception { LOG.info("Running testBasicLeaderElection"); - runWithNewCluster(5, this::runTestBasicLeaderElection); - } - - void runTestBasicLeaderElection(MiniRaftCluster cluster) throws Exception { + final MiniRaftCluster cluster = newCluster(5); + cluster.start(); RaftTestUtil.waitAndKillLeader(cluster); RaftTestUtil.waitAndKillLeader(cluster); RaftTestUtil.waitAndKillLeader(cluster); testFailureCase("waitForLeader after killed a majority of servers", () -> RaftTestUtil.waitForLeader(cluster, null, false), IllegalStateException.class); + cluster.shutdown(); } static class SleepCode implements CodeInjectionForTesting.Code { @@ -122,11 +123,9 @@ public void testWaitServerReady() throws Exception { final int sleepMs = 1000 + ThreadLocalRandom.current().nextInt(1000); LOG.info("Running testWaitServerReady, sleep = {}ms", sleepMs); CodeInjectionForTesting.put(RaftServerImpl.START_COMPLETE, new SleepCode(sleepMs)); + final MiniRaftCluster cluster = newCluster(1); final Timestamp startTime = Timestamp.currentTime(); - runWithNewCluster(1, c -> runTestWaitServerReady(c, sleepMs, startTime)); - } - - void runTestWaitServerReady(MiniRaftCluster cluster, int sleepMs, Timestamp startTime) throws Exception { + cluster.start(); LOG.info("Cluster started at {}ms", startTime.elapsedTimeMs()); final RaftGroupId groupId = cluster.getGroupId(); final RaftServerImpl server = (RaftServerImpl) cluster.getServers().iterator().next().getDivision(groupId); @@ -137,18 +136,17 @@ void runTestWaitServerReady(MiniRaftCluster cluster, int sleepMs, Timestamp star assertNotNull(waitForLeader(cluster), "No leader is elected."); final long elapsedMs = startTime.elapsedTimeMs(); // allow a small difference to tolerate system timer inaccuracy - Assertions.assertTrue(elapsedMs > sleepMs - 10, () -> "elapseMs = " + elapsedMs + " but sleepMs = " + sleepMs); + assertTrue(elapsedMs > sleepMs - 10, () -> "elapseMs = " + elapsedMs + " but sleepMs = " + sleepMs); + cluster.shutdown(); CodeInjectionForTesting.remove(RaftServerImpl.START_COMPLETE); } @Test - public void testAddServerForWaitReady() throws Exception { + public void testAddServerForWaitReady() throws IOException, InterruptedException { LOG.info("Running testAddServerForWaitReady"); // normal startup cluster with 3 server - runWithNewCluster(3, this::runTestAddServerForWaitReady); - } - - void runTestAddServerForWaitReady(MiniRaftCluster cluster) throws Exception { + final MiniRaftCluster cluster = newCluster(3); + cluster.start(); RaftTestUtil.waitForLeader(cluster); try (RaftClient client = cluster.createClient()) { for (int i = 0; i < 10; ++i) { @@ -171,20 +169,23 @@ void runTestAddServerForWaitReady(MiniRaftCluster cluster) throws Exception { }); } } + cluster.shutdown();; CodeInjectionForTesting.remove(RaftServerImpl.START_COMPLETE); } @Test public void testChangeLeader() throws Exception { + SegmentedRaftLogTestUtils.setRaftLogWorkerLogLevel(Level.TRACE); LOG.info("Running testChangeLeader"); - runWithNewCluster(3, this::runTestChangeLeader); - } + final MiniRaftCluster cluster = newCluster(3); + cluster.start(); - void runTestChangeLeader(MiniRaftCluster cluster) throws Exception { RaftPeerId leader = RaftTestUtil.waitForLeader(cluster).getId(); for(int i = 0; i < 10; i++) { leader = RaftTestUtil.changeLeader(cluster, leader, Assertions::fail); } + SegmentedRaftLogTestUtils.setRaftLogWorkerLogLevel(Level.INFO); + cluster.shutdown(); } @Test @@ -212,18 +213,18 @@ public void testLeaderNotCountListenerForMajority() throws Exception { void runTestLeaderNotCountListenerForMajority(CLUSTER cluster) throws Exception { final RaftServer.Division leader = waitForLeader(cluster); - Assertions.assertEquals(2, ((RaftConfigurationImpl)cluster.getLeader().getRaftConf()).getMajorityCount()); + assertEquals(2, ((RaftConfigurationImpl)cluster.getLeader().getRaftConf()).getMajorityCount()); try (RaftClient client = cluster.createClient(leader.getId())) { client.io().send(new RaftTestUtil.SimpleMessage("message")); List listeners = cluster.getListeners() .stream().map(RaftServer.Division::getPeer).collect(Collectors.toList()); - Assertions.assertEquals(2, listeners.size()); + assertEquals(2, listeners.size()); RaftClientReply reply = client.admin().setConfiguration(cluster.getPeers()); - Assertions.assertTrue(reply.isSuccess()); + assertTrue(reply.isSuccess()); Collection peer = leader.getRaftConf().getAllPeers(RaftProtos.RaftPeerRole.LISTENER); - Assertions.assertEquals(0, peer.size()); + assertEquals(0, peer.size()); } - Assertions.assertEquals(3, ((RaftConfigurationImpl)cluster.getLeader().getRaftConf()).getMajorityCount()); + assertEquals(3, ((RaftConfigurationImpl)cluster.getLeader().getRaftConf()).getMajorityCount()); } @Test @@ -241,7 +242,7 @@ void runTestListenerNotStartLeaderElection(CLUSTER cluster) throws Exception { RaftTestUtil.isolate(cluster, listenerId); maxTimeout.sleep(); maxTimeout.sleep(); - Assertions.assertEquals(RaftProtos.RaftPeerRole.LISTENER, listener.getInfo().getCurrentRole()); + assertEquals(RaftProtos.RaftPeerRole.LISTENER, listener.getInfo().getCurrentRole()); } finally { RaftTestUtil.deIsolate(cluster, listener.getId()); } @@ -257,19 +258,21 @@ public void testTransferLeader() throws Exception { client.io().send(new RaftTestUtil.SimpleMessage("message")); List followers = cluster.getFollowers(); - Assertions.assertEquals(2, followers.size()); + assertEquals(2, followers.size()); RaftServer.Division newLeader = followers.get(0); RaftClientReply reply = client.admin().transferLeadership(newLeader.getId(), 20000); - Assertions.assertTrue(reply.isSuccess()); + assertTrue(reply.isSuccess()); final RaftServer.Division currLeader = waitForLeader(cluster); - Assertions.assertEquals(newLeader.getId(), currLeader.getId()); + assertEquals(newLeader.getId(), currLeader.getId()); reply = client.io().send(new RaftTestUtil.SimpleMessage("message")); - Assertions.assertEquals(newLeader.getId().toString(), reply.getReplierId()); - Assertions.assertTrue(reply.isSuccess()); + assertEquals(newLeader.getId().toString(), reply.getReplierId()); + assertTrue(reply.isSuccess()); } + + cluster.shutdown(); } } @@ -283,25 +286,27 @@ public void testYieldLeaderToHigherPriority() throws Exception { client.io().send(new RaftTestUtil.SimpleMessage("message")); List followers = cluster.getFollowers(); - Assertions.assertEquals(2, followers.size()); + assertEquals(2, followers.size()); RaftServer.Division newLeader = followers.get(0); List peers = cluster.getPeers(); List peersWithNewPriority = getPeersWithPriority(peers, newLeader.getPeer()); RaftClientReply reply = client.admin().setConfiguration(peersWithNewPriority.toArray(new RaftPeer[0])); - Assertions.assertTrue(reply.isSuccess()); + assertTrue(reply.isSuccess()); // Wait the old leader to step down. // TODO: make it more deterministic. TimeDuration.valueOf(1, TimeUnit.SECONDS).sleep(); final RaftServer.Division currLeader = waitForLeader(cluster); - Assertions.assertEquals(newLeader.getId(), currLeader.getId()); + assertEquals(newLeader.getId(), currLeader.getId()); reply = client.io().send(new RaftTestUtil.SimpleMessage("message")); - Assertions.assertEquals(newLeader.getId().toString(), reply.getReplierId()); - Assertions.assertTrue(reply.isSuccess()); + assertEquals(newLeader.getId().toString(), reply.getReplierId()); + assertTrue(reply.isSuccess()); } + + cluster.shutdown(); } } @@ -313,7 +318,7 @@ public void testTransferLeaderTimeout() throws Exception { final RaftServer.Division leader = waitForLeader(cluster); try (RaftClient client = cluster.createClient(leader.getId())) { List followers = cluster.getFollowers(); - Assertions.assertEquals(followers.size(), 2); + assertEquals(followers.size(), 2); RaftServer.Division newLeader = followers.get(0); // isolate new leader, so that transfer leadership will timeout @@ -329,9 +334,9 @@ public void testTransferLeaderTimeout() throws Exception { client.admin().transferLeadership(newLeader.getId(), timeoutMs); } catch (TransferLeadershipException e) { long cost = System.currentTimeMillis() - start; - Assertions.assertTrue(cost > timeoutMs); - Assertions.assertTrue(e.getMessage().contains("Failed to transfer leadership to")); - Assertions.assertTrue(e.getMessage().contains(TransferLeadership.Result.Type.TIMED_OUT.toString())); + assertTrue(cost > timeoutMs); + assertTrue(e.getMessage().contains("Failed to transfer leadership to")); + assertTrue(e.getMessage().contains(TransferLeadership.Result.Type.TIMED_OUT.toString())); } return true; @@ -345,20 +350,22 @@ public void testTransferLeaderTimeout() throws Exception { try { client.io().send(new RaftTestUtil.SimpleMessage("message")); } catch (LeaderSteppingDownException e) { - Assertions.assertTrue(e.getMessage().contains("is stepping down")); + assertTrue(e.getMessage().contains("is stepping down")); } return null; }, 5, TimeDuration.ONE_SECOND, "check leader steppingDown", RaftServer.LOG); - Assertions.assertTrue(transferTimeoutFuture.get()); + assertTrue(transferTimeoutFuture.get()); // after transfer timeout, leader should accept request RaftClientReply reply = client.io().send(new RaftTestUtil.SimpleMessage("message")); - Assertions.assertEquals(leader.getId().toString(), reply.getReplierId()); - Assertions.assertTrue(reply.isSuccess()); + assertEquals(leader.getId().toString(), reply.getReplierId()); + assertTrue(reply.isSuccess()); RaftTestUtil.deIsolate(cluster, newLeader.getId()); } + + cluster.shutdown(); } } @@ -391,23 +398,19 @@ static void enforceLeader(MiniRaftCluster cluster, final String newLeader, Logge log.info(cluster.printServers()); final RaftServer.Division leader = cluster.getLeader(); - Assertions.assertEquals(newLeader, leader.getId().toString()); + assertEquals(newLeader, leader.getId().toString()); } @Test public void testLateServerStart() throws Exception { + final int numServer = 3; LOG.info("Running testLateServerStart"); - try (final MiniRaftCluster cluster = newCluster(3)) { - runTestLateServerStart(cluster); - } - } - - void runTestLateServerStart(MiniRaftCluster cluster) throws Exception { + final MiniRaftCluster cluster = newCluster(numServer); cluster.initServers(); // start all except one servers final Iterator i = cluster.getServers().iterator(); - for(int j = 1; j < cluster.getNumServers(); j++) { + for(int j = 1; j < numServer; j++) { i.next().start(); } @@ -424,7 +427,8 @@ void runTestLateServerStart(MiniRaftCluster cluster) throws Exception { .orElseThrow(() -> new IllegalStateException("No leader yet")), 10, ONE_SECOND, "getLeaderId", LOG); LOG.info(cluster.printServers()); - Assertions.assertEquals(leader.getId(), lastServerLeaderId); + assertEquals(leader.getId(), lastServerLeaderId); + cluster.shutdown(); } protected void testDisconnectLeader() throws Exception { @@ -438,10 +442,12 @@ protected void testDisconnectLeader() throws Exception { RaftTestUtil.isolate(cluster, leader.getId()); RaftClientReply reply = client.io().send(new RaftTestUtil.SimpleMessage("message")); Assertions.assertNotEquals(reply.getReplierId(), leader.getId().toString()); - Assertions.assertTrue(reply.isSuccess()); + assertTrue(reply.isSuccess()); } finally { RaftTestUtil.deIsolate(cluster, leader.getId()); } + + cluster.shutdown(); } } @@ -453,16 +459,17 @@ public void testAddListener() throws Exception { try (RaftClient client = cluster.createClient(leader.getId())) { client.io().send(new RaftTestUtil.SimpleMessage("message")); List servers = cluster.getPeers(); - Assertions.assertEquals(servers.size(), 3); - MiniRaftCluster.PeerChanges changes = cluster.addNewPeers(1, - true, false, RaftProtos.RaftPeerRole.LISTENER); - RaftClientReply reply = client.admin().setConfiguration(servers, Arrays.asList(changes.newPeers)); - Assertions.assertTrue(reply.isSuccess()); + assertEquals(servers.size(), 3); + final PeerChanges changes = cluster.addNewPeers(1, true); + final List added = changes.getAddedPeers(); + final RaftClientReply reply = client.admin().setConfiguration(servers, added); + assertTrue(reply.isSuccess()); Collection listener = leader.getRaftConf().getAllPeers(RaftProtos.RaftPeerRole.LISTENER); - Assertions.assertEquals(1, listener.size()); - Assertions.assertEquals(changes.newPeers[0].getId(), new ArrayList<>(listener).get(0).getId()); + assertEquals(1, listener.size()); + assertEquals(added.get(0).getId(), listener.iterator().next().getId()); } + cluster.shutdown(); } } @@ -474,20 +481,21 @@ public void testAddFollowerWhenExistsListener() throws Exception { try (RaftClient client = cluster.createClient(leader.getId())) { client.io().send(new RaftTestUtil.SimpleMessage("message")); List servers = cluster.getPeers(); - Assertions.assertEquals(4, servers.size()); + assertEquals(4, servers.size()); List listener = new ArrayList<>( leader.getRaftConf().getAllPeers(RaftProtos.RaftPeerRole.LISTENER)); - Assertions.assertEquals(1, listener.size()); - MiniRaftCluster.PeerChanges changes = cluster.addNewPeers(1, true, false); - ArrayList newPeers = new ArrayList<>(Arrays.asList(changes.newPeers)); + assertEquals(1, listener.size()); + final PeerChanges changes = cluster.addNewPeers(1, true); + final List newPeers = new ArrayList<>(changes.getAddedPeers()); newPeers.addAll(leader.getRaftConf().getAllPeers(RaftProtos.RaftPeerRole.FOLLOWER)); RaftClientReply reply = client.admin().setConfiguration(newPeers, listener); - Assertions.assertTrue(reply.isSuccess()); - Assertions.assertEquals(4, + assertTrue(reply.isSuccess()); + assertEquals(4, leader.getRaftConf().getAllPeers(RaftProtos.RaftPeerRole.FOLLOWER).size()); - Assertions.assertEquals(1, + assertEquals(1, leader.getRaftConf().getAllPeers(RaftProtos.RaftPeerRole.LISTENER).size()); } + cluster.shutdown(); } } @@ -498,14 +506,15 @@ public void testRemoveListener() throws Exception { final RaftServer.Division leader = waitForLeader(cluster); try (RaftClient client = cluster.createClient(leader.getId())) { client.io().send(new RaftTestUtil.SimpleMessage("message")); - Assertions.assertEquals(1, cluster.getListeners().size()); + assertEquals(1, cluster.getListeners().size()); List servers = cluster.getFollowers().stream().map(RaftServer.Division::getPeer).collect( Collectors.toList()); servers.add(leader.getPeer()); RaftClientReply reply = client.admin().setConfiguration(servers); - Assertions.assertTrue(reply.isSuccess()); - Assertions.assertEquals(0, leader.getRaftConf().getAllPeers(RaftProtos.RaftPeerRole.LISTENER).size()); + assertTrue(reply.isSuccess()); + assertEquals(0, leader.getRaftConf().getAllPeers(RaftProtos.RaftPeerRole.LISTENER).size()); } + cluster.shutdown(); } } @@ -518,16 +527,17 @@ public void testChangeFollowerToListener() throws Exception { client.io().send(new RaftTestUtil.SimpleMessage("message")); List followers = cluster.getFollowers().stream().map( RaftServer.Division::getPeer).collect(Collectors.toList()); - Assertions.assertEquals(2, followers.size()); + assertEquals(2, followers.size()); List listeners = new ArrayList<>(); listeners.add(followers.get(1)); followers.remove(1); RaftClientReply reply = client.admin().setConfiguration(followers, listeners); - Assertions.assertTrue(reply.isSuccess()); + assertTrue(reply.isSuccess()); Collection peer = leader.getRaftConf().getAllPeers(RaftProtos.RaftPeerRole.LISTENER); - Assertions.assertEquals(1, peer.size()); - Assertions.assertEquals(listeners.get(0).getId(), new ArrayList<>(peer).get(0).getId()); + assertEquals(1, peer.size()); + assertEquals(listeners.get(0).getId(), new ArrayList<>(peer).get(0).getId()); } + cluster.shutdown(); } } @@ -540,22 +550,27 @@ public void testChangeListenerToFollower() throws Exception { client.io().send(new RaftTestUtil.SimpleMessage("message")); List listeners = cluster.getListeners() .stream().map(RaftServer.Division::getPeer).collect(Collectors.toList()); - Assertions.assertEquals(listeners.size(), 1); + assertEquals(listeners.size(), 1); RaftClientReply reply = client.admin().setConfiguration(cluster.getPeers()); - Assertions.assertTrue(reply.isSuccess()); + assertTrue(reply.isSuccess()); Collection peer = leader.getRaftConf().getAllPeers(RaftProtos.RaftPeerRole.LISTENER); - Assertions.assertEquals(0, peer.size()); + assertEquals(0, peer.size()); + + RaftServerTestUtil.waitAndCheckNewConf(cluster, cluster.getPeers(), 0, null); + + listeners = cluster.getListeners() + .stream().map(RaftServer.Division::getPeer).collect(Collectors.toList()); + assertEquals(0, listeners.size()); } + cluster.shutdown(); } } @Test - public void testLeaderElectionMetrics() throws Exception { - runWithNewCluster(3, this::runTestLeaderElectionMetrics); - } - - void runTestLeaderElectionMetrics(MiniRaftCluster cluster) throws Exception { + public void testLeaderElectionMetrics() throws IOException, InterruptedException { Timestamp timestamp = Timestamp.currentTime(); + final MiniRaftCluster cluster = newCluster(3); + cluster.start(); final RaftServer.Division leaderServer = waitForLeader(cluster); final RatisMetricRegistryImpl ratisMetricRegistry = (RatisMetricRegistryImpl) @@ -625,7 +640,7 @@ public void testPreVote() { RaftTestUtil.isolate(cluster, follower.getId()); // send message so that the isolated follower's log lag the others RaftClientReply reply = client.io().send(new RaftTestUtil.SimpleMessage("message")); - Assertions.assertTrue(reply.isSuccess()); + assertTrue(reply.isSuccess()); final long savedTerm = leader.getInfo().getCurrentTerm(); LOG.info("Wait follower {} timeout and trigger pre-vote", follower.getId()); @@ -640,8 +655,10 @@ public void testPreVote() { assertEquals(savedTerm, leader.getInfo().getCurrentTerm()); reply = client.io().send(new RaftTestUtil.SimpleMessage("message")); - Assertions.assertTrue(reply.isSuccess()); + assertTrue(reply.isSuccess()); } + + cluster.shutdown(); } catch (Exception e) { fail(e.getMessage()); } @@ -673,23 +690,23 @@ void runTestPauseResumeLeaderElection(CLUSTER cluster) throws IOException, Inter final RaftServer.Division leader = RaftTestUtil.waitForLeader(cluster); final RaftPeerId leaderId = leader.getId(); final List followers = cluster.getFollowers(); - Assertions.assertTrue(followers.size() >= 1); + assertTrue(followers.size() >= 1); final RaftServerImpl f1 = (RaftServerImpl)followers.get(0); try (final RaftClient client = cluster.createClient()) { pauseLeaderReply = client.getLeaderElectionManagementApi(f1.getId()).pause(); - Assertions.assertTrue(pauseLeaderReply.isSuccess()); + assertTrue(pauseLeaderReply.isSuccess()); client.io().send(new RaftTestUtil.SimpleMessage("message")); RaftServer.Division newLeader = followers.get(0); List peers = cluster.getPeers(); List peersWithNewPriority = getPeersWithPriority(peers, newLeader.getPeer()); RaftClientReply reply = client.admin().setConfiguration(peersWithNewPriority.toArray(new RaftPeer[0])); - Assertions.assertTrue(reply.isSuccess()); - JavaUtils.attempt(() -> Assertions.assertEquals(leaderId, leader.getId()), + assertTrue(reply.isSuccess()); + JavaUtils.attempt(() -> assertEquals(leaderId, leader.getId()), 20, HUNDRED_MILLIS, "check leader id", LOG); final RaftClientReply resumeLeaderReply = client.getLeaderElectionManagementApi(f1.getId()).resume(); - Assertions.assertTrue(resumeLeaderReply.isSuccess()); - JavaUtils.attempt(() -> Assertions.assertEquals(f1.getId(), cluster.getLeader().getId()), + assertTrue(resumeLeaderReply.isSuccess()); + JavaUtils.attempt(() -> assertEquals(f1.getId(), cluster.getLeader().getId()), 20, HUNDRED_MILLIS, "check new leader", LOG); } } @@ -715,15 +732,15 @@ void runTestLeaderLease(CLUSTER cluster, long leaseTimeoutMs) throws Exception { try (final RaftClient client = cluster.createClient(leader.getId())) { client.io().send(new RaftTestUtil.SimpleMessage("message")); - Assertions.assertTrue(leader.getInfo().isLeader()); - Assertions.assertTrue(leader.getInfo().isLeaderReady()); + assertTrue(leader.getInfo().isLeader()); + assertTrue(leader.getInfo().isLeaderReady()); RaftServerTestUtil.assertLeaderLease(leader, true); RaftTestUtil.isolate(cluster, leader.getId()); Thread.sleep(leaseTimeoutMs); - Assertions.assertTrue(leader.getInfo().isLeader()); - Assertions.assertTrue(leader.getInfo().isLeaderReady()); + assertTrue(leader.getInfo().isLeader()); + assertTrue(leader.getInfo().isLeaderReady()); RaftServerTestUtil.assertLeaderLease(leader, false); } finally { RaftTestUtil.deIsolate(cluster, leader.getId()); @@ -744,8 +761,8 @@ void runTestLeaderLeaseDuringReconfiguration(CLUSTER cluster, long leaseTimeoutM try (final RaftClient client = cluster.createClient(leader.getId())) { client.io().send(new RaftTestUtil.SimpleMessage("message")); - Assertions.assertTrue(leader.getInfo().isLeader()); - Assertions.assertTrue(leader.getInfo().isLeaderReady()); + assertTrue(leader.getInfo().isLeader()); + assertTrue(leader.getInfo().isLeaderReady()); RaftServerTestUtil.assertLeaderLease(leader, true); final List followers = cluster.getFollowers(); @@ -766,8 +783,8 @@ void runTestLeaderLeaseDuringReconfiguration(CLUSTER cluster, long leaseTimeoutM Thread.sleep(leaseTimeoutMs); - Assertions.assertTrue(leader.getInfo().isLeader()); - Assertions.assertTrue(leader.getInfo().isLeaderReady()); + assertTrue(leader.getInfo().isLeader()); + assertTrue(leader.getInfo().isLeaderReady()); RaftServerTestUtil.assertLeaderLease(leader, false); } finally { diff --git a/ratis-server/src/test/java/org/apache/ratis/server/impl/MiniRaftCluster.java b/ratis-server/src/test/java/org/apache/ratis/server/impl/MiniRaftCluster.java index df78069952..825ae8f55b 100644 --- a/ratis-server/src/test/java/org/apache/ratis/server/impl/MiniRaftCluster.java +++ b/ratis-server/src/test/java/org/apache/ratis/server/impl/MiniRaftCluster.java @@ -49,7 +49,6 @@ import org.apache.ratis.util.JavaUtils; import org.apache.ratis.util.NetUtils; import org.apache.ratis.util.Preconditions; -import org.apache.ratis.util.ReferenceCountedLeakDetector; import org.apache.ratis.util.ReflectionUtils; import org.apache.ratis.util.TimeDuration; import org.apache.ratis.util.function.CheckedConsumer; @@ -95,6 +94,8 @@ public abstract class MiniRaftCluster implements Closeable { TimeDuration.valueOf(100, TimeUnit.MILLISECONDS); static final AtomicInteger THREAD_COUNT = new AtomicInteger(0); + static final AtomicInteger CLIENT_ID = new AtomicInteger(0); + public abstract static class Factory { public interface Get { Supplier PROPERTIES = JavaUtils.memoize(RaftProperties::new); @@ -138,27 +139,17 @@ default void runWithNewCluster(int numServers, int numListeners, boolean startCl final StackTraceElement caller = JavaUtils.getCallerStackTraceElement(); LOG.info("Running {}", caller.getMethodName()); final CLUSTER cluster = newCluster(numServers, numListeners); - Throwable failed = null; try { if (startCluster) { cluster.start(); } testCase.accept(cluster); - } catch(Throwable t) { + } catch(Exception t) { LOG.info(cluster.printServers()); LOG.error("Failed {}", caller, t); - failed = t; throw t; } finally { - try { - cluster.shutdown(); - } catch (Exception e) { - if (failed == null) { - throw e; - } else { - failed.addSuppressed(e); - } - } + cluster.shutdown(); } } @@ -819,42 +810,18 @@ public void shutdown() { final ExecutorService executor = Executors.newFixedThreadPool(servers.size(), (t) -> Daemon.newBuilder().setName("MiniRaftCluster-" + THREAD_COUNT.incrementAndGet()).setRunnable(t).build()); getServers().forEach(proxy -> executor.submit(() -> JavaUtils.runAsUnchecked(proxy::close))); - final int maxRetries = 30; - final TimeDuration retrySleep = TimeDuration.ONE_SECOND; - executor.shutdown(); - boolean terminated = false; - - for(int i = 0; i < maxRetries && !terminated; ) { - try { - terminated = executor.awaitTermination(retrySleep.getDuration(), retrySleep.getUnit()); - if (!terminated) { - i++; - if (i < maxRetries) { - LOG.warn("{}/{}: Not yet able to shutdown executor, will wait again ...", i, maxRetries); - } else { - LOG.error("Failed to shutdown executor, some servers may be still running:\n{}", printServers()); - } - } - } catch (InterruptedException e) { + try { + executor.shutdown(); + // just wait for a few seconds + executor.awaitTermination(5, TimeUnit.SECONDS); + } catch (InterruptedException e) { LOG.warn("shutdown interrupted", e); - } + Thread.currentThread().interrupt(); } Optional.ofNullable(timer.get()).ifPresent(Timer::cancel); ExitUtils.assertNotTerminated(); LOG.info("{} shutdown completed", JavaUtils.getClassSimpleName(getClass())); - - // GC to ensure leak detection work. - try { - RaftTestUtil.gc(); - } catch (InterruptedException e) { - LOG.warn("gc interrupted.", e); - } - try { - ReferenceCountedLeakDetector.getLeakDetector().assertNoLeaks(maxRetries, retrySleep); - } catch (InterruptedException e) { - LOG.warn("LeakDetector interrupted.", e); - } } /** diff --git a/ratis-server/src/test/java/org/apache/ratis/server/impl/RaftReconfigurationBaseTest.java b/ratis-server/src/test/java/org/apache/ratis/server/impl/RaftReconfigurationBaseTest.java index c06b1565aa..7a2f1a24ee 100644 --- a/ratis-server/src/test/java/org/apache/ratis/server/impl/RaftReconfigurationBaseTest.java +++ b/ratis-server/src/test/java/org/apache/ratis/server/impl/RaftReconfigurationBaseTest.java @@ -269,7 +269,7 @@ private void runTestSetConfigurationInAddMode(CLUSTER cluster) throws Exception .setServersInNewConf(peers) .setMode(SetConfigurationRequest.Mode.ADD).build()); Assertions.assertTrue(reply.isSuccess()); - waitAndCheckNewConf(cluster, change.allPeersInNewConf, 0, null); + waitAndCheckNewConf(cluster, change.getPeersInNewConf(), 0, null); } cluster.close(); } @@ -307,7 +307,7 @@ private void runTestSetConfigurationInCasMode(CLUSTER cluster) throws Exception .setMode(SetConfigurationRequest.Mode.COMPARE_AND_SET) .build()); Assertions.assertTrue(reply.isSuccess()); - waitAndCheckNewConf(cluster, change.allPeersInNewConf, 0, null); + waitAndCheckNewConf(cluster, change.getPeersInNewConf(), 0, null); } cluster.close(); } @@ -387,8 +387,7 @@ void runTestReconfTimeout(CLUSTER cluster) throws Exception { try (final RaftClient client = cluster.createClient(leaderId)) { PeerChanges c1 = cluster.addNewPeers(2, false); - LOG.info("Start changing the configuration: {}", - asList(c1.allPeersInNewConf)); + LOG.info("Start changing the configuration: {}", c1.getPeersInNewConf()); Assertions.assertFalse(((RaftConfigurationImpl)cluster.getLeader().getRaftConf()).isTransitional()); final RaftClientRpc sender = client.getClientRpc(); @@ -420,7 +419,7 @@ void runTestReconfTimeout(CLUSTER cluster) throws Exception { for (RaftPeer np : c1.getAddedPeers()) { cluster.restartServer(np.getId(), false); } - Assertions.assertTrue(client.admin().setConfiguration(c1.allPeersInNewConf).isSuccess()); + Assertions.assertTrue(client.admin().setConfiguration(c1.getPeersInNewConf()).isSuccess()); } } @@ -440,7 +439,7 @@ public void testBootstrapReconfWithSingleNodeAddTwo() throws Exception { try (final RaftClient client = cluster.createClient(leaderId)) { final PeerChanges c1 = cluster.addNewPeers(2, true); - assertThrows(SetConfigurationException.class, () -> client.admin().setConfiguration(c1.allPeersInNewConf), + assertThrows(SetConfigurationException.class, () -> client.admin().setConfiguration(c1.getPeersInNewConf()), "Expect change majority error."); } }); diff --git a/ratis-server/src/test/java/org/apache/ratis/server/impl/RaftStateMachineExceptionTests.java b/ratis-server/src/test/java/org/apache/ratis/server/impl/RaftStateMachineExceptionTests.java index a1aed3ff46..1e46907d10 100644 --- a/ratis-server/src/test/java/org/apache/ratis/server/impl/RaftStateMachineExceptionTests.java +++ b/ratis-server/src/test/java/org/apache/ratis/server/impl/RaftStateMachineExceptionTests.java @@ -187,4 +187,31 @@ private void runTestRetryOnExceptionDuringReplication(CLUSTER cluster) throws Ex failPreAppend = false; } } + + @Test + public void testNoCancelTransactionOnPreAppendFailure() throws Exception { + runWithNewCluster(3, this::runTestNoCancelTransactionOnPreAppendFailure); + } + + private void runTestNoCancelTransactionOnPreAppendFailure(CLUSTER cluster) throws Exception { + final RaftPeerId leaderId = RaftTestUtil.waitForLeader(cluster).getId(); + failPreAppend = true; + numCancelTransaction.set(0); + try (final RaftClient client = cluster.createClient(leaderId)) { + try { + client.io().send(new SimpleMessage("cancel-transaction")); + fail("Exception expected"); + } catch (StateMachineException e) { + Assertions.assertTrue(e.getCause().getMessage().contains("Fake Exception in preAppend")); + } + + JavaUtils.attemptRepeatedly(() -> { + Assertions.assertEquals(0, numCancelTransaction.get(), + () -> "Expected cancelTransaction() not to be called but got " + numCancelTransaction.get()); + return null; + }, 10, ONE_SECOND, "wait for cancelTransaction", LOG); + } finally { + failPreAppend = false; + } + } } diff --git a/ratis-server/src/test/java/org/apache/ratis/statemachine/RaftSnapshotBaseTest.java b/ratis-server/src/test/java/org/apache/ratis/statemachine/RaftSnapshotBaseTest.java index f1106f2cf9..09b5ee59a6 100644 --- a/ratis-server/src/test/java/org/apache/ratis/statemachine/RaftSnapshotBaseTest.java +++ b/ratis-server/src/test/java/org/apache/ratis/statemachine/RaftSnapshotBaseTest.java @@ -21,7 +21,6 @@ import static org.apache.ratis.server.impl.StateMachineMetrics.RATIS_STATEMACHINE_METRICS_DESC; import static org.apache.ratis.server.impl.StateMachineMetrics.STATEMACHINE_TAKE_SNAPSHOT_TIMER; import static org.apache.ratis.metrics.RatisMetrics.RATIS_APPLICATION_NAME_METRICS; -import static org.apache.ratis.server.storage.RaftStorageTestUtils.getLogUnsafe; import org.apache.ratis.BaseTest; import org.apache.ratis.metrics.LongCounter; @@ -51,9 +50,7 @@ import org.apache.ratis.util.JavaUtils; import org.apache.ratis.util.LifeCycle; import org.apache.ratis.util.Slf4jUtils; -import org.junit.jupiter.api.AfterEach; import org.junit.jupiter.api.Assertions; -import org.junit.jupiter.api.BeforeEach; import org.junit.jupiter.api.Test; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -102,13 +99,13 @@ public static void assertLeaderContent(MiniRaftCluster cluster) throws Exception public static void checkMetadataEntry(RaftServer.Division server) throws Exception { final RaftLog log = server.getRaftLog(); final long lastIndex = log.getLastEntryTermIndex().getIndex(); - final LogEntryProto e = getLogUnsafe(log, lastIndex); + final LogEntryProto e = log.get(lastIndex); Assertions.assertTrue(e.hasMetadataEntry()); + Assertions.assertEquals(log.getLastCommittedIndex() - 1, e.getMetadataEntry().getCommitIndex()); + } - JavaUtils.attemptRepeatedly(() -> { - Assertions.assertEquals(log.getLastCommittedIndex() - 1, e.getMetadataEntry().getCommitIndex()); - return null; - }, 50, BaseTest.HUNDRED_MILLIS, "CheckMetadataEntry", LOG); + public static void assertLogContent(RaftServer.Division server, boolean isLeader) throws Exception { + JavaUtils.attempt(() -> checkMetadataEntry(server), 50, HUNDRED_MILLIS, "checkMetadataEntry", LOG); SimpleStateMachine4Testing simpleStateMachine = SimpleStateMachine4Testing.get(server); if (isLeader) { @@ -126,29 +123,6 @@ public static void checkMetadataEntry(RaftServer.Division server) throws Excepti } } - private MiniRaftCluster cluster; - - public abstract MiniRaftCluster.Factory getFactory(); - - @BeforeEach - public void setup() throws IOException { - final RaftProperties prop = new RaftProperties(); - prop.setClass(MiniRaftCluster.STATEMACHINE_CLASS_KEY, - SimpleStateMachine4Testing.class, StateMachine.class); - RaftServerConfigKeys.Snapshot.setAutoTriggerThreshold( - prop, SNAPSHOT_TRIGGER_THRESHOLD); - RaftServerConfigKeys.Snapshot.setAutoTriggerEnabled(prop, true); - this.cluster = getFactory().newCluster(1, prop); - cluster.start(); - } - - @AfterEach - public void tearDown() { - if (cluster != null) { - cluster.shutdown(); - } - } - /** * Keep generating writing traffic and make sure snapshots are taken. * We then restart the whole raft peer and check if it can correctly load diff --git a/ratis-server/src/test/java/org/apache/ratis/statemachine/impl/SimpleStateMachine4Testing.java b/ratis-server/src/test/java/org/apache/ratis/statemachine/impl/SimpleStateMachine4Testing.java index 3a7c8aa774..1ffbdbcb99 100644 --- a/ratis-server/src/test/java/org/apache/ratis/statemachine/impl/SimpleStateMachine4Testing.java +++ b/ratis-server/src/test/java/org/apache/ratis/statemachine/impl/SimpleStateMachine4Testing.java @@ -47,7 +47,6 @@ import org.apache.ratis.util.LifeCycle; import org.apache.ratis.util.MD5FileUtil; import org.apache.ratis.util.Preconditions; -import org.apache.ratis.util.ReferenceCountedObject; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -248,11 +247,10 @@ public synchronized void reinitialize() throws IOException { @Override public CompletableFuture applyTransaction(TransactionContext trx) { blocking.await(Blocking.Type.APPLY_TRANSACTION); - ReferenceCountedObject entryRef = Objects.requireNonNull(trx.getLogEntryRef()); - LogEntryProto entry = entryRef.get(); + LogEntryProto entry = Objects.requireNonNull(trx.getLogEntry()); LOG.info("applyTransaction for log index {}", entry.getIndex()); - put(LogProtoUtils.copy(entry)); + put(entry); updateLastAppliedTermIndex(entry.getTerm(), entry.getIndex()); final SimpleMessage m = new SimpleMessage(entry.getIndex() + " OK"); @@ -333,10 +331,10 @@ public CompletableFuture query(Message request) { final String string = request.getContent().toStringUtf8(); Exception exception; try { - LOG.info("query {}, all available: {}", string, dataMap.keySet()); + LOG.info("query {}", string); final LogEntryProto entry = dataMap.get(string); if (entry != null) { - return CompletableFuture.completedFuture(Message.valueOf(entry)); + return CompletableFuture.completedFuture(Message.valueOf(entry.toByteString())); } exception = new IndexOutOfBoundsException(getId() + ": LogEntry not found for query " + string); } catch (Exception e) { @@ -360,8 +358,7 @@ public TransactionContext startTransaction(RaftClientRequest request) { } @Override - public CompletableFuture write(ReferenceCountedObject entry, TransactionContext context) { - Preconditions.assertTrue(entry.get() != null); + public CompletableFuture write(LogEntryProto entry) { return blocking.getFuture(Blocking.Type.WRITE_STATE_MACHINE_DATA); } @@ -382,8 +379,6 @@ public void close() { running = false; checkpointer.interrupt(); }); - indexMap.clear(); - dataMap.clear(); } public LogEntryProto[] getContent() { diff --git a/ratis-shell/pom.xml b/ratis-shell/pom.xml index ddd1ad0737..495187b2a8 100644 --- a/ratis-shell/pom.xml +++ b/ratis-shell/pom.xml @@ -17,7 +17,7 @@ ratis org.apache.ratis - 3.2.0-SNAPSHOT + 3.3.0-SNAPSHOT ratis-shell diff --git a/ratis-test/pom.xml b/ratis-test/pom.xml index 5f55e4cde3..577262d84c 100644 --- a/ratis-test/pom.xml +++ b/ratis-test/pom.xml @@ -17,7 +17,7 @@ ratis org.apache.ratis - 3.2.0-SNAPSHOT + 3.3.0-SNAPSHOT ratis-test diff --git a/ratis-tools/pom.xml b/ratis-tools/pom.xml index 2dccaf7a2e..2d81f2467a 100644 --- a/ratis-tools/pom.xml +++ b/ratis-tools/pom.xml @@ -17,7 +17,7 @@ ratis org.apache.ratis - 3.2.0-SNAPSHOT + 3.3.0-SNAPSHOT ratis-tools