diff --git a/ivy.xml b/ivy.xml
index 0d1d9ed5..048d8466 100644
--- a/ivy.xml
+++ b/ivy.xml
@@ -3,7 +3,7 @@
xsi:noNamespaceSchemaLocation="http://ant.apache.org/ivy/schemas/ivy.xsd"
xmlns:e="http://ant.apache.org/ivy/extra">
-
+
@@ -13,11 +13,12 @@
-
+
+
diff --git a/pom.xml b/pom.xml
index b3845180..50ac2dd4 100644
--- a/pom.xml
+++ b/pom.xml
@@ -11,13 +11,17 @@
11
11
+
true
jboss-releases-repository
https://repository.jboss.org/nexus
jboss-snapshots-repository
https://repository.jboss.org/nexus/content/repositories/snapshots/
- 7.7.0
+
+
+ 7.7.1
+ 3.24.2
5.3.0.Final
1.8
2.22.0
@@ -42,6 +46,7 @@
5
true
false
+ false
-Djgroups.udp.ip_ttl=0
@@ -52,6 +57,7 @@
-Dlog4j.configurationFile=${conf.dir}/log4j2.xml
-Djava.net.preferIPv4Stack=${jgroups.useIPv4}
-Djava.net.preferIPv6Addresses=${jgroups.useIPv6}
+ -Dorg.jgrops.test.trace=${trace}
-Xms400M
-Xmx800M
@@ -214,7 +220,13 @@
${jmh.version}
test
+
+ org.assertj
+ assertj-core
+ ${assertj.version}
+
+
src
diff --git a/src/org/jgroups/protocols/raft/ELECTION.java b/src/org/jgroups/protocols/raft/ELECTION.java
index 77596a15..a815b7e9 100644
--- a/src/org/jgroups/protocols/raft/ELECTION.java
+++ b/src/org/jgroups/protocols/raft/ELECTION.java
@@ -51,6 +51,7 @@ public class ELECTION extends BaseElection {
protected void handleView(View v) {
Majority result=Utils.computeMajority(view, v, raft().majority(), raft.leader());
log.debug("%s: existing view: %s, new view: %s, result: %s", local_addr, this.view, v, result);
+ System.out.printf("%s: existing view: %s, new view: %s, result: %s%n", local_addr, this.view, v, result);
List joiners=View.newMembers(this.view, v);
boolean has_new_members=joiners != null && !joiners.isEmpty();
this.view=v;
diff --git a/src/org/jgroups/protocols/raft/Leader.java b/src/org/jgroups/protocols/raft/Leader.java
index aeeae02b..dca3d115 100644
--- a/src/org/jgroups/protocols/raft/Leader.java
+++ b/src/org/jgroups/protocols/raft/Leader.java
@@ -32,8 +32,11 @@ public void init() {
public void destroy() {
super.destroy();
+ RequestTable reqTable = raft.request_table;
raft.request_table=null;
raft.commit_table=null;
+
+ if (reqTable != null) reqTable.destroy(raft.notCurrentLeader());
}
@@ -50,9 +53,17 @@ public void handleAppendEntriesResponse(Address sender, long term, AppendResult
switch(result.result) {
case OK:
raft.commit_table.update(sender, result.index(), result.index() + 1, result.commit_index, false);
- if(reqtab.add(result.index, sender_raft_id, this.majority)) {
+ boolean done = reqtab.add(result.index, sender_raft_id, this.majority);
+ if(done) {
raft.commitLogTo(result.index, true);
- if(raft.send_commits_immediately)
+ }
+ // Send commits immediately.
+ // Note that, an entry is committed by a MAJORITY, this means that some of the nodes doesn't know the entry exist yet.
+ // This way, send the commit messages any time we handle an append response.
+ if(raft.send_commits_immediately) {
+ // Done is only true when reaching a majority threshold, we also need to check is committed to resend
+ // to slower nodes.
+ if (done || reqtab.isCommitted(result.index))
sendCommitMessageToFollowers();
}
break;
diff --git a/src/org/jgroups/protocols/raft/RAFT.java b/src/org/jgroups/protocols/raft/RAFT.java
index 15964d1a..a66680a9 100644
--- a/src/org/jgroups/protocols/raft/RAFT.java
+++ b/src/org/jgroups/protocols/raft/RAFT.java
@@ -651,7 +651,7 @@ public CompletableFuture setAsync(byte[] buf, int offset, int length, Op
public CompletableFuture setAsync(byte[] buf, int offset, int length, boolean internal, Options options) {
Address leader = leader();
if(leader == null || (local_addr != null && !leader.equals(local_addr)))
- throw new IllegalStateException("I'm not the leader (local_addr=" + local_addr + ", leader=" + leader + ")");
+ throw notCurrentLeader();
if(buf == null)
throw new IllegalArgumentException("buffer must not be null");
CompletableFuture retval=new CompletableFuture<>();
@@ -686,7 +686,7 @@ protected void handleDownRequest(CompletableFuture f, byte[] buf, int of
boolean internal, Options opts) {
Address leader = leader();
if(leader == null || !Objects.equals(leader,local_addr))
- throw new IllegalStateException("I'm not the leader (local_addr=" + local_addr + ", leader=" + leader + ")");
+ throw notCurrentLeader();
RequestTable reqtab=request_table;
@@ -797,7 +797,7 @@ else if(r instanceof UpRequest)
}
}
- protected void process (List q) {
+ protected void process(List q) {
RequestTable reqtab=request_table;
LogEntries entries=new LogEntries();
long index=last_appended+1;
@@ -813,6 +813,14 @@ protected void process (List q) {
}
else if(r instanceof DownRequest) {
DownRequest dr=(DownRequest)r;
+ // Complete the request exceptionally.
+ // The request could either be lost in the reqtab reference or fail with an NPE below.
+ // It would only complete in case a timeout is associated.
+ if (!isLeader()) {
+ dr.f.completeExceptionally(notCurrentLeader());
+ continue;
+ }
+
entries.add(new LogEntry(current_term, dr.buf, dr.offset, dr.length, dr.internal));
// Add the request to the client table, so we can return results to clients when done
@@ -830,7 +838,7 @@ else if(r instanceof DownRequest) {
// handle down requests
if(leader == null || !Objects.equals(leader,local_addr))
- throw new IllegalStateException("I'm not the leader (local_addr=" + local_addr + ", leader=" + leader + ")");
+ throw notCurrentLeader();
// Append to the log
long prev_index=last_appended;
@@ -860,6 +868,10 @@ else if(r instanceof DownRequest) {
snapshotIfNeeded(length);
}
+ IllegalStateException notCurrentLeader() {
+ return new IllegalStateException("I'm not the leader (local_addr=" + local_addr + ", leader=" + leader() + ")");
+ }
+
/** Populate with non-committed entries (from log) (https://github.com/belaban/jgroups-raft/issues/31) */
protected void createRequestTable() {
request_table=new RequestTable<>();
@@ -948,14 +960,17 @@ protected CompletableFuture changeMembers(String name, InternalCommand.T
Address leader = leader();
if(leader == null || !Objects.equals(leader, local_addr))
- throw new IllegalStateException("I'm not the leader (local_addr=" + local_addr + ", leader=" + leader + ")");
+ throw notCurrentLeader();
InternalCommand cmd=new InternalCommand(type, name);
byte[] buf=Util.streamableToByteBuffer(cmd);
// only add/remove one server at a time (https://github.com/belaban/jgroups-raft/issues/175)
return add_server_future=add_server_future
- .thenCompose(s -> setAsync(buf, 0, buf.length, true, null));
+ // Use handle, so we can execute even if the previous execution failed.
+ .handle((ignore, t) -> setAsync(buf, 0, buf.length, true, null))
+ // Chain the new setAsync invocation.
+ .thenCompose(Function.identity());
}
protected void resend(Address target, long index) {
diff --git a/src/org/jgroups/protocols/raft/state/RaftState.java b/src/org/jgroups/protocols/raft/state/RaftState.java
index 60087096..f835749d 100644
--- a/src/org/jgroups/protocols/raft/state/RaftState.java
+++ b/src/org/jgroups/protocols/raft/state/RaftState.java
@@ -148,7 +148,10 @@ public synchronized void setLeader(Address newLeader) {
this.leader = newLeader;
// We only should invoke the listener when a new leader is set/step down.
- if (updated) onLeaderUpdate.accept(this.leader);
+ if (updated) {
+ raft.getLog().trace("%s: change leader from %s -> %s", raft.addr(), leader, newLeader);
+ onLeaderUpdate.accept(this.leader);
+ }
}
/**
diff --git a/src/org/jgroups/raft/testfwk/MockRaftCluster.java b/src/org/jgroups/raft/testfwk/MockRaftCluster.java
index 69fa4243..6ba5dc53 100644
--- a/src/org/jgroups/raft/testfwk/MockRaftCluster.java
+++ b/src/org/jgroups/raft/testfwk/MockRaftCluster.java
@@ -1,5 +1,6 @@
package org.jgroups.raft.testfwk;
+import org.jgroups.Address;
import org.jgroups.Message;
import org.jgroups.View;
@@ -8,29 +9,138 @@
import java.util.concurrent.ThreadPoolExecutor;
import java.util.concurrent.TimeUnit;
+/**
+ * Base class for the cluster implementations in the test framework.
+ *
+ *
+ * The cluster abstraction facilitates the creation of a cluster during the tests. This approach avoids the need to
+ * create actual {@link org.jgroups.JChannel} and complex configurations. This abstraction provides a simplified and
+ * controllable way to test the cluster, emitting events and sending messages.
+ *
+ *
+ *
+ * The cluster abstraction works through {@link View} updates. Members are added and removed manually. When a new view
+ * is received, the members receive the update accordingly and follow the synchronous configuration. The user utilizing
+ * this abstraction has more control over when a member joins and which messages it receives.
+ *
+ *
+ * Our test suite contains examples of uses of the cluster abstraction.
+ *
+ * @since 1.0.12
+ */
public abstract class MockRaftCluster {
protected final Executor thread_pool=createThreadPool(1000);
protected boolean async;
+ /**
+ * Emit the view update to all cluster members.
+ *
+ * How the view is updated might vary from implementation. The basic idea is to iterate over all members and
+ * invoke each protocol in the stack to handle the view.
+ *
+ *
+ * @param view: The new {@link View} instance to update the member.
+ */
public abstract void handleView(View view);
+ /**
+ * Send a message in the cluster.
+ *
+ * @param msg: Message to send.
+ */
public abstract void send(Message msg);
+ /**
+ * The number of member in the cluster.
+ *
+ * @return The size of the cluster.
+ */
+ public abstract int size();
+
+ /**
+ * Add a new member to the cluster.
+ *
+ *
+ * The new member is associated with the given address. A member is resolved by the address when sending a message.
+ * Also, note that a view update is necessary to propagate the member addition.
+ *
+ *
+ * @param addr: The new member's address.
+ * @param node: The member abstraction wrapped in {@link RaftNode}.
+ * @return The fluent current class.
+ * @param : The current instance type.
+ */
+ public abstract T add(Address addr, RaftNode node);
+
+ /**
+ * Remove a member from the cluster.
+ *
+ * A view update is necessary to propagate the member removal.
+ *
+ *
+ * @param addr: The address of the member to remove.
+ * @return The fluent current class.
+ * @param : The current instance type.
+ */
+ public abstract T remove(Address addr);
+
+ /**
+ * Remove all members from the cluster.
+ *
+ * @return The fluent current class.
+ * @param : The current instance type.
+ */
+ public abstract T clear();
+
+ /**
+ * Utility to create a fluent use.
+ *
+ * @return The fluent current class.
+ * @param : The current instance type.
+ */
@SuppressWarnings("unchecked")
- protected T self() {
+ protected final T self() {
return (T) this;
}
+ /**
+ * Check whether the cluster is in asynchronous mode.
+ *
+ * @return true
if asynchronous, false
, otherwise.
+ */
public boolean async() {return async;}
- public T async(boolean b) {async=b; return self();}
- protected static Executor createThreadPool(long max_idle_ms) {
+ /**
+ * Update the cluster mode between synchronous and asynchronous.
+ *
+ * @param b: true
to run asynchronous, false
to run synchronous.
+ * @return The fluent current class.
+ * @param : The current instance type.
+ */
+ public T async(boolean b) {
+ async=b;
+ return self();
+ }
+
+ /**
+ * Create the {@link Executor} to submit tasks during asynchronous mode.
+ *
+ * @param max_idle_ms: Executor configuration parameter.
+ * @return The {@link Executor} instance to utilize in the cluster abstraction.
+ */
+ protected Executor createThreadPool(long max_idle_ms) {
int max_cores=Runtime.getRuntime().availableProcessors();
return new ThreadPoolExecutor(0, max_cores, max_idle_ms, TimeUnit.MILLISECONDS,
new SynchronousQueue<>());
}
+ /**
+ * Asynchronously sends a message up the node stack.
+ *
+ * @param node: The node to handle the message.
+ * @param msg: The message to send.
+ */
protected void deliverAsync(RaftNode node, Message msg) {
thread_pool.execute(() -> node.up(msg));
}
diff --git a/src/org/jgroups/raft/testfwk/PartitionedRaftCluster.java b/src/org/jgroups/raft/testfwk/PartitionedRaftCluster.java
index 0feb1389..6d8dc5d6 100644
--- a/src/org/jgroups/raft/testfwk/PartitionedRaftCluster.java
+++ b/src/org/jgroups/raft/testfwk/PartitionedRaftCluster.java
@@ -20,8 +20,13 @@ public class PartitionedRaftCluster extends MockRaftCluster {
protected final Map> partitions = new ConcurrentHashMap<>();
protected final Map nodes = new ConcurrentHashMap<>();
- public PartitionedRaftCluster clear() {nodes.clear(); return this;}
+ @Override
+ public PartitionedRaftCluster clear() {
+ nodes.clear();
+ return self();
+ }
+ @Override
public PartitionedRaftCluster add(Address addr, RaftNode node) {
nodes.put(addr, node);
return this;
@@ -62,6 +67,17 @@ public void send(Message msg) {
}
}
+ @Override
+ public int size() {
+ return nodes.size();
+ }
+
+ @Override
+ public PartitionedRaftCluster remove(Address addr) {
+ nodes.remove(addr);
+ return self();
+ }
+
private void send(RaftNode node, Message msg) {
if (async) deliverAsync(node, msg);
else node.up(msg);
diff --git a/src/org/jgroups/raft/testfwk/RaftCluster.java b/src/org/jgroups/raft/testfwk/RaftCluster.java
index 32e14b9c..9d140504 100644
--- a/src/org/jgroups/raft/testfwk/RaftCluster.java
+++ b/src/org/jgroups/raft/testfwk/RaftCluster.java
@@ -22,12 +22,16 @@ public class RaftCluster extends MockRaftCluster {
protected final Map dropped_members=new ConcurrentHashMap<>();
protected boolean async;
+ @Override
public RaftCluster add(Address addr, RaftNode node) {
nodes.put(addr, node);
return this;
}
+ @Override
public RaftCluster remove(Address addr) {nodes.remove(addr); return this;}
+
+ @Override
public RaftCluster clear() {nodes.clear(); return this;}
public boolean dropTraffic() {return !dropped_members.isEmpty();}
public RaftCluster dropTrafficTo(Address a) {move(a, nodes, dropped_members); return this;}
@@ -46,6 +50,11 @@ public void send(Message msg) {
send(msg, false);
}
+ @Override
+ public int size() {
+ return nodes.size();
+ }
+
public void send(Message msg, boolean async) {
Address dest=msg.dest(), src=msg.src();
if(dest != null) {
diff --git a/src/org/jgroups/raft/testfwk/RaftTestUtils.java b/src/org/jgroups/raft/testfwk/RaftTestUtils.java
new file mode 100644
index 00000000..6d182d78
--- /dev/null
+++ b/src/org/jgroups/raft/testfwk/RaftTestUtils.java
@@ -0,0 +1,120 @@
+package org.jgroups.raft.testfwk;
+
+import org.jgroups.JChannel;
+import org.jgroups.protocols.raft.Log;
+import org.jgroups.protocols.raft.RAFT;
+import org.jgroups.protocols.raft.election.BaseElection;
+
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.locks.LockSupport;
+import java.util.function.BooleanSupplier;
+
+/**
+ * Utilities for developing tests with Raft.
+ *
+ * @since 1.0.13
+ * @author José Bolina
+ */
+public final class RaftTestUtils {
+
+ private RaftTestUtils() { }
+
+ /**
+ * Retrieves the {@link RAFT} protocol from the provided {@link JChannel}.
+ *
+ * @param ch: The channel to search the protocol.
+ * @return The {@link RAFT} instance or null
if not found.
+ */
+ public static RAFT raft(JChannel ch) {
+ return ch.getProtocolStack().findProtocol(RAFT.class);
+ }
+
+ /**
+ * Retrieves the {@link BaseElection} protocol from the provided {@link JChannel}.
+ * The concrete type might vary according to the configured protocol stack.
+ *
+ * @param ch: The channel to search the protocol.
+ * @return The {@link BaseElection} instance or null
if not found.
+ */
+ public static BaseElection election(JChannel ch) {
+ return ch.getProtocolStack().findProtocol(BaseElection.class);
+ }
+
+ /**
+ * Checks if the node is the current {@link RAFT} leader.
+ *
+ * This method search the protocol stack to retrieve the {@link RAFT} instance and then verifies if elected.
+ *
+ *
+ * @param ch: The channel to verify.
+ * @return true
if currently the leader, and false
otherwise
+ * or not found the {@link RAFT} protocol.
+ */
+ public static boolean isRaftLeader(JChannel ch) {
+ RAFT r = raft(ch);
+ return r.isLeader() && r.leader() != null && ch.getAddress().equals(r.leader());
+ }
+
+ /**
+ * Checks that given the time constraints, eventually {@link #isRaftLeader(JChannel)} is true
.
+ *
+ * @param ch: Channel to verify.
+ * @param timeoutMs: The timeout in milliseconds.
+ * @return true
if it became the leader, false
, otherwise.
+ * @see #isRaftLeader(JChannel)
+ */
+ public static boolean eventuallyIsRaftLeader(JChannel ch, long timeoutMs) {
+ return eventually(() -> isRaftLeader(ch), timeoutMs, TimeUnit.MILLISECONDS);
+ }
+
+ /**
+ * Deletes all the replicated data and internal state for the given {@link RAFT} instance.
+ *
+ * This is useful when multiple tests for the same uses stable storage. After running each test the data can
+ * be deleted to no affect subsequent tests.
+ *
+ *
+ * @param r: {@link RAFT} instance to delete all information.
+ * @throws Exception: If an exception happens while deleting the data.
+ * @see Log#delete()
+ */
+ public static void deleteRaftLog(RAFT r) throws Exception {
+ Log log = r != null ? r.log() : null;
+ if (log != null) {
+ log.delete();
+ r.log(null);
+ }
+ }
+
+ /**
+ * Verify that in the given time constraints, the expression returns true
.
+ *
+ * @param bs: Boolean expression to verify.
+ * @param timeout: Timeout value.
+ * @param unit: Timeout unit.
+ * @return true
if expression valid before time out, false
, otherwise.
+ * @throws RuntimeException: If an exception is thrown while verifying the expression.
+ */
+ public static boolean eventually(BooleanSupplier bs, long timeout, TimeUnit unit) {
+ try {
+ long timeoutNanos = unit.toNanos(timeout);
+ // We want the sleep time to increase in arithmetic progression
+ // 30 loops with the default timeout of 30 seconds means the initial wait is ~ 65 millis
+ int loops = 30;
+ int progressionSum = loops * (loops + 1) / 2;
+ long initialSleepNanos = timeoutNanos / progressionSum;
+ long sleepNanos = initialSleepNanos;
+ long expectedEndTime = System.nanoTime() + timeoutNanos;
+ while (expectedEndTime - System.nanoTime() > 0) {
+ if (bs.getAsBoolean())
+ return true;
+ LockSupport.parkNanos(sleepNanos);
+ sleepNanos += initialSleepNanos;
+ }
+
+ return bs.getAsBoolean();
+ } catch (Exception e) {
+ throw new RuntimeException("Unexpected!", e);
+ }
+ }
+}
diff --git a/src/org/jgroups/raft/util/RequestTable.java b/src/org/jgroups/raft/util/RequestTable.java
index bcef9875..45fe5a2c 100644
--- a/src/org/jgroups/raft/util/RequestTable.java
+++ b/src/org/jgroups/raft/util/RequestTable.java
@@ -22,6 +22,10 @@ public class RequestTable {
// maps an index to a set of (response) senders
protected ArrayRingBuffer> requests;
+ // Identify the request table was destroyed.
+ // All subsequent requests should complete exceptionally immediately.
+ private Throwable destroyed;
+
public void create(long index, T vote, CompletableFuture future, Supplier majority) {
create(index, vote, future, majority, null);
@@ -34,6 +38,32 @@ public void create(long index, T vote, CompletableFuture future, Supplie
}
requests.set(index, entry);
entry.add(vote, majority);
+ // In case the leader steps down while still adding elements.
+ if (destroyed != null) entry.notify(destroyed);
+ }
+
+ /**
+ * Completes all uncommitted requests with the provided exception.
+ *
+ *
+ * This method should be invoked before setting the instance to null
when the leader steps down.
+ * This provides a more responsive completion of requests to the users, instead of having requests time out.
+ * Internal operations, such as membership changes, do not have a timeout associated, which would hang and any
+ * subsequent changes would not complete.
+ *
+ *
+ * @param t: Throwable to complete the requests exceptionally.
+ */
+ public void destroy(Throwable t) {
+ // Keep throwable so any entries created *after* destroying also complete exceptionally.
+ destroyed = t;
+ if (requests != null) {
+ requests.forEach((e, ignore) -> {
+ if (!e.committed) {
+ e.notify(t);
+ }
+ });
+ }
}
/**
diff --git a/src/org/jgroups/raft/util/Utils.java b/src/org/jgroups/raft/util/Utils.java
index 390be329..a6ee5852 100644
--- a/src/org/jgroups/raft/util/Utils.java
+++ b/src/org/jgroups/raft/util/Utils.java
@@ -2,8 +2,8 @@
import org.jgroups.Address;
import org.jgroups.View;
-import org.jgroups.protocols.raft.Log;
import org.jgroups.protocols.raft.RAFT;
+import org.jgroups.raft.testfwk.RaftTestUtils;
/**
* @author Bela Ban
@@ -32,12 +32,19 @@ public static Majority computeMajority(View old, View new_view, int majority, Ad
return Majority.no_change;
}
+ /**
+ * Deletes the log data for the given {@link RAFT} instance.
+ *
+ * Warning: This should be used in tests only.
+ *
+ *
+ * @param r: RAFT instance to delete the log contents.
+ * @throws Exception: If an exception happens while deleting the log.
+ * @deprecated Use {@link RaftTestUtils#deleteRaftLog(RAFT)} instead.
+ */
+ @Deprecated(since = "1.0.13", forRemoval = true)
public static void deleteLog(RAFT r) throws Exception {
- Log log=r != null? r.log() : null;
- if(log != null) {
- log.delete();
- r.log(null);
- }
+ RaftTestUtils.deleteRaftLog(r);
}
}
diff --git a/tests/junit-functional/org/jgroups/tests/AppendEntriesTest.java b/tests/junit-functional/org/jgroups/tests/AppendEntriesTest.java
index 1a5e595f..3bfaeaab 100644
--- a/tests/junit-functional/org/jgroups/tests/AppendEntriesTest.java
+++ b/tests/junit-functional/org/jgroups/tests/AppendEntriesTest.java
@@ -6,35 +6,32 @@
import org.jgroups.protocols.DISCARD;
import org.jgroups.protocols.TP;
import org.jgroups.protocols.raft.AppendResult;
-import org.jgroups.protocols.raft.ELECTION;
import org.jgroups.protocols.raft.Log;
import org.jgroups.protocols.raft.LogEntries;
import org.jgroups.protocols.raft.LogEntry;
import org.jgroups.protocols.raft.RAFT;
-import org.jgroups.protocols.raft.REDIRECT;
import org.jgroups.protocols.raft.RaftImpl;
+import org.jgroups.protocols.raft.RaftLeaderException;
import org.jgroups.raft.RaftHandle;
import org.jgroups.raft.blocks.ReplicatedStateMachine;
-import org.jgroups.raft.util.Utils;
+import org.jgroups.raft.testfwk.RaftTestUtils;
import org.jgroups.stack.ProtocolStack;
+import org.jgroups.tests.harness.BaseStateMachineTest;
import org.jgroups.util.Util;
-import org.testng.annotations.AfterMethod;
-import org.testng.annotations.Test;
import java.util.Arrays;
-import java.util.Collections;
-import java.util.List;
-import java.util.Objects;
import java.util.concurrent.TimeUnit;
import java.util.concurrent.TimeoutException;
import java.util.function.BooleanSupplier;
-import java.util.function.Function;
import java.util.stream.Collectors;
import java.util.stream.Stream;
-import static org.testng.Assert.assertEquals;
-import static org.testng.Assert.assertFalse;
-import static org.testng.Assert.assertTrue;
+import org.assertj.core.api.Assertions;
+import org.testng.annotations.AfterMethod;
+import org.testng.annotations.Test;
+
+import static org.assertj.core.api.Assertions.assertThat;
+import static org.jgroups.raft.testfwk.RaftTestUtils.eventually;
/**
* Tests the AppendEntries functionality: appending log entries in regular operation, new members, late joiners etc
@@ -42,27 +39,31 @@
* @since 0.2
*/
@Test(groups=Global.FUNCTIONAL,singleThreaded=true)
-public class AppendEntriesTest {
- // A is always the leader because started first and logs are equal
- protected JChannel a, b, c;
- protected ReplicatedStateMachine as, bs, cs;
- protected static final String CLUSTER="AppendEntriesTest";
- protected final List members=Arrays.asList("A", "B", "C");
- protected static final long TIMEOUT=5000, INTERVAL=100;
- protected static final Address leader=Util.createRandomAddress("A");
+public class AppendEntriesTest extends BaseStateMachineTest> {
+
protected static final byte[] buf=new byte[10];
protected static final int[] terms={0,1,1,1,4,4,5,5,6,6,6};
+ {
+ // We want to customize the cluster size per method. We need to manually create and clear the resources.
+ createManually = true;
+ }
+
+ @Override
+ protected ReplicatedStateMachine createStateMachine(JChannel ch) {
+ return new ReplicatedStateMachine<>(ch);
+ }
- @AfterMethod
- protected void destroy() {
- close(c, b, a);
+ @AfterMethod(alwaysRun = true)
+ void clearResources() throws Exception {
+ destroyCluster();
}
public void testSingleMember() throws Exception {
- // given
- a=create("A", Collections.singletonList("A"));
- a.connect(CLUSTER);
+ withClusterSize(1);
+ createCluster();
+
+ JChannel a = channel(0);
RaftHandle raft=new RaftHandle(a, new DummyStateMachine());
Util.waitUntil(1000, 250, raft::isLeader);
@@ -71,41 +72,41 @@ public void testSingleMember() throws Exception {
byte[] result=raft.set(data, 0, data.length, 1, TimeUnit.SECONDS);
// then
- assert Arrays.equals(result, new byte[0]) : "should have received empty byte array";
+ assertThat(result)
+ .as("should have received empty byte array")
+ .isEqualTo(new byte[0]);
}
public void testNormalOperation() throws Exception {
init(true);
for(int i=1; i <= 10; i++)
- as.put(i, i);
- assertSame(as, bs, cs);
- bs.remove(5);
- cs.put(11, 11);
- cs.remove(1);
- as.put(1, 1);
- assertSame(as, bs, cs);
+ stateMachine(0).put(i, i);
+
+ assertStateMachineEventuallyMatch(0, 1, 2);
+ stateMachine(1).remove(5);
+ stateMachine(2).put(11, 11);
+ stateMachine(2).remove(1);
+ stateMachine(0).put(1, 1);
+ assertStateMachineEventuallyMatch(0, 1, 2);
}
public void testRedirect() throws Exception {
init(true);
- cs.put(5, 5);
- assertSame(as, bs, cs);
+ stateMachine(2).put(5, 5);
+ assertStateMachineEventuallyMatch(0, 1, 2);
}
public void testPutWithoutLeader() throws Exception {
- a=create("A"); // leader
- as=new ReplicatedStateMachine<>(a);
- a.connect(CLUSTER);
- assert !isLeader(a);
- try {
- as.put(1, 1);
- assert false : "put() should fail as we don't have a leader";
- }
- catch(Throwable t) {
- System.out.println("received exception as expected: " + t);
- }
+ withClusterSize(3);
+ createCluster(1);
+
+ JChannel a=channel(0); // leader
+ assertThat(RaftTestUtils.isRaftLeader(a)).isFalse();
+ Assertions.assertThatThrownBy(() -> stateMachine(0).put(1, 1))
+ .isInstanceOf(RaftLeaderException.class)
+ .hasMessage("there is currently no leader to forward set() request to");
}
@@ -115,17 +116,24 @@ public void testPutWithoutLeader() throws Exception {
*/
public void testNonCommitWithoutMajority() throws Exception {
init(true);
- close(b, c);
- as.timeout(500);
+ close(2);
+ close(1);
+
+ stateMachine(0).timeout(500);
+
+ BooleanSupplier bs = () -> Arrays.stream(actualChannels())
+ .map(this::raft)
+ .allMatch(r -> r.leader() == null);
+ assertThat(eventually(bs, 5, TimeUnit.SECONDS))
+ .as(this::dumpLeaderAndTerms)
+ .isTrue();
for(int i=1; i <= 3; i++) {
- try {
- as.put(i, i);
- }
- catch(Exception ex) {
- System.out.printf("received %s as expected; cache size is %d\n", ex.getClass().getSimpleName(), as.size());
- }
- assert as.size() == 0;
+ int v = i;
+ Assertions.assertThatThrownBy(() -> stateMachine(0).put(v, v))
+ .isInstanceOf(RaftLeaderException.class)
+ .hasMessage("there is currently no leader to forward set() request to");
+ assertThat(stateMachine(0).size()).isEqualTo(0);
}
}
@@ -135,30 +143,31 @@ public void testNonCommitWithoutMajority() throws Exception {
*/
public void testCatchingUp() throws Exception {
init(true);
+
// A, B and C commit entries 1-2
for(int i=1; i <= 2; i++)
- as.put(i,i);
- assertSame(as, bs, cs);
+ stateMachine(0).put(i,i);
+
+ assertStateMachineEventuallyMatch(0, 1, 2);
// Now C leaves
- close(c);
+ close(2);
// A and B commit entries 3-5
for(int i=3; i <= 5; i++)
- as.put(i,i);
- assertSame(as, bs);
+ stateMachine(0).put(i,i);
+
+ assertStateMachineEventuallyMatch(0, 1);
// Now start C again: entries 1-5 will have to get resent to C as its log was deleted above (otherwise only 3-5
// would have to be resent)
System.out.println("-- starting C again, needs to catch up");
- c=create("C"); // follower
- cs=new ReplicatedStateMachine<>(c);
- c.connect(CLUSTER);
- Util.waitUntilAllChannelsHaveSameView(5000, 100, a,b,c);
+ createCluster();
+ JChannel c = channel(2); // follower
// Now C should also have the same entries (1-5) as A and B
- raft(a).resendInterval(200);
- assertSame(as, bs, cs);
+ raft(0).resendInterval(200);
+ assertStateMachineEventuallyMatch(0, 1, 2);
}
/**
@@ -170,38 +179,38 @@ public void testCatchingUpFirstEntry() throws Exception {
init(false);
// make B and C drop all traffic; this means A won't be able to commit
- for(JChannel ch: Arrays.asList(b,c)) {
+ for(JChannel ch: Arrays.asList(channel(1), channel(2))) {
ProtocolStack stack=ch.getProtocolStack();
DISCARD discard=new DISCARD().discardAll(true).setAddress(ch.getAddress());
stack.insertProtocol(discard, ProtocolStack.Position.ABOVE, TP.class);
};
// Add the first entry, this will time out as there's no majority
- as.timeout(500);
- try {
- as.put(1, 1);
- assert false : "should have gotten a TimeoutException";
- }
- catch(TimeoutException ignored) {
- System.out.println("The first put() timed out as expected as there's no majority to commit it");
- }
+ stateMachine(0).timeout(500);
+ Assertions.assertThatThrownBy(() -> stateMachine(0).put(1, 1))
+ .isInstanceOf(TimeoutException.class);
- RAFT raft=a.getProtocolStack().findProtocol(RAFT.class);
- System.out.printf("A: last-applied=%d, commit-index=%d\n", raft.lastAppended(), raft.commitIndex());
- assert raft.lastAppended() == 1;
- assert raft.commitIndex() == 0;
+ RAFT raft=raft(0);
+ assertThat(raft)
+ .isNotNull()
+ .as(String.format("A: last-applied=%d, commit-index=%d\n", raft.lastAppended(), raft.commitIndex()))
+ .returns(1L, RAFT::lastAppended)
+ .returns(0L, RAFT::commitIndex);
// now remove the DISCARD protocol from B and C
- Stream.of(b,c).forEach(ch -> ch.getProtocolStack().removeProtocol(DISCARD.class));
+ Stream.of(channel(1), channel(2)).forEach(ch -> ch.getProtocolStack().removeProtocol(DISCARD.class));
- assertCommitIndex(10000, 500, raft.lastAppended(), raft.lastAppended(), a, b);
- for(JChannel ch: Arrays.asList(a,b)) {
+ assertCommitIndex(10000, raft.lastAppended(), raft.lastAppended(), channel(0), channel(1));
+ for(JChannel ch: Arrays.asList(channel(0), channel(1))) {
raft=ch.getProtocolStack().findProtocol(RAFT.class);
- System.out.printf("%s: last-applied=%d, commit-index=%d\n", ch.getAddress(), raft.lastAppended(), raft.commitIndex());
- assert raft.lastAppended() == 1;
- assert raft.commitIndex() == 1;
+ String check = String.format("%s: last-applied=%d, commit-index=%d\n", ch.getAddress(), raft.lastAppended(), raft.commitIndex());
+ System.out.println(check);
+ assertThat(raft)
+ .as(check)
+ .returns(1L, RAFT::lastAppended)
+ .returns(1L, RAFT::commitIndex);
}
- assertSame(as, bs, cs);
+ assertStateMachineEventuallyMatch(0, 1, 2);
}
/**
@@ -210,43 +219,38 @@ public void testCatchingUpFirstEntry() throws Exception {
* on the leader should throw an exception!
*/
public void testLeaderRestart() throws Exception {
- a=create("A");
- raft(a).stateMachine(new DummyStateMachine());
- b=create("B");
- raft(b).stateMachine(new DummyStateMachine());
- a.connect(CLUSTER);
- b.connect(CLUSTER);
+ withClusterSize(2);
+
// A and B now have a majority and A is leader
- Util.waitUntilAllChannelsHaveSameView(10000, 500, a, b);
+ createCluster();
+
+ assertThat(RaftTestUtils.eventuallyIsRaftLeader(channel(0), 10_000))
+ .as("Channel A never elected")
+ .isTrue();
+ assertThat(RaftTestUtils.isRaftLeader(channel(1))).isFalse();
- assertLeader(a, 10000, 500);
- assert !raft(b).isLeader();
System.out.println("--> disconnecting B");
- b.disconnect(); // stop B; it was only needed to make A the leader
- Util.waitUntil(5000, 100, () -> !raft(a).isLeader());
+
+ close(1); // stop B; it was only needed to make A the leader
+ Util.waitUntil(5000, 100, () -> !RaftTestUtils.isRaftLeader(channel(0)));
// Now try to make a change on A. This will fail as A is not leader anymore
- try {
- raft(a).set(new byte[]{'b', 'e', 'l', 'a'}, 0, 4, 500, TimeUnit.MILLISECONDS);
- assert false : "set() should have thrown a timeout as we cannot commit the change";
- }
- catch(IllegalStateException ex) {
- System.out.printf("got exception as expected: %s\n", ex);
- }
+ Assertions.assertThatThrownBy(() -> raft(0).set(new byte[]{'b', 'e', 'l', 'a'}, 0, 4, 500, TimeUnit.MILLISECONDS))
+ .as("Trying to set value without leader")
+ .isInstanceOf(IllegalStateException.class)
+ .hasMessageStartingWith("I'm not the leader ");
// A now has last_applied=1 and commit_index=0:
- assertCommitIndex(10000, 500, 0, 0, a);
+ assertCommitIndex(10000, 0, 0, channel(0));
// Now start B again, this gives us a majority and entry #1 should be able to be committed
System.out.println("--> restarting B");
- b=create("B");
- raft(b).stateMachine(new DummyStateMachine());
- b.connect(CLUSTER);
+
// A and B now have a majority and A is leader
- Util.waitUntilAllChannelsHaveSameView(10000, 500, a, b);
+ createCluster();
// A and B should now have last_applied=0 and commit_index=0
- assertCommitIndex(10000, 500, 0, 0, a,b);
+ assertCommitIndex(10000, 0, 0, channel(0), channel(1));
}
@@ -256,75 +260,86 @@ public void testLeaderRestart() throws Exception {
*/
public void testInstallSnapshotInC() throws Exception {
init(true);
- close(c);
+ close(2);
+
for(int i=1; i <= 5; i++)
- as.put(i,i);
- assertSame(as, bs);
+ stateMachine(0).put(i,i);
+ assertStateMachineEventuallyMatch(0, 1);
// Snapshot A:
- as.snapshot();
+ stateMachine(0).snapshot();
// Now start C
- c=create("C"); // follower
- cs=new ReplicatedStateMachine<>(c);
- c.connect(CLUSTER);
- Util.waitUntilAllChannelsHaveSameView(10000, 500, a, b, c);
+ createCluster(); // follower
- assertSame(as, bs, cs);
+ assertStateMachineEventuallyMatch(0, 1, 2);
}
/** Tests an append at index 1 with prev_index 0 and index=2 with prev_index=1 */
public void testInitialAppends() throws Exception {
initB();
- RaftImpl impl=getImpl(b);
+ Address leader = leaderAddress();
+ RaftImpl impl=getImpl(channel(1));
Log log=impl.raft().log();
AppendResult result=append(impl, 1, 0, new LogEntry(4, buf), leader, 1);
- assert result.success();
- assertEquals(result.index(), 1);
- assertEquals(result.commitIndex(), 1);
+ assertThat(result)
+ .as("Validating: " + result)
+ .returns(true, AppendResult::success)
+ .returns(1L, AppendResult::index)
+ .returns(1L, AppendResult::commitIndex);
assertLogIndices(log, 1, 1, 4);
result=append(impl, 2, 4, new LogEntry(4, buf), leader, 1);
- assert result.success();
- assertEquals(result.index(), 2);
- assertEquals(result.commitIndex(), 1);
+ assertThat(result)
+ .as("Validating: " + result)
+ .returns(true, AppendResult::success)
+ .returns(2L, AppendResult::index)
+ .returns(1L, AppendResult::commitIndex);
assertLogIndices(log, 2, 1, 4);
result=append(impl, 2, 4, new LogEntry(4, null), leader, 2);
- assert result.success();
- assertEquals(result.index(), 2);
- assertEquals(result.commitIndex(), 2);
+ assertThat(result)
+ .as("Validating: " + result)
+ .returns(true, AppendResult::success)
+ .returns(2L, AppendResult::index)
+ .returns(2L, AppendResult::commitIndex);
assertLogIndices(log, 2, 2, 4);
}
/** Tests append _after_ the last appended index; returns an AppendResult with index=last_appended */
- public void testAppendAfterLastAppened() throws Exception {
+ public void testAppendAfterLastAppended() throws Exception {
initB();
- RaftImpl impl=getImpl(b);
+ Address leader = leaderAddress();
+ RaftImpl impl=getImpl(channel(1));
Log log=impl.raft().log();
// initial append at index 1
AppendResult result=append(impl, 1, 0, new LogEntry(4, buf), leader, 1);
- assert result.success();
- assertEquals(result.index(), 1);
+ assertThat(result)
+ .as("Validating: " + result)
+ .returns(true, AppendResult::success)
+ .returns(1L, AppendResult::index);
assertLogIndices(log, 1, 1, 4);
// append at index 3 fails because there is no entry at index 2
result=append(impl, 3, 4, new LogEntry(4, buf), leader, 1);
- assert !result.success();
- assertEquals(result.index(), 1);
+ assertThat(result)
+ .as("Validating: " + result)
+ .returns(false, AppendResult::success)
+ .returns(1L, AppendResult::index);
assertLogIndices(log, 1, 1, 4);
}
-
-
public void testSendCommitsImmediately() throws Exception {
// Force leader to send commit messages immediately
- init(true, r -> r.resendInterval(60_000).sendCommitsImmediately(true));
- as.put(1,1);
- assertSame(as, bs, cs);
+ init(true);
+ leader().resendInterval(60_000).sendCommitsImmediately(true);
+ System.out.println("-- sending value");
+ stateMachine(0).put(1,1);
+ System.out.println("-- finished operation, waiting for broadcast");
+ assertStateMachineEventuallyMatch(0, 1, 2);
}
// Index 01 02 03 04 05 06 07 08 09 10 11 12
@@ -332,7 +347,8 @@ public void testSendCommitsImmediately() throws Exception {
// Append 07 <--- wrong prev_term at index 11
public void testAppendWithConflictingTerm() throws Exception {
initB();
- RaftImpl impl=getImpl(b);
+ Address leader = leaderAddress();
+ RaftImpl impl=getImpl(channel(1));
Log log=impl.raft().log();
for(int i=1; i <= 10; i++)
@@ -340,9 +356,10 @@ public void testAppendWithConflictingTerm() throws Exception {
// now append(index=11,term=7) -> should return false result with index=8
AppendResult result=append(impl, 11, 7, new LogEntry(6, buf), leader, 1);
- assert !result.success();
- assertEquals(result.index(), 8);
- assertEquals(result.nonMatchingTerm(), 6);
+ assertThat(result)
+ .satisfies(r -> assertThat(r.success()).isFalse())
+ .satisfies(r -> assertThat(r.index()).isEqualTo(8))
+ .satisfies(r -> assertThat(r.nonMatchingTerm()).isEqualTo(6));
assertLogIndices(log, 7, 1, 5);
}
@@ -352,16 +369,18 @@ public void testAppendWithConflictingTerm() throws Exception {
// Append 07 <--- wrong prev_term at index 1
public void testAppendWithConflictingTerm2() throws Exception {
initB();
- RaftImpl impl=getImpl(b);
+ Address leader = leaderAddress();
+ RaftImpl impl=getImpl(channel(1));
Log log=impl.raft().log();
append(impl, 1, 0, new LogEntry(1, buf), leader, 0);
// now append(index=2,term=7) -> should return false result with index=1
AppendResult result=append(impl, 2, 7, new LogEntry(7, buf), leader, 1);
- assert !result.success();
- assertEquals(result.index(), 1);
- assertEquals(result.nonMatchingTerm(), 1);
+ assertThat(result)
+ .satisfies(r -> assertThat(r.success()).isFalse())
+ .satisfies(r -> assertThat(r.index()).isEqualTo(1))
+ .satisfies(r -> assertThat(r.nonMatchingTerm()).isEqualTo(1));
assertLogIndices(log, 0, 0, 0);
}
@@ -371,7 +390,8 @@ public void testAppendWithConflictingTerm2() throws Exception {
// Append 07 <--- wrong prev_term at index 3
public void testAppendWithConflictingTerm3() throws Exception {
initB();
- RaftImpl impl=getImpl(b);
+ Address leader = leaderAddress();
+ RaftImpl impl=getImpl(channel(1));
Log log=impl.raft().log();
append(impl, 1, 0, new LogEntry(1, buf), leader, 1);
@@ -380,9 +400,10 @@ public void testAppendWithConflictingTerm3() throws Exception {
// now append(index=2,term=7) -> should return false result with index=1
AppendResult result=append(impl, 4, 7, new LogEntry(7, buf), leader, 1);
- assert !result.success();
- assertEquals(result.index(), 3);
- assertEquals(result.nonMatchingTerm(), 5);
+ assertThat(result)
+ .as("Verifying: " + result)
+ .returns(3L, AppendResult::index)
+ .returns(5L, AppendResult::nonMatchingTerm);
assertLogIndices(log, 2, 1, 3);
}
@@ -391,7 +412,8 @@ public void testAppendWithConflictingTerm3() throws Exception {
// Leader 01 01 01 04 04 05 05 06 06 06
public void testRAFTPaperAppendOnLeader() throws Exception {
initB();
- RaftImpl impl=getImpl(b);
+ Address leader = leaderAddress();
+ RaftImpl impl=getImpl(channel(1));
Log log=impl.raft().log();
append(impl, 1, 0, new LogEntry(1, buf), leader, 1);
append(impl, 2, 1, new LogEntry(1, buf), leader, 1);
@@ -404,8 +426,10 @@ public void testRAFTPaperAppendOnLeader() throws Exception {
append(impl, 9, 6, new LogEntry(6, buf), leader, 1);
append(impl, 10, 6, new LogEntry(6, buf), leader, 10);
AppendResult result=append(impl, 11, 6, new LogEntry(6, buf), leader, 1);
- assertTrue(result.success());
- assertEquals(result.index(), 11);
+ assertThat(result)
+ .as("Verifying: " + result)
+ .returns(true, AppendResult::success)
+ .returns(11L, AppendResult::index);
assertLogIndices(log, 11, 10, 6);
}
@@ -415,7 +439,8 @@ public void testRAFTPaperAppendOnLeader() throws Exception {
// Flwr A 01 01 01 04 04 05 05 06 06 06 <-- add
public void testRAFTPaperScenarioA() throws Exception {
initB();
- RaftImpl impl=getImpl(b);
+ Address leader = leaderAddress();
+ RaftImpl impl=getImpl(channel(1));
Log log=impl.raft().log();
append(impl, 1, 0, new LogEntry(1, buf), leader, 1);
append(impl, 2, 1, new LogEntry(1, buf), leader, 1);
@@ -427,8 +452,10 @@ public void testRAFTPaperScenarioA() throws Exception {
append(impl, 8, 5, new LogEntry(6, buf), leader, 1);
append(impl, 9, 6, new LogEntry(6, buf), leader, 9);
AppendResult result=append(impl, 11, 6, new LogEntry(6, buf), leader, 9);
- assertFalse(result.success());
- assertEquals(result.index(), 9);
+ assertThat(result)
+ .as("Verifying: " + result)
+ .returns(false, AppendResult::success)
+ .returns(9L, AppendResult::index);
assertLogIndices(log, 9, 9, 6);
}
@@ -439,15 +466,18 @@ public void testRAFTPaperScenarioA() throws Exception {
// Flwr A 01 01 01 04 06 <-- add
public void testRAFTPaperScenarioB() throws Exception {
initB();
- RaftImpl impl=getImpl(b);
+ Address leader = leaderAddress();
+ RaftImpl impl=getImpl(channel(1));
Log log=impl.raft().log();
append(impl, 1, 0, new LogEntry(1, buf), leader, 1);
append(impl, 2, 1, new LogEntry(1, buf), leader, 1);
append(impl, 3, 1, new LogEntry(1, buf), leader, 1);
append(impl, 4, 1, new LogEntry(4, buf), leader, 4);
AppendResult result=append(impl, 11, 6, new LogEntry(6, buf), leader, 4);
- assertFalse(result.success());
- assertEquals(result.index(), 4);
+ assertThat(result)
+ .as("Verifying: " + result)
+ .returns(false, AppendResult::success)
+ .returns(4L, AppendResult::index);
assertLogIndices(log, 4, 4, 4);
}
@@ -457,7 +487,8 @@ public void testRAFTPaperScenarioB() throws Exception {
// Flwr A 01 01 01 04 04 05 05 06 06 06 06
public void testRAFTPaperScenarioC() throws Exception {
initB();
- RaftImpl impl=getImpl(b);
+ Address leader = leaderAddress();
+ RaftImpl impl=getImpl(channel(1));
Log log=impl.raft().log();
append(impl, 1, 0, new LogEntry(1, buf), leader, 1);
append(impl, 2, 1, new LogEntry(1, buf), leader, 1);
@@ -472,8 +503,10 @@ public void testRAFTPaperScenarioC() throws Exception {
append(impl, 11, 6, new LogEntry(6, buf), leader, 10);
// Overwrites existing entry; does *not* advance last_applied in log
AppendResult result=append(impl, 11, 6, new LogEntry(6, buf), leader, 10);
- assertTrue(result.success());
- assertEquals(result.index(), 11);
+ assertThat(result)
+ .as("Verifying: " + result)
+ .returns(true, AppendResult::success)
+ .returns(11L, AppendResult::index);
assertLogIndices(log, 11, 10, 6);
}
@@ -484,7 +517,8 @@ public void testRAFTPaperScenarioC() throws Exception {
// Flwr A 01 01 01 04 04 05 05 06 06 06 07 07
public void testRAFTPaperScenarioD() throws Exception {
initB();
- RaftImpl impl=getImpl(b);
+ Address leader = leaderAddress();
+ RaftImpl impl=getImpl(channel(1));
Log log=impl.raft().log();
append(impl, 1, 0, new LogEntry(1, buf), leader, 1);
append(impl, 2, 1, new LogEntry(1, buf), leader, 1);
@@ -501,20 +535,26 @@ public void testRAFTPaperScenarioD() throws Exception {
// add 11
AppendResult result=append(impl, buf, leader, 10, 6, 8, 10);
- assertTrue(result.success());
- assertEquals(result.index(), 11);
+ assertThat(result)
+ .as("Verifying: " + result)
+ .returns(true, AppendResult::success)
+ .returns(11L, AppendResult::index);
assertLogIndices(log, 11, 10, 8);
// add 12
result=append(impl, buf, leader, 11, 8, 8, 10);
- assertTrue(result.success());
- assertEquals(result.index(), 12);
+ assertThat(result)
+ .as("Verifying: " + result)
+ .returns(true, AppendResult::success)
+ .returns(12L, AppendResult::index);
assertLogIndices(log, 12, 10, 8);
// commit 12
result=append(impl, null, leader, 0, 0, 0, 12);
- assertTrue(result.success());
- assertEquals(result.index(), 12);
+ assertThat(result)
+ .as("Verifying: " + result)
+ .returns(true, AppendResult::success)
+ .returns(12L, AppendResult::index);
assertLogIndices(log, 12, 12, 8);
}
@@ -524,7 +564,8 @@ public void testRAFTPaperScenarioD() throws Exception {
// Flwr A 01 01 01 04 04 04 04
public void testRAFTPaperScenarioE() throws Exception {
initB();
- RaftImpl impl=getImpl(b);
+ Address leader = leaderAddress();
+ RaftImpl impl=getImpl(channel(1));
Log log=impl.raft().log();
append(impl, 1, 0, new LogEntry(1, buf), leader, 1);
append(impl, 2, 1, new LogEntry(1, buf), leader, 1);
@@ -537,29 +578,36 @@ public void testRAFTPaperScenarioE() throws Exception {
System.out.printf("log entries of follower before fix:\n%s", printLog(log));
AppendResult result=append(impl, 11, 6, new LogEntry(6, buf), leader, 10);
- assertFalse(result.success());
- assertEquals(result.index(), 7);
+ assertThat(result)
+ .as("Verifying: " + result)
+ .returns(false, AppendResult::success)
+ .returns(7L, AppendResult::index);
assertLogIndices(log, 7, 3, 4);
// now try to append 8 (fails because of wrong term)
result=append(impl, 8, 5, new LogEntry(6, buf), leader, 10);
- assert !result.success();
- assertEquals(result.index(), 4);
- assert result.commitIndex() == 3;
+ assertThat(result)
+ .as("Verifying: " + result)
+ .returns(false, AppendResult::success)
+ .returns(4L, AppendResult::index)
+ .returns(3L, AppendResult::commitIndex);
assertLogIndices(log, 3, 3, 1);
// now append 4-10
for(int i=4; i <= 10; i++) {
result=append(impl, i, terms[i-1], new LogEntry(terms[i], buf), leader, 10);
- assert result.success();
- assertEquals(result.index(), i);
- assert result.commitIndex() == i;
+ assertThat(result)
+ .as("Verifying: " + result)
+ .returns(true, AppendResult::success)
+ .returns((long) i, AppendResult::index)
+ .returns((long) i, AppendResult::commitIndex);
assertLogIndices(log, i, i, terms[i]);
}
System.out.printf("log entries of follower after fix:\n%s", printLog(log));
for(int i=0; i < terms.length; i++) {
LogEntry entry=log.get(i);
- assert entry == null && i == 0 || entry.term() == terms[i];
+ if (i == 0) assertThat(entry).isNull();
+ else assertThat(entry).as("Verifying: " + entry).isNotNull().returns((long) terms[i], LogEntry::term);
}
}
@@ -570,7 +618,8 @@ public void testRAFTPaperScenarioE() throws Exception {
// Flwr A 01 01 01 02 02 02 03 03 03 03 03
public void testRAFTPaperScenarioF() throws Exception {
initB();
- RaftImpl impl=getImpl(b);
+ Address leader = leaderAddress();
+ RaftImpl impl=getImpl(channel(1));
Log log=impl.raft().log();
int[] incorrect_terms={0,1,1,1,2,2,2,3,3,3,3,3};
for(int i=1; i < incorrect_terms.length; i++)
@@ -579,15 +628,19 @@ public void testRAFTPaperScenarioF() throws Exception {
System.out.printf("log entries of follower before fix:\n%s", printLog(log));
AppendResult result=append(impl, 10, 6, new LogEntry(6, buf), leader, 10);
- assert !result.success();
- assert result.index() == 7;
+ assertThat(result)
+ .as("Verifying: " + result)
+ .returns(false, AppendResult::success)
+ .returns(7L, AppendResult::index);
assertLogIndices(log, 6, 3, 2);
System.out.printf("log entries of follower after first fix:\n%s", printLog(log));
result=append(impl, 7, 5, new LogEntry(5, buf), leader, 10);
- assert !result.success();
- assert result.index() == 4;
+ assertThat(result)
+ .as("Verifying: " + result)
+ .returns(false, AppendResult::success)
+ .returns(4L, AppendResult::index);
assertLogIndices(log, 3, 3, 1);
System.out.printf("log entries of follower after second fix:\n%s", printLog(log));
@@ -595,98 +648,45 @@ public void testRAFTPaperScenarioF() throws Exception {
// now append 4-10:
for(int i=4; i <= 10; i++) {
result=append(impl, i, terms[i-1], new LogEntry(terms[i], buf), leader, 10);
- assert result.success();
- assertEquals(result.index(), i);
- assert result.commitIndex() == i;
+ assertThat(result)
+ .as("Verifying: " + result)
+ .returns(true, AppendResult::success)
+ .returns((long) i, AppendResult::index)
+ .returns((long) i, AppendResult::commitIndex);
assertLogIndices(log, i, i, terms[i]);
}
System.out.printf("log entries of follower after final fix:\n%s", printLog(log));
for(int i=0; i < terms.length; i++) {
LogEntry entry=log.get(i);
- assert entry == null && i == 0 || entry.term() == terms[i];
- }
- }
-
-
- protected JChannel create(String name) throws Exception {
- return create(name, r -> r);
- }
-
- protected JChannel create(String name, Function config) throws Exception {
- return create(name, members, config);
- }
-
- protected static JChannel create(String name, final List members) throws Exception {
- return create(name, members, r -> r);
- }
-
- protected static JChannel create(String name, final List members, Function config) throws Exception {
- ELECTION election=new ELECTION();
- RAFT raft=config.apply(new RAFT()).members(members).raftId(name)
- .logClass("org.jgroups.protocols.raft.InMemoryLog").logPrefix(name + "-" + CLUSTER);
- return new JChannel(Util.getTestStack(election, raft, new REDIRECT())).name(name);
- }
-
-
- protected static void close(JChannel... channels) {
- for(JChannel ch: channels) {
- if(ch == null)
- continue;
- Util.close(ch);
- RAFT raft=ch.getProtocolStack().findProtocol(RAFT.class);
- try {
- Utils.deleteLog(raft);
- }
- catch(Exception ignored) {}
+ if (i == 0) assertThat(entry).isNull();
+ else assertThat(entry).isNotNull().returns((long) terms[i], LogEntry::term);
}
}
protected void init(boolean verbose) throws Exception {
- init(verbose, r -> r);
- }
-
- protected void init(boolean verbose, Function config) throws Exception {
- a=create("A", config); // leader
- as=new ReplicatedStateMachine<>(a);
- a.connect(CLUSTER);
-
- b=create("B", config); // follower
- bs=new ReplicatedStateMachine<>(b);
- b.connect(CLUSTER);
+ withClusterSize(3);
+ createCluster();
- c=create("C", config); // follower
- cs=new ReplicatedStateMachine<>(c);
- c.connect(CLUSTER);
- Util.waitUntilAllChannelsHaveSameView(10000, 500, a,b,c);
Util.waitUntil(5000, 100,
- () -> Stream.of(a,b,c).map(AppendEntriesTest::raft).allMatch(r -> r.leader() != null),
- () -> Stream.of(a,b,c).map(ch -> String.format("%s: leader=%s", ch.getAddress(), raft(ch).leader()))
+ () -> Stream.of(channels()).map(this::raft).allMatch(r -> r.leader() != null),
+ () -> Stream.of(channels()).map(ch -> String.format("%s: leader=%s", ch.getAddress(), raft(ch).leader()))
.collect(Collectors.joining("\n")));
- for(int i=0; i < 20; i++) {
- if(isLeader(a) && !isLeader(b) && !isLeader(c))
- break;
- Util.sleep(500);
- }
if(verbose) {
- System.out.println("A: is leader? -> " + isLeader(a));
- System.out.println("B: is leader? -> " + isLeader(b));
- System.out.println("C: is leader? -> " + isLeader(c));
+ System.out.println("A: is leader? -> " + RaftTestUtils.isRaftLeader(channel(0)));
+ System.out.println("B: is leader? -> " + RaftTestUtils.isRaftLeader(channel(1)));
+ System.out.println("C: is leader? -> " + RaftTestUtils.isRaftLeader(channel(2)));
}
- assert isLeader(a);
- assert !isLeader(b);
- assert !isLeader(c);
- }
- protected void initB() throws Exception {
- b=create("B"); // follower
- raft(b).stateMachine(new DummyStateMachine());
- b.connect(CLUSTER);
+ assertThat(RaftTestUtils.isRaftLeader(channel(0))).isTrue();
+ assertThat(RaftTestUtils.isRaftLeader(channel(1))).isFalse();
+ assertThat(RaftTestUtils.isRaftLeader(channel(2))).isFalse();
}
- protected static boolean isLeader(JChannel ch) {
- RAFT raft=raft(ch);
- return raft.leader() != null && ch.getAddress().equals(raft.leader());
+ protected void initB() throws Exception {
+ withClusterSize(2);
+ createCluster();
+ close(0);
}
protected static RaftImpl getImpl(JChannel ch) {
@@ -700,118 +700,37 @@ protected static String printLog(Log l) {
return sb.toString();
}
- protected static void assertLeader(JChannel ch, long timeout, long interval) {
- RAFT raft=raft(ch);
- long stop_time=System.currentTimeMillis() + timeout;
- while(System.currentTimeMillis() < stop_time) {
- if(raft.isLeader())
- break;
- Util.sleep(interval);
- }
- assert raft.isLeader();
- }
-
- protected void assertPresent(int key, int value, ReplicatedStateMachine ... rsms) throws Exception {
- if(rsms == null || rsms.length == 0)
- rsms=new ReplicatedStateMachine[]{as,bs,cs};
- for(int i=0; i < 10; i++) {
- boolean found=true;
- for(ReplicatedStateMachine rsm: rsms) {
- Integer val=rsm.get(key);
- if(!Objects.equals(val, value)) {
- found=false;
- break;
- }
- }
- if(found)
- break;
- Util.sleep(500);
- }
-
- for(ReplicatedStateMachine rsm: rsms) {
- Integer val=rsm.get(key);
- assert Objects.equals(val, value);
- System.out.println("rsm = " + rsm);
- }
- }
-
- @SafeVarargs
- protected final void assertSame(ReplicatedStateMachine... rsms) {
- assertSame(TIMEOUT, INTERVAL, rsms);
- }
-
-
- @SafeVarargs
- protected final void assertSame(long timeout, long interval, ReplicatedStateMachine... rsms) {
- // Wait until the leader receives the responses and all have matching commit indexes.
- List channels = Stream.of(rsms).map(ReplicatedStateMachine::channel).collect(Collectors.toList());
- BooleanSupplier bs = () -> channels.stream()
- .map(AppendEntriesTest::raft)
- .map(RAFT::commitIndex)
- .distinct()
- .count() == 1;
- assert Util.waitUntilTrue(timeout, interval, bs) : generateErrorMessage(rsms);
-
- System.out.println(dumpStateMachines(rsms));
-
- // Wait until all state machines are equal. Meaning they all applied the same commands.
- // In this point, the commit indexes are equals, meaning up to a point everything was applied.
- assert Util.waitUntilTrue(timeout, interval, () -> Stream.of(rsms).distinct().count() == 1)
- : generateErrorMessage(rsms) + " where " + dumpStateMachines(rsms);
- }
-
- private static String dumpStateMachines(ReplicatedStateMachine,?>... rsms) {
- StringBuilder sb = new StringBuilder();
- for (ReplicatedStateMachine, ?> rsm : rsms) {
- sb.append(rsm.raftId())
- .append(" -> ")
- .append(rsm)
- .append("\n");
- }
- return sb.toString();
- }
-
- private static String generateErrorMessage(ReplicatedStateMachine,?>... rsms) {
- StringBuilder sb = new StringBuilder();
- RAFT leader = raft(rsms[0].channel());
- sb.append(leader.raftId())
- .append(": commit-index=").append(leader.commitIndex())
- .append(leader.dumpCommitTable());
- return sb.toString();
- }
-
protected static void assertLogIndices(Log log, int last_appended, int commit_index, int term) {
- assertEquals(log.lastAppended(), last_appended);
- assertEquals(log.commitIndex(), commit_index);
- assertEquals(log.currentTerm(), term);
+ assertThat(log.lastAppended()).isEqualTo(last_appended);
+ assertThat(log.commitIndex()).isEqualTo(commit_index);
+ assertThat(log.currentTerm()).isEqualTo(term);
}
- protected static void assertCommitIndex(long timeout, long interval, long expected_commit, long expected_applied,
- JChannel... channels) {
- long target_time=System.currentTimeMillis() + timeout;
- while(System.currentTimeMillis() <= target_time) {
+ protected void assertCommitIndex(long timeout, long expected_commit, long expected_applied, JChannel... channels) {
+ BooleanSupplier bs = () -> {
boolean all_ok=true;
for(JChannel ch: channels) {
RAFT raft=raft(ch);
if(expected_commit != raft.commitIndex() || expected_applied != raft.lastAppended())
all_ok=false;
}
- if(all_ok)
- break;
- Util.sleep(interval);
- }
+ return all_ok;
+ };
+ assertThat(eventually(bs, timeout, TimeUnit.MILLISECONDS))
+ .as("Commit indexes never matched")
+ .isTrue();
+
for(JChannel ch: channels) {
RAFT raft=raft(ch);
- System.out.printf("%s: last-applied=%d, commit-index=%d\n", ch.getAddress(), raft.lastAppended(), raft.commitIndex());
- assert raft.commitIndex() == expected_commit && raft.lastAppended() == expected_applied
- : String.format("%s: last-applied=%d, commit-index=%d", ch.getAddress(), raft.lastAppended(), raft.commitIndex());
+ String check = String.format("%s: last-applied=%d, commit-index=%d\n", ch.getAddress(), raft.lastAppended(), raft.commitIndex());
+ System.out.printf(check);
+ assertThat(raft)
+ .as(check)
+ .returns(expected_commit, RAFT::commitIndex)
+ .returns(expected_applied, RAFT::lastAppended);
}
}
- protected static RAFT raft(JChannel ch) {
- return ch.getProtocolStack().findProtocol(RAFT.class);
- }
-
protected static AppendResult append(RaftImpl impl, long index, long prev_term, LogEntry entry, Address leader,
long leader_commit) throws Exception {
return append(impl, entry.command(), leader, Math.max(0, index-1), prev_term, entry.term(), leader_commit);
diff --git a/tests/junit-functional/org/jgroups/tests/DynamicMembershipTest.java b/tests/junit-functional/org/jgroups/tests/DynamicMembershipTest.java
index 02124ce9..117429f8 100644
--- a/tests/junit-functional/org/jgroups/tests/DynamicMembershipTest.java
+++ b/tests/junit-functional/org/jgroups/tests/DynamicMembershipTest.java
@@ -3,29 +3,36 @@
import org.jgroups.Address;
import org.jgroups.Global;
import org.jgroups.JChannel;
-import org.jgroups.protocols.raft.ELECTION;
import org.jgroups.protocols.raft.FileBasedLog;
-import org.jgroups.protocols.raft.InMemoryLog;
import org.jgroups.protocols.raft.RAFT;
-import org.jgroups.protocols.raft.REDIRECT;
-import org.jgroups.raft.util.Utils;
+import org.jgroups.raft.testfwk.RaftTestUtils;
+import org.jgroups.tests.harness.BaseRaftChannelTest;
+import org.jgroups.tests.harness.BaseRaftElectionTest;
+import org.jgroups.tests.harness.RaftAssertion;
import org.jgroups.util.Bits;
import org.jgroups.util.CompletableFutures;
import org.jgroups.util.Util;
-import org.testng.annotations.AfterMethod;
-import org.testng.annotations.Test;
-
import java.util.ArrayList;
import java.util.Arrays;
+import java.util.Collection;
import java.util.HashSet;
import java.util.List;
+import java.util.Objects;
+import java.util.Set;
import java.util.concurrent.CompletableFuture;
import java.util.concurrent.CountDownLatch;
import java.util.concurrent.TimeUnit;
import java.util.concurrent.TimeoutException;
+import java.util.function.BooleanSupplier;
+import java.util.function.Supplier;
import java.util.stream.Collectors;
-import java.util.stream.Stream;
+
+import org.assertj.core.api.Assertions;
+import org.testng.annotations.AfterMethod;
+import org.testng.annotations.Test;
+
+import static org.assertj.core.api.Assertions.assertThat;
/**
* Tests the addServer() / removeServer) functionality
@@ -33,48 +40,40 @@
* @since 0.2
*/
@Test(groups=Global.FUNCTIONAL,singleThreaded=true)
-public class DynamicMembershipTest {
- protected JChannel[] channels;
- protected RAFT[] rafts;
- protected Address leader;
- protected List mbrs;
- protected static final String CLUSTER=DynamicMembershipTest.class.getSimpleName();
- private static final Class> DEFAULT_LOG=InMemoryLog.class;
-
- @AfterMethod protected void destroy() throws Exception {
- close(channels);
+public class DynamicMembershipTest extends BaseRaftChannelTest {
+
+ {
+ // We want to change the members.
+ createManually = true;
+ }
+
+ @AfterMethod
+ protected void destroy() throws Exception {
+ destroyCluster();
}
/** Start a member not in {A,B,C} -> expects an exception */
- public void testStartOfNonMember() {
- JChannel non_member=null;
- try {
- init("A", "B", "C");
- channels=Arrays.copyOf(channels, channels.length+1);
- channels[channels.length-1]=create("X");
- assert false : "Starting a non-member should throw an exception";
- }
- catch(Exception e) {
- System.out.println("received exception (as expected): " + e);
- }
- finally {
- close(non_member);
- }
+ public void testStartOfNonMember() throws Exception {
+ withClusterSize(3);
+ createCluster();
+
+ JChannel nonMember = createDisconnectedChannel("X");
+ Assertions.assertThatThrownBy(() -> nonMember.connect(clusterName()))
+ .isInstanceOf(IllegalStateException.class)
+ .hasMessage(String.format("raft-id X is not listed in members %s", getRaftMembers()));
+ close(nonMember);
}
/** Calls addServer() on non-leader. Calling {@link org.jgroups.protocols.raft.RAFT#addServer(String)}
* must throw an exception */
public void testMembershipChangeOnNonLeader() throws Exception {
- init("A","B");
- RAFT raft=raft(channels[1]); // non-leader B
- try {
- raft.addServer("X");
- assert false : "Calling RAFT.addServer() on a non-leader must throw an exception";
- }
- catch(Exception ex) {
- System.out.println("received exception calling RAFT.addServer() on a non-leader (as expected): " + ex);
- }
+ withClusterSize(2);
+ createCluster();
+ RAFT raft=raft(1); // non-leader B
+ assertThat(raft).isNotNull();
+ // Operation should fail without needing to wait on CF.
+ RaftAssertion.assertLeaderlessOperationThrows(() -> raft.addServer("X"));
}
/** {A,B,C} +D +E -E -D. Note that we can _add_ a 6th server, as adding it requires the majority of the existing
@@ -82,24 +81,26 @@ public void testMembershipChangeOnNonLeader() throws Exception {
* the 6th server. This is because we only have 3 'real' channels (members).
*/
public void testSimpleAddAndRemove() throws Exception {
- init("A", "B", "C");
- leader=leader(10000, 500, channels);
+ withClusterSize(3);
+ createCluster();
+
+ waitUntilAllRaftsHaveLeader(channels());
+ Address leader=leaderAddress();
System.out.println("leader = " + leader);
- assert leader != null;
- waitUntilAllRaftsHaveLeader(channels);
- assertSameLeader(leader, channels);
- assertMembers(5000, 200, mbrs, 2, channels);
+ assertThat(leader).isNotNull();
+ assertSameLeader(leader, channels());
+ assertMembers(5000, getRaftMembers(), 2, channels());
List new_mbrs=List.of("D", "E");
RAFT raft=raft(leader);
- List expected_mbrs=new ArrayList<>(mbrs);
+ List expected_mbrs=new ArrayList<>(getRaftMembers());
// adding:
for(String mbr: new_mbrs) {
System.out.printf("\nAdding %s\n", mbr);
raft.addServer(mbr);
expected_mbrs.add(mbr);
- assertMembers(10000, 200, expected_mbrs, expected_mbrs.size()/2+1, channels);
+ assertMembers(10000, expected_mbrs, expected_mbrs.size()/2+1, channels());
}
// removing:
@@ -108,28 +109,35 @@ public void testSimpleAddAndRemove() throws Exception {
System.out.printf("\nRemoving %s\n", mbr);
raft.removeServer(mbr);
expected_mbrs.remove(mbr);
- assertMembers(10000, 200, expected_mbrs, expected_mbrs.size()/2+1, channels);
+ assertMembers(10000, expected_mbrs, expected_mbrs.size()/2+1, channels());
}
}
/**
* Tests that after adding a new member, this information persists through restarts. Since the cluster restarts,
* it uses a persistent log instead of an in-memory.
- *
- * The cluster starts with {A, B}, then X joins, and {A, B, X} restarts. Information about X is still in {A, B}.
+ *
+ * The cluster starts with {A, B}, then C joins, and {A, B, C} restarts. Information about C is still in {A, B}.
+ *
*/
public void testMembersRemainAfterRestart() throws Exception {
- Class> log=FileBasedLog.class;
// In memory log will lose the snapshot on restart.
- init(log, "A", "B");
- leader=leader(10000, 500, channels);
+ withClusterSize(2);
+ createCluster();
+
+ for (JChannel ch : channels()) {
+ RAFT r = raft(ch);
+ r.logClass(FileBasedLog.class.getName());
+ }
+
+ waitUntilAllRaftsHaveLeader(channels());
+ Address leader=leaderAddress();
System.out.println("leader = " + leader);
- assert leader != null : "Leader still null";
+ assertThat(leader).isNotNull();
- waitUntilAllRaftsHaveLeader(channels);
- assertSameLeader(leader, channels);
- assertMembers(5000, 200, mbrs, 2, channels);
+ assertSameLeader(leader, channels());
+ assertMembers(5000, getRaftMembers(), 2, channels());
RAFT raft=raft(leader);
// Fill the log with some entries.
@@ -140,49 +148,70 @@ public void testMembersRemainAfterRestart() throws Exception {
f.get(10, TimeUnit.SECONDS);
}
- String new_mbr="X";
+ String new_mbr="C";
System.out.printf("Adding [%s]\n", new_mbr);
- raft.addServer(new_mbr);
- mbrs.add(new_mbr);
- channels=Arrays.copyOf(channels, channels.length+1);
- channels[channels.length-1]=create(new_mbr, log);
- assertMembers(10000, 200, mbrs, 2, channels);
+ withClusterSize(3);
+ createCluster();
+
+ // Also set 'C' to use file based log.
+ raft(2).logClass(FileBasedLog.class.getName());
+
+ raft.addServer(new_mbr).get(10, TimeUnit.SECONDS);
+ assertMembers(10000, getRaftMembers(), 2, channels());
System.out.println("\nShutdown cluster");
- for (JChannel channel : channels) {
- RAFT r=raft(channel);
+ JChannel[] channels = channels();
+ for (int i = 0; i < channels.length; i++) {
+ JChannel ch = channels[i];
+ RAFT r = raft(ch);
r.snapshot();
- channel.close();
+ Util.close(ch);
+ channels[i] = null;
}
+ destroyCluster();
System.out.println("\nRestarting cluster");
+ // We restart with only 2 nodes.
+ withClusterSize(2);
+ createCluster();
+ for (JChannel ch : channels()) {
+ RAFT r = raft(ch);
+ r.logClass(FileBasedLog.class.getName());
+ }
+
// Nodes restart using the file configuration/previous configuration.
- // Should restore member `X` from log.
- init(log, "A", "B");
- assert !mbrs.contains(new_mbr) : "New member should not be in initial configuration";
+ // Should restore member `C` from log.
+ assertThat(raft(0).members())
+ .as("New member should not be in initial configuration")
+ .contains(new_mbr);
- List extended_mbrs=new ArrayList<>(mbrs);
+ List extended_mbrs=new ArrayList<>(getRaftMembers());
extended_mbrs.add(new_mbr);
- assertMembers(10000, 200, extended_mbrs, 2, channels);
+ assertMembers(10_000, extended_mbrs, 2, channels());
+
+ withClusterSize(3);
+ createCluster();
+ // Also set 'C' to use file based log.
+ raft(2).logClass(FileBasedLog.class.getName());
- JChannel extraneous=create(new_mbr, log);
- close(extraneous);
+ assertMembers(10_000, extended_mbrs, 2, channels());
}
/**
* {A,B,C} +D +E +F +G +H +I +J
*/
public void testAddServerSimultaneously() throws Exception {
- init("A", "B", "C", "D");
- leader = leader(10000, 500, channels);
+ withClusterSize(4);
+ createCluster();
+ waitUntilAllRaftsHaveLeader(channels());
+ Address leader = leaderAddress();
System.out.println("leader = " + leader);
- assert leader != null;
- waitUntilAllRaftsHaveLeader(channels);
- assertSameLeader(leader, channels);
- assertMembers(5000, 500, mbrs, mbrs.size()/2+1, channels);
+ assertThat(leader).isNotNull();
+ assertSameLeader(leader, channels());
+ assertMembers(5000, getRaftMembers(), getRaftMembers().size()/2+1, channels());
final RAFT raft = raft(leader);
@@ -204,11 +233,9 @@ public void testAddServerSimultaneously() throws Exception {
}
addServerLatch.countDown();
- List expected_mbrs=new ArrayList<>(this.mbrs);
+ List expected_mbrs=new ArrayList<>(getRaftMembers());
expected_mbrs.addAll(newServers);
- assertMembers(20000, 500, expected_mbrs, expected_mbrs.size()/2+1, channels);
- System.out.printf("\nmembers:\n%s\n", Stream.of(rafts).map(r -> String.format("%s: %s", r.getAddress(), r.members()))
- .collect(Collectors.joining("\n")));
+ assertMembers(20000, expected_mbrs, expected_mbrs.size()/2+1, channels());
}
/**
@@ -216,109 +243,58 @@ public void testAddServerSimultaneously() throws Exception {
* make B rejoin, and addServer("C") will succeed
*/
public void testAddServerOnLeaderWhichCantCommit() throws Exception {
- init("A", "B");
- Util.waitUntilAllChannelsHaveSameView(10000, 500, channels);
- leader=leader(10000, 500, channels);
+ withClusterSize(2);
+ createCluster();
+
+ waitUntilAllRaftsHaveLeader(channels());
+ Address leader=leaderAddress();
System.out.println("leader = " + leader);
- assert leader != null;
+ assertThat(leader).isNotNull();
// close non-leaders
- for(JChannel ch: channels)
- if(!ch.getAddress().equals(leader))
- close(ch);
+ close(channel(1));
- RAFT raft=raft(leader);
- try { // this will fail as leader A stepped down when it found that the view's size dropped below the majority
- raft.addServer("C").get(10, TimeUnit.SECONDS);
- assert false : "Adding server C should fail as the leader stepped down";
- } catch(Exception ex) {
- System.out.println("Caught exception (as expected) trying to add C: " + ex);
- assert !raft.isLeader() : "Still seen as leader!";
- }
+ RaftAssertion.assertLeaderlessOperationThrows(
+ () -> raft(0).addServer("C").get(10, TimeUnit.SECONDS),
+ "Adding member without leader");
- // Now start B again, so that addServer("C") can succeed
- for(int i=0; i < channels.length; i++) {
- if(channels[i].isClosed())
- channels[i]=create(String.valueOf((char)('A' + i)));
- }
- Util.waitUntilAllChannelsHaveSameView(10000, 500, channels);
+ RAFT raft = raft(leader);
+ assertThat(raft.isLeader()).isFalse();
+ assertThat(raft(0).isLeader()).isFalse();
- leader=leader(10000, 500, channels);
+ // Now start B again, so that addServer("C") can succeed
+ createCluster();
+ waitUntilAllRaftsHaveLeader(channels());
+ leader=leaderAddress();
System.out.println("leader = " + leader);
- assert leader != null;
+ assertThat(leader).isNotNull();
raft=raft(leader);
- raft.addServer("C"); // adding C should now succeed, as we have a valid leader again
+
+ System.out.println("-- adding member C");
+ CompletableFuture addC = raft.addServer("C"); // adding C should now succeed, as we have a valid leader again
// Now create and connect C
- channels=Arrays.copyOf(channels, 3);
- if(!this.mbrs.contains("C"))
- this.mbrs.add("C");
- channels[2]=create("C");
+ withClusterSize(3);
+ createCluster();
- assertMembers(10000, 500, mbrs, 2, channels);
+ addC.get(10, TimeUnit.SECONDS);
+ assertMembers(10000, getRaftMembers(), 2, channels());
// wait until everyone has committed the addServer(C) operation
- assertCommitIndex(20000, 500, raft(leader).lastAppended(), channels);
- }
-
- protected void init(String ... nodes) throws Exception {
- init(DEFAULT_LOG, nodes);
- }
-
- protected void init(Class> log, String ... nodes) throws Exception {
- mbrs=new ArrayList<>(List.of(nodes));
- channels=new JChannel[nodes.length];
- rafts=new RAFT[nodes.length];
- for(int i=0; i < nodes.length; i++) {
- channels[i]=create(nodes, i, log);
- rafts[i]=raft(channels[i]);
- }
- }
-
- protected JChannel create(String name) throws Exception {
- return create(name, DEFAULT_LOG);
- }
-
- protected JChannel create(String name, Class> log) throws Exception {
- RAFT raft=new RAFT().members(mbrs).raftId(name).stateMachine(new DummyStateMachine())
- .logClass(log.getCanonicalName()).logPrefix(name + "-" + CLUSTER);
- JChannel ch=new JChannel(Util.getTestStack(new ELECTION(), raft, new REDIRECT())).name(name);
- ch.connect(CLUSTER);
- return ch;
- }
-
- protected static JChannel create(String[] names, int index, Class> log) throws Exception {
- RAFT raft=new RAFT().members(Arrays.asList(names)).raftId(names[index]).stateMachine(new DummyStateMachine())
- .logClass(log.getCanonicalName()).logPrefix(names[index] + "-" + CLUSTER)
- .resendInterval(500);
- JChannel ch=new JChannel(Util.getTestStack(new ELECTION(), raft, new REDIRECT())).name(names[index]);
- ch.connect(CLUSTER);
- return ch;
- }
-
- protected static Address leader(long timeout, long interval, JChannel ... channels) {
- long target_time=System.currentTimeMillis() + timeout;
- while(System.currentTimeMillis() <= target_time) {
- for(JChannel ch: channels) {
- if(ch.isConnected() && raft(ch).isLeader())
- return raft(ch).leader();
- }
- Util.sleep(interval);
- }
- return null;
+ assertCommitIndex(20000, raft(leader).lastAppended(), channels());
}
- protected static void assertSameLeader(Address leader, JChannel... channels) {
+ protected void assertSameLeader(Address leader, JChannel... channels) {
for(JChannel ch: channels) {
final Address raftLeader = raft(ch).leader();
- assert leader.equals(raftLeader)
- : String.format("expected leader to be '%s' but was '%s'", leader, raftLeader);
+ assertThat(raftLeader)
+ .as(() -> String.format("expected leader to be '%s' but was '%s'", leader, raftLeader))
+ .satisfiesAnyOf(l -> assertThat(l).isNull(), l -> assertThat(l).isEqualTo(leader));
}
}
- protected static void assertMembers(long timeout, long interval, List members, int expected_majority, JChannel... channels) {
- long target_time=System.currentTimeMillis() + timeout;
- while(System.currentTimeMillis() <= target_time) {
+ protected void assertMembers(long timeout, Collection members, int expected_majority, JChannel... channels) {
+ BooleanSupplier bs = () -> {
boolean all_ok=true;
for(JChannel ch: channels) {
if(!ch.isConnected())
@@ -327,76 +303,74 @@ protected static void assertMembers(long timeout, long interval, List me
if(!new HashSet<>(raft.members()).equals(new HashSet<>(members)))
all_ok=false;
}
- if(all_ok)
- break;
- Util.sleep(interval);
- }
+ return all_ok;
+ };
+ Supplier message = () -> Arrays.stream(channels)
+ .map(ch -> {
+ if (!ch.isConnected()) {
+ return String.format("%s -- disconnected", ch.getName());
+ }
+ RAFT r = raft(ch);
+ return String.format("%s: %s", r.raftId(), r.members());
+ })
+ .collect(Collectors.joining(System.lineSeparator())) + " while waiting for " + members;
+
+ assertThat(RaftTestUtils.eventually(bs, timeout, TimeUnit.MILLISECONDS))
+ .as(message)
+ .isTrue();
+
for(JChannel ch: channels) {
if(!ch.isConnected())
continue;
+
RAFT raft=raft(ch);
System.out.printf("%s: members=%s, majority=%d\n", ch.getAddress(), raft.members(), raft.majority());
- assert new HashSet<>(raft.members()).equals(new HashSet<>(members))
- : String.format("expected members=%s, actual members=%s", members, raft.members());
- assert raft.majority() == expected_majority
- : ch.getName() + ": expected majority=" + expected_majority + ", actual=" + raft.majority();
+ assertThat(Set.of(raft.members()))
+ .as(() -> String.format("expected members=%s, actual members=%s", members, raft.members()))
+ .containsExactlyInAnyOrderElementsOf(Set.of(raft.members()));
+
+ assertThat(raft.majority())
+ .as(() -> ch.getName() + ": expected majority=" + expected_majority + ", actual=" + raft.majority())
+ .isEqualTo(expected_majority);
}
}
- protected static void assertCommitIndex(long timeout, long interval, long expected_commit, JChannel... channels) {
- long target_time=System.currentTimeMillis() + timeout;
- while(System.currentTimeMillis() <= target_time) {
+ protected void assertCommitIndex(long timeout, long expected_commit, JChannel... channels) {
+ BooleanSupplier bs = () -> {
boolean all_ok=true;
for(JChannel ch: channels) {
RAFT raft=raft(ch);
if(expected_commit != raft.commitIndex())
all_ok=false;
}
- if(all_ok)
- break;
- Util.sleep(interval);
- }
+ return all_ok;
+ };
+
+ assertThat(RaftTestUtils.eventually(bs, timeout, TimeUnit.MILLISECONDS))
+ .as("Commit index never match between channels")
+ .isTrue();
+
for(JChannel ch: channels) {
RAFT raft=raft(ch);
System.out.printf("%s: members=%s, last-applied=%d, commit-index=%d\n", ch.getAddress(), raft.members(),
raft.lastAppended(), raft.commitIndex());
- assert raft.commitIndex() == expected_commit : String.format("%s: last-applied=%d, commit-index=%d",
- ch.getAddress(), raft.lastAppended(), raft.commitIndex());
+
+ assertThat(raft.commitIndex())
+ .as(() -> String.format("%s: last-applied=%d, commit-index=%d", ch.getAddress(), raft.lastAppended(), raft.commitIndex()))
+ .isEqualTo(expected_commit);
}
}
- protected static void waitUntilAllRaftsHaveLeader(JChannel[] channels) throws TimeoutException {
- Util.waitUntil(5000, 250, () -> Arrays.stream(channels).allMatch(ch -> raft(ch).leader() != null));
+ protected void waitUntilAllRaftsHaveLeader(JChannel[] channels) throws TimeoutException {
+ RAFT[] rafts = Arrays.stream(channels)
+ .filter(Objects::nonNull)
+ .map(this::raft)
+ .toArray(RAFT[]::new);
+ BaseRaftElectionTest.waitUntilLeaderElected(rafts, 10_000);
}
protected RAFT raft(Address addr) {
return raft(channel(addr));
}
-
- protected JChannel channel(Address addr) {
- for(JChannel ch: channels) {
- if(ch.getAddress() != null && ch.getAddress().equals(addr))
- return ch;
- }
- return null;
- }
-
- protected static RAFT raft(JChannel ch) {
- return ch.getProtocolStack().findProtocol(RAFT.class);
- }
-
- protected static void close(JChannel... channels) {
- for(JChannel ch: channels) {
- if(ch == null)
- continue;
- RAFT raft=ch.getProtocolStack().findProtocol(RAFT.class);
- try {
- Utils.deleteLog(raft);
- }
- catch(Exception ignored) {}
- Util.close(ch);
- }
- }
-
}
diff --git a/tests/junit-functional/org/jgroups/tests/ElectionsTest.java b/tests/junit-functional/org/jgroups/tests/ElectionsTest.java
index 803380ed..7cbea6fd 100644
--- a/tests/junit-functional/org/jgroups/tests/ElectionsTest.java
+++ b/tests/junit-functional/org/jgroups/tests/ElectionsTest.java
@@ -7,178 +7,133 @@
import org.jgroups.protocols.raft.LogEntries;
import org.jgroups.protocols.raft.LogEntry;
import org.jgroups.protocols.raft.RAFT;
-import org.jgroups.protocols.raft.REDIRECT;
import org.jgroups.protocols.raft.election.BaseElection;
-import org.jgroups.raft.util.Utils;
-import org.jgroups.tests.election.BaseElectionTest;
-import org.jgroups.util.Util;
-import org.testng.annotations.AfterMethod;
-import org.testng.annotations.BeforeMethod;
-import org.testng.annotations.Test;
+import org.jgroups.raft.testfwk.RaftTestUtils;
+import org.jgroups.tests.harness.BaseRaftElectionTest;
import java.util.ArrayList;
import java.util.Arrays;
import java.util.List;
-import java.util.function.Supplier;
-
-import static org.jgroups.tests.election.BaseElectionTest.ALL_ELECTION_CLASSES_PROVIDER;
-
- /**
- * Tests elections
- * @author Bela Ban
- * @since 0.2
- */
- @Test(groups=Global.FUNCTIONAL,singleThreaded=true, dataProvider=ALL_ELECTION_CLASSES_PROVIDER)
- public class ElectionsTest extends BaseElectionTest {
- protected JChannel a,b,c;
- protected static final String CLUSTER="ElectionsTest";
- protected final List members=Arrays.asList("A", "B", "C");
- protected static final byte[] BUF={};
-
- @BeforeMethod protected void init() throws Exception {
- a=create("A"); a.connect(CLUSTER);
- b=create("B"); b.connect(CLUSTER);
- c=create("C"); c.connect(CLUSTER);
- Util.waitUntilAllChannelsHaveSameView(10000, 500, a,b,c);
- }
-
- @AfterMethod protected void destroy() {
- close(c, b, a);
- }
-
-
- /** All members have the same (initial) logs, so any member can be elected as leader */
- public void testSimpleElection(Class> ignore) throws Exception {
- assertLeader(20, 500, null, a,b,c);
- }
-
-
- /** B and C have longer logs than A: one of {B,C} must become coordinator, but *not* A */
- public void testElectionWithLongLog(Class> ignore) throws Exception {
- setLog(b, 1,1,2);
- setLog(c, 1,1,2);
-
- JChannel coord=findCoord(a,b,c);
- System.out.printf("\n\n-- starting the voting process on %s:\n", coord.getAddress());
- BaseElection el=coord.getProtocolStack().findProtocol(electionClass);
-
- // Assert that B and C have a longer log.
- long aSize = logSize(a);
- assert aSize < logSize(b) : "A log longer than B";
- assert aSize < logSize(c) : "A log longer than C";
-
- el.startVotingThread();
- Util.waitUntilTrue(5000, 500, () -> !el.isVotingThreadRunning());
-
- Address leader=assertLeader(20, 500, null, a, b, c);
- assert leader.equals(b.getAddress()) || leader.equals(c.getAddress()) : "Leader was " + leader;
- assert !leader.equals(a.getAddress());
- }
-
- /** ELECTION should look for RAFT or its subclasses */
- public void testRAFTSubclass(Class> ignore) throws Exception {
- close(c);
- c=createWithRAFTSubclass("C");
- c.connect(CLUSTER);
- }
-
-
- protected static JChannel findCoord(JChannel... channels) {
- for(JChannel ch: channels)
- if(ch.getView().getCoord().equals(ch.getAddress()))
- return ch;
- return null;
- }
-
- protected JChannel createWithRAFTSubclass(String name) throws Exception {
- return create(name, () -> new RAFT(){});
- }
-
- protected JChannel create(String name) throws Exception {
- return create(name, RAFT::new);
- }
-
- protected JChannel create(String name, Supplier raftSupplier) throws Exception {
- BaseElection election=instantiate();
- RAFT raft=raftSupplier.get().members(members).raftId(name)
- .logClass("org.jgroups.protocols.raft.InMemoryLog").logPrefix(name + "-" + CLUSTER);
- REDIRECT client=new REDIRECT();
- return new JChannel(Util.getTestStack(election, raft, client)).name(name);
- }
-
-
- protected static void close(JChannel... channels) {
- for(JChannel ch: channels) {
- if(ch == null)
- continue;
- close(ch);
- }
- }
-
- protected static void close(JChannel ch) {
- if(ch == null)
- return;
- RAFT raft=ch.getProtocolStack().findProtocol(RAFT.class);
- try {
- Utils.deleteLog(raft);
- }
- catch(Exception ignored) {}
- Util.close(ch);
- }
-
- protected static void setLog(JChannel ch, int... terms) {
- RAFT raft=ch.getProtocolStack().findProtocol(RAFT.class);
- Log log=raft.log();
- long index=log.lastAppended();
- LogEntries le=new LogEntries();
- for(int term: terms)
- le.add(new LogEntry(term, BUF));
- log.append(index+1, le);
- }
-
- private static long logSize(JChannel ch) {
- RAFT raft=ch.getProtocolStack().findProtocol(RAFT.class);
- Log log=raft.log();
- return log.size();
- }
-
-
- protected static boolean isLeader(JChannel ch) {
- RAFT raft=ch.getProtocolStack().findProtocol(RAFT.class);
- return ch.getAddress().equals(raft.leader());
- }
-
- protected static List leaders(JChannel... channels) {
- List leaders=new ArrayList<>(channels.length);
- for(JChannel ch: channels) {
- if(isLeader(ch))
- leaders.add(ch.getAddress());
- }
- return leaders;
- }
-
- /** If expected is null, then any member can be a leader */
- protected static Address assertLeader(int times, long sleep, Address expected, JChannel... channels) {
- // wait until there is 1 leader
- for(int i=0; i < times; i++) {
- List leaders=leaders(channels);
- if(!leaders.isEmpty()) {
- int size=leaders.size();
- assert size <= 1;
- Address leader=leaders.get(0);
- System.out.println("leader: " + leader);
- assert expected == null || expected.equals(leader);
- break;
- }
-
- Util.sleep(sleep);
- }
- List leaders=leaders(channels);
- assert leaders.size() == 1 : "leaders=" + leaders;
- Address leader=leaders.get(0);
- System.out.println("leader = " + leader);
- return leader;
- }
-
-
- }
+
+import org.testng.annotations.Test;
+
+import static org.assertj.core.api.Assertions.assertThat;
+import static org.jgroups.tests.harness.BaseRaftElectionTest.ALL_ELECTION_CLASSES_PROVIDER;
+
+/**
+ * Tests elections
+ * @author Bela Ban
+ * @since 0.2
+ */
+@Test(groups = Global.FUNCTIONAL, singleThreaded = true, dataProvider = ALL_ELECTION_CLASSES_PROVIDER)
+public class ElectionsTest extends BaseRaftElectionTest.ChannelBased {
+
+ protected JChannel a, b, c;
+ protected static final byte[] BUF = {};
+
+ {
+ clusterSize = 3;
+ recreatePerMethod = true;
+ }
+
+ @Override
+ protected void afterClusterCreation() {
+ a = channel(0);
+ b = channel(1);
+ c = channel(2);
+ }
+
+ /**
+ * All members have the same (initial) logs, so any member can be elected as leader
+ */
+ public void testSimpleElection(Class> ignore) {
+ assertLeader(10_000, null, a, b, c);
+ }
+
+
+ /**
+ * B and C have longer logs than A: one of {B,C} must become coordinator, but *not* A
+ */
+ public void testElectionWithLongLog(Class> ignore) {
+ // Let node A be elected the first leader.
+ assertLeader(10_000, a.getAddress(), a, b, c);
+
+ // Add the entries, creating longer logs.
+ setLog(b, 1, 1, 2);
+ setLog(c, 1, 1, 2);
+
+ // Assert that B and C have a longer log.
+ long aSize = logLastAppended(a);
+ assert aSize < logLastAppended(b) : "A log longer than B";
+ assert aSize < logLastAppended(c) : "A log longer than C";
+
+ JChannel coord = findCoord(a, b, c);
+ assertThat(coord).isNotNull();
+
+ System.out.printf("\n\n-- starting the voting process on %s:\n", coord.getAddress());
+ BaseElection el = election(coord);
+
+ System.out.printf("-- current status: %n%s%n", dumpLeaderAndTerms());
+
+ // Election is not running.
+ assertThat(el.isVotingThreadRunning()).isFalse();
+
+ // We start the election process. Eventually, the thread stops after collecting the necessary votes.
+ el.startVotingThread();
+ waitUntilVotingThreadStops(5_000, 0, 1, 2);
+
+ Address leader = assertLeader(10_000, null, b, c);
+ assert leader.equals(b.getAddress()) || leader.equals(c.getAddress()) : dumpLeaderAndTerms();
+ assert !leader.equals(a.getAddress());
+ }
+
+ protected static JChannel findCoord(JChannel... channels) {
+ for (JChannel ch : channels)
+ if (ch.getView().getCoord().equals(ch.getAddress()))
+ return ch;
+ return null;
+ }
+
+ protected void setLog(JChannel ch, int... terms) {
+ RAFT raft = raft(ch);
+ Log log = raft.log();
+ long index = log.lastAppended();
+ LogEntries le = new LogEntries();
+ for (int term : terms)
+ le.add(new LogEntry(term, BUF));
+ log.append(index + 1, le);
+ }
+
+ private long logLastAppended(JChannel ch) {
+ RAFT raft = raft(ch);
+ Log log = raft.log();
+ return log.lastAppended();
+ }
+
+ protected static List leaders(JChannel... channels) {
+ List leaders = new ArrayList<>(channels.length);
+ for (JChannel ch : channels) {
+ if (RaftTestUtils.isRaftLeader(ch))
+ leaders.add(ch.getAddress());
+ }
+ return leaders;
+ }
+
+ /**
+ * If expected is null, then any member can be a leader
+ */
+ protected Address assertLeader(int timeout, Address expected, JChannel... channels) {
+ RAFT[] rafts = Arrays.stream(channels)
+ .map(this::raft)
+ .toArray(RAFT[]::new);
+ BaseRaftElectionTest.waitUntilLeaderElected(rafts, timeout);
+ List leaders = leaders(channels);
+ assert leaders.size() == 1 : "leaders=" + leaders;
+ Address leader = leaders.get(0);
+ assert expected == null || leader.equals(expected) : String.format("elected %s instead of %s", leader, expected);
+ System.out.println("leader = " + leader);
+ return leader;
+ }
+
+
+}
diff --git a/tests/junit-functional/org/jgroups/tests/MergeTest.java b/tests/junit-functional/org/jgroups/tests/MergeTest.java
index 708af6df..97155f2a 100644
--- a/tests/junit-functional/org/jgroups/tests/MergeTest.java
+++ b/tests/junit-functional/org/jgroups/tests/MergeTest.java
@@ -7,59 +7,48 @@
import org.jgroups.protocols.pbcast.GMS;
import org.jgroups.protocols.pbcast.NAKACK2;
import org.jgroups.protocols.raft.RAFT;
-import org.jgroups.protocols.raft.REDIRECT;
-import org.jgroups.protocols.raft.election.BaseElection;
-import org.jgroups.raft.util.Utils;
+import org.jgroups.raft.testfwk.RaftTestUtils;
import org.jgroups.stack.ProtocolStack;
-import org.jgroups.tests.election.BaseElectionTest;
-import org.jgroups.util.Util;
-import org.testng.annotations.AfterMethod;
-import org.testng.annotations.BeforeMethod;
-import org.testng.annotations.Test;
+import org.jgroups.tests.harness.BaseRaftElectionTest;
-import java.util.ArrayList;
import java.util.Arrays;
import java.util.HashMap;
import java.util.HashSet;
import java.util.List;
import java.util.Map;
import java.util.Set;
-import java.util.concurrent.TimeoutException;
+import java.util.concurrent.TimeUnit;
+import java.util.function.BooleanSupplier;
import java.util.stream.Collectors;
import java.util.stream.Stream;
+import org.testng.annotations.Test;
+
+import static org.assertj.core.api.Assertions.assertThat;
+
/**
* Tests leader election on a merge (https://github.com/belaban/jgroups-raft/issues/125). Verifies that there can be
* at most 1 leader during concurrent election rounds run by different coordinators.
* @author Bela Ban
* @since 1.0.10
*/
-@Test(groups=Global.FUNCTIONAL,singleThreaded=true, dataProvider = BaseElectionTest.ALL_ELECTION_CLASSES_PROVIDER)
-public class MergeTest extends BaseElectionTest {
+@Test(groups=Global.FUNCTIONAL,singleThreaded=true, dataProvider = BaseRaftElectionTest.ALL_ELECTION_CLASSES_PROVIDER)
+public class MergeTest extends BaseRaftElectionTest.ChannelBased {
protected JChannel a,b,c,d,e;
- protected static final String CLUSTER=MergeTest.class.getSimpleName();
- protected final List members=Arrays.asList("A", "B", "C", "D", "E");
- protected RAFT[] rafts;
-
- @BeforeMethod
- protected void init() throws Exception {
- a=create("A"); a.connect(CLUSTER);
- b=create("B"); b.connect(CLUSTER);
- c=create("C"); c.connect(CLUSTER);
- d=create("D"); d.connect(CLUSTER);
- e=create("E"); e.connect(CLUSTER);
- Util.waitUntilAllChannelsHaveSameView(10000, 500, a,b,c,d,e);
- rafts=new RAFT[members.size()];
- int index=0;
- for(JChannel ch: Arrays.asList(a,b,c,d,e)) {
- RAFT r=ch.getProtocolStack().findProtocol(RAFT.class);
- rafts[index++]=r;
- }
+
+ {
+ // We create nodes `A` to `E`, strip down and setup per method.
+ clusterSize = 5;
+ recreatePerMethod = true;
}
- @AfterMethod
- protected void destroy() {
- close(e, d, c, b, a);
+ @Override
+ protected void afterClusterCreation() {
+ a = channel(0);
+ b = channel(1);
+ c = channel(2);
+ d = channel(3);
+ e = channel(4);
}
/**
@@ -70,7 +59,7 @@ protected void destroy() {
* for A _or_ E, but not both, in a given term.
*/
//@Test(invocationCount=10)
- public void testMerge(Class> ignore) throws TimeoutException {
+ public void testMerge(Class> ignore) {
long id=a.getView().getViewId().getId() +1;
View v1=createView(id, a, b), v2=createView(id, c, d), v3=createView(id, e);
@@ -82,8 +71,8 @@ public void testMerge(Class> ignore) throws TimeoutException {
assertView(v2, c, d);
assertView(v3, e);
- List leaders=leaders(a, b, c, d, e);
- assert leaders.isEmpty();
+ List leaders=leaders().stream().map(RAFT::getAddress).collect(Collectors.toList());
+ assert leaders.isEmpty() : dumpLeaderAndTerms();
System.out.printf("-- channels before:\n%s\n", print(a,b,c,d,e));
@@ -95,17 +84,24 @@ public void testMerge(Class> ignore) throws TimeoutException {
new Thread(injecter1).start();
new Thread(injecter2).start();
-
- Util.waitUntilTrue(3000, 200, () -> Stream.of(rafts).allMatch(r -> r.leader() != null));
- Stream.of(a,e).forEach(ch -> ((BaseElection)ch.getProtocolStack().findProtocol(electionClass)).stopVotingThread());
+ System.out.println("-- waiting election after merge");
+ // After the merge, either 'A' or 'E' will never have a leader.
+ // We stop waiting after one node sees itself as leader.
+ BooleanSupplier bs = () -> Arrays.stream(channels())
+ .map(this::raft)
+ .anyMatch(RAFT::isLeader);
+ assertThat(RaftTestUtils.eventually(bs, 5_000, TimeUnit.MILLISECONDS))
+ .as("Waiting leader after merge")
+ .isTrue();
+ stopVotingThread();
assertNoMoreThanOneLeaderInSameTerm(a,b,c,d,e);
System.out.printf("\n-- channels after:\n%s\n", print(a,b,c,d,e));
}
- protected static void assertNoMoreThanOneLeaderInSameTerm(JChannel... channels) {
+ protected void assertNoMoreThanOneLeaderInSameTerm(JChannel... channels) {
Map> m=new HashMap<>();
for(JChannel ch :channels) {
- RAFT r=ch.getProtocolStack().findProtocol(RAFT.class);
+ RAFT r=raft(ch);
long current_term=r.currentTerm();
Address leader=r.leader();
@@ -120,7 +116,7 @@ protected static void assertNoMoreThanOneLeaderInSameTerm(JChannel... channels)
}
for(Map.Entry> e: m.entrySet()) {
Set v=e.getValue();
- assert v.size() <= 1 : String.format("term %d had more than 1 leader: %s", e.getKey(), v);
+ assert v.size() == 1 : String.format("term %d had more than 1 leader: %s", e.getKey(), v);
}
}
@@ -132,13 +128,6 @@ protected static void assertView(View v, JChannel... channels) {
}
}
- protected static JChannel findCoord(JChannel... channels) {
- for(JChannel ch: channels)
- if(ch.getView().getCoord().equals(ch.getAddress()))
- return ch;
- return null;
- }
-
protected static String print(JChannel... channels) {
return Stream.of(channels).map(ch -> {
ProtocolStack stack=ch.getProtocolStack();
@@ -147,19 +136,13 @@ protected static String print(JChannel... channels) {
}).collect(Collectors.joining("\n"));
}
- protected JChannel create(String name) throws Exception {
- BaseElection election=instantiate();
- RAFT raft=new RAFT().members(members).raftId(name)
- .logClass("org.jgroups.protocols.raft.InMemoryLog").logPrefix(name + "-" + CLUSTER);
- REDIRECT client=new REDIRECT();
- //noinspection resource
- JChannel ch=new JChannel(Util.getTestStack(election,raft,client)).name(name);
+ @Override
+ protected void beforeChannelConnection(JChannel ch) throws Exception {
NAKACK2 nak=ch.getProtocolStack().findProtocol(NAKACK2.class);
if(nak != null)
nak.logDiscardMessages(false);
GMS gms=ch.getProtocolStack().findProtocol(GMS.class);
gms.printLocalAddress(false).logViewWarnings(false);
- return ch;
}
protected static void injectView(View v, JChannel... channels) {
@@ -169,73 +152,11 @@ protected static void injectView(View v, JChannel... channels) {
}
}
- protected static void close(JChannel... channels) {
- for(JChannel ch: channels) {
- if(ch == null)
- continue;
- close(ch);
- }
- }
-
- protected static void close(JChannel ch) {
- if(ch == null)
- return;
- RAFT raft=ch.getProtocolStack().findProtocol(RAFT.class);
- try {
- Utils.deleteLog(raft);
- }
- catch(Exception ignored) {}
- Util.close(ch);
- }
-
protected static View createView(long id, JChannel... mbrs) {
List addrs=Stream.of(mbrs).map(JChannel::getAddress).collect(Collectors.toList());
return View.create(mbrs[0].getAddress(), id, addrs);
}
- protected static boolean isLeader(JChannel ch) {
- RAFT raft=ch.getProtocolStack().findProtocol(RAFT.class);
- return ch.getAddress().equals(raft.leader());
- }
-
- protected static boolean hasLeader(JChannel ch) {
- RAFT raft=ch.getProtocolStack().findProtocol(RAFT.class);
- return raft.leader() != null;
- }
-
- protected static List leaders(JChannel... channels) {
- List leaders=new ArrayList<>(channels.length);
- for(JChannel ch: channels) {
- if(isLeader(ch))
- leaders.add(ch.getAddress());
- }
- return leaders;
- }
-
- /** If expected is null, then any member can be a leader */
- protected static Address assertLeader(int times, long sleep, Address expected, JChannel... channels) {
- // wait until there is 1 leader
- for(int i=0; i < times; i++) {
- List leaders=leaders(channels);
- if(!leaders.isEmpty()) {
- int size=leaders.size();
- assert size <= 1;
- Address leader=leaders.get(0);
- System.out.println("leader: " + leader);
- assert expected == null || expected.equals(leader);
- break;
- }
-
- Util.sleep(sleep);
- }
- List leaders=leaders(channels);
- assert leaders.size() == 1 : "leaders=" + leaders;
- Address leader=leaders.get(0);
- System.out.println("leader = " + leader);
- return leader;
- }
-
-
protected static class ViewInjecter implements Runnable {
protected final View v;
protected final JChannel[] channels;
diff --git a/tests/junit-functional/org/jgroups/tests/PartialConnectivityTest.java b/tests/junit-functional/org/jgroups/tests/PartialConnectivityTest.java
index ac32daf1..81daa3a8 100644
--- a/tests/junit-functional/org/jgroups/tests/PartialConnectivityTest.java
+++ b/tests/junit-functional/org/jgroups/tests/PartialConnectivityTest.java
@@ -5,155 +5,94 @@
import org.jgroups.View;
import org.jgroups.protocols.raft.ELECTION;
import org.jgroups.protocols.raft.ELECTION2;
-import org.jgroups.protocols.raft.InMemoryLog;
import org.jgroups.protocols.raft.RAFT;
-import org.jgroups.protocols.raft.election.BaseElection;
import org.jgroups.raft.testfwk.PartitionedRaftCluster;
import org.jgroups.raft.testfwk.RaftNode;
-import org.jgroups.raft.util.Utils;
-import org.jgroups.stack.Protocol;
-import org.jgroups.util.ExtendedUUID;
+import org.jgroups.tests.harness.BaseRaftElectionTest;
import org.jgroups.util.Util;
-import org.testng.annotations.AfterMethod;
-import org.testng.annotations.BeforeMethod;
-import org.testng.annotations.Factory;
-import org.testng.annotations.Test;
-import java.util.ArrayList;
-import java.util.Arrays;
import java.util.List;
import java.util.Objects;
-import java.util.stream.Collectors;
+import java.util.concurrent.TimeUnit;
+import java.util.function.BooleanSupplier;
+import java.util.stream.IntStream;
import java.util.stream.Stream;
-@Test(groups= Global.FUNCTIONAL, singleThreaded=true)
-public class PartialConnectivityTest {
-
- private static final String CLUSTER = PartialConnectivityTest.class.getSimpleName();
- protected final PartitionedRaftCluster cluster = new PartitionedRaftCluster();
- protected final List members= Arrays.asList("A", "B", "C", "D", "E");
- protected RaftNode[] nodes;
- protected RaftNode a, b, c, d, e;
- protected RAFT[] rafts;
- protected Class extends BaseElection> electionClass;
-
- public PartialConnectivityTest() { }
+import org.testng.annotations.Test;
- public PartialConnectivityTest(Class extends BaseElection> electionClass) {
- this.electionClass = electionClass;
- }
+import static org.assertj.core.api.Assertions.assertThat;
+import static org.jgroups.raft.testfwk.RaftTestUtils.eventually;
+import static org.jgroups.tests.harness.BaseRaftElectionTest.ALL_ELECTION_CLASSES_PROVIDER;
+import static org.jgroups.tests.harness.BaseRaftElectionTest.waitUntilAllHaveLeaderElected;
- @BeforeMethod
- protected void init() throws Exception {
- nodes = new RaftNode[5];
- rafts = new RAFT[5];
-
- int i=0;
- a = createNode("A", i++);
- b = createNode("B", i++);
- c = createNode("C", i++);
- d = createNode("D", i++);
- e = createNode("E", i);
- }
+@Test(groups= Global.FUNCTIONAL, singleThreaded=true, dataProvider = ALL_ELECTION_CLASSES_PROVIDER)
+public class PartialConnectivityTest extends BaseRaftElectionTest.ClusterBased {
- @AfterMethod
- protected void destroy() throws Exception {
- for (int i=nodes.length - 1; i >= 0; i--) {
- nodes[i].stop();
- nodes[i].destroy();
- Utils.deleteLog(rafts[i]);
- }
- Util.close(a,b,c,d,e);
- cluster.clear();
- }
+ {
+ // Create nodes A, B, C, D, E.
+ clusterSize = 5;
- @Factory
- protected static Object[] electionClass() {
- return new Object[] {
- new PartialConnectivityTest(ELECTION.class),
- new PartialConnectivityTest(ELECTION2.class),
- };
+ // Since it uses a data provider, it needs to execute per method to inject the values.
+ recreatePerMethod = true;
}
- public void testQuorumLossAndRecovery() {
- long id=1;
-
- View initial=createView(id++, a,b,c,d,e);
+ public void testQuorumLossAndRecovery(Class> ignore) {
+ int id=1;
+ View initial=createView(id++, 0, 1, 2, 3, 4);
cluster.handleView(initial);
- assert Util.waitUntilTrue(5000, 200, () -> Stream.of(nodes).allMatch(n -> n.raft().leader() != null));
- List leaders = leaders(nodes);
- assert leaders.size() == 1 : "there should be only one leader, but found " + leaders;
+ waitUntilLeaderElected(5_000, 0, 1, 2, 3, 4);
+ waitUntilAllHaveLeaderElected(rafts(), 10_000);
+ List leaders = leaders();
+ assertThat(leaders).hasSize(1);
Address leader = leaders.get(0);
System.out.println("leader is " + leader);
// Nodes D and E do not update their view.
- cluster.handleView(createView(id++, a, c));
- cluster.handleView(createView(id++, b, c));
-
- assert Util.waitUntilTrue(3000, 200, () -> Stream.of(a, b, c).allMatch(n -> n.raft().leader() == null));
-
- assert leader.equals(d.raft().leader()) : "leader should be " + leader + ", but found " + d.raft().leader();
- assert leader.equals(e.raft().leader()) : "leader should be " + leader + ", but found " + e.raft().leader();
-
- for (RaftNode n : nodes) {
- assert !n.election().isVotingThreadRunning() : "election thread should not be running in " + n;
+ cluster.handleView(createView(id++, 0, 2));
+ cluster.handleView(createView(id++, 1, 2));
+
+ BooleanSupplier bs = () -> IntStream.of(0, 1, 2)
+ .mapToObj(this::raft)
+ .filter(Objects::nonNull)
+ .allMatch(r -> r.leader() == null);
+ assertThat(eventually(bs, 5, TimeUnit.SECONDS))
+ .as(this::dumpLeaderAndTerms)
+ .isTrue();
+ assertThat(raft(3).leader()).as("leader should be " + leader + ", but found " + raft(3).leader()).isEqualTo(leader);
+ assertThat(raft(4).leader()).as("leader should be " + leader + ", but found " + raft(4).leader()).isEqualTo(leader);
+
+ for (RaftNode n : nodes()) {
+ assertThat(n.election().isVotingThreadRunning())
+ .as("election thread should not be running in " + n)
+ .isFalse();
}
- View after = createView(id++, e, d, a, b, c);
+ // Node `E` is the new view coordinator.
+ View after = createView(id++, 4, 3, 0, 1, 2);
System.out.println("after restored network: " + after);
cluster.handleView(after);
- boolean elected = Util.waitUntilTrue(3000, 200, () -> Stream.of(nodes).allMatch(n -> n.raft().leader() != null));
- if (electionClass.equals(ELECTION2.class)) {
- assert elected : "leader was never elected again";
- leaders = leaders(nodes);
- assert leaders.size() == 1 : "there should be only one leader, but found " + leaders;
+ boolean elected = Util.waitUntilTrue(3000, 200, () -> Stream.of(nodes()).allMatch(n -> n.raft().leader() != null));
+ if (election(0).getClass().equals(ELECTION2.class)) {
+ assertThat(elected).as("leader was never elected again").isTrue();
+ leaders = leaders();
+ assertThat(leaders).hasSize(1);
System.out.println("Leader after restored network: " + leaders.get(0));
} else {
- assert !elected : "leader was elected again";
- assert electionClass.equals(ELECTION.class);
+ assertThat(elected).as("Leader was elected again").isFalse();
+ assertThat(election(0)).isInstanceOf(ELECTION.class);
}
}
- protected static View createView(long id, RaftNode... mbrs) {
- List l=Stream.of(mbrs).filter(Objects::nonNull).map(RaftNode::getAddress).collect(Collectors.toList());
- return View.create(l.get(0), id, l);
+ @Override
+ protected PartitionedRaftCluster createNewMockCluster() {
+ return new PartitionedRaftCluster();
}
- protected static Address createAddress(String name) {
- ExtendedUUID.setPrintFunction(RAFT.print_function);
- return ExtendedUUID.randomUUID(name).put(RAFT.raft_id_key, Util.stringToBytes(name));
- }
-
- protected RaftNode createNode(String name, int i) throws Exception {
- Address a=createAddress(name);
- RAFT r=rafts[i]=new RAFT().members(members).raftId(name)
- .logClass(InMemoryLog.class.getCanonicalName())
- .logPrefix(name + "-" + CLUSTER)
- .stateMachine(new DummyStateMachine())
- .synchronous(true)
- .setAddress(a);
- BaseElection e=election(r, a);
- RaftNode node=nodes[i]=new RaftNode(cluster, new Protocol[]{e, r});
- node.init();
- node.start();
- cluster.add(a, node);
- return node;
- }
-
- private BaseElection election(RAFT r, Address a) throws Exception {
- return electionClass.getConstructor().newInstance().raft(r).setAddress(a);
- }
-
- public static List leaders(RaftNode... nodes) {
- List leaders = new ArrayList<>();
- for (RaftNode node : nodes) {
- if (node.raft().isLeader()) {
- leaders.add(node.getAddress());
- }
- }
- return leaders;
+ @Override
+ protected void amendRAFTConfiguration(RAFT raft) {
+ raft.synchronous(true);
}
}
diff --git a/tests/junit-functional/org/jgroups/tests/RaftTest.java b/tests/junit-functional/org/jgroups/tests/RaftTest.java
index 88d76845..b8860d41 100644
--- a/tests/junit-functional/org/jgroups/tests/RaftTest.java
+++ b/tests/junit-functional/org/jgroups/tests/RaftTest.java
@@ -1,6 +1,10 @@
package org.jgroups.tests;
-import org.jgroups.*;
+import org.jgroups.Address;
+import org.jgroups.Global;
+import org.jgroups.JChannel;
+import org.jgroups.Message;
+import org.jgroups.ObjectMessage;
import org.jgroups.protocols.FRAG2;
import org.jgroups.protocols.SHARED_LOOPBACK;
import org.jgroups.protocols.SHARED_LOOPBACK_PING;
@@ -8,23 +12,28 @@
import org.jgroups.protocols.pbcast.GMS;
import org.jgroups.protocols.pbcast.NAKACK2;
import org.jgroups.protocols.pbcast.STABLE;
-import org.jgroups.protocols.raft.*;
+import org.jgroups.protocols.raft.AppendEntriesRequest;
+import org.jgroups.protocols.raft.AppendResult;
+import org.jgroups.protocols.raft.LogEntries;
+import org.jgroups.protocols.raft.LogEntry;
+import org.jgroups.protocols.raft.NO_DUPES;
+import org.jgroups.protocols.raft.RAFT;
+import org.jgroups.protocols.raft.REDIRECT;
+import org.jgroups.protocols.raft.RaftImpl;
import org.jgroups.raft.Options;
import org.jgroups.raft.RaftHandle;
import org.jgroups.raft.util.CommitTable;
import org.jgroups.raft.util.CounterStateMachine;
import org.jgroups.stack.Protocol;
+import org.jgroups.tests.harness.BaseStateMachineTest;
import org.jgroups.util.Bits;
import org.jgroups.util.Util;
-import org.testng.annotations.AfterMethod;
-import org.testng.annotations.BeforeMethod;
-import org.testng.annotations.Test;
-import java.util.Arrays;
-import java.util.List;
import java.util.concurrent.CompletableFuture;
import java.util.concurrent.TimeUnit;
+import org.testng.annotations.Test;
+
/**
* Tests the various stages of the Raft protocoll, e.g. regular append, incorrect append, snapshots, leader change,
* leader election etc
@@ -32,54 +41,71 @@
* @since 1.0.5
*/
@Test(groups=Global.FUNCTIONAL,singleThreaded=true)
-public class RaftTest {
- protected JChannel a, b;
- protected RaftHandle rha, rhb;
- protected RAFT raft_a, raft_b;
- protected CounterStateMachine sma, smb;
- protected static final String GRP="RaftTest";
-
- @BeforeMethod
- protected void create() throws Exception {
- a=create("A", 600_000, 1_000_000);
- rha=new RaftHandle(a, sma=new CounterStateMachine());
- a.connect(GRP);
- raft_a=raft(a).setLeaderAndTerm(a.getAddress());
-
- b=create("B", 600_000, 1_000_000);
- rhb=new RaftHandle(b, smb=new CounterStateMachine());
- b.connect(GRP);
- raft_b=raft(b).setLeaderAndTerm(a.getAddress());
- Util.waitUntilAllChannelsHaveSameView(10000, 500, a,b);
- assert raft_a.isLeader();
- assert !raft_b.isLeader();
+public class RaftTest extends BaseStateMachineTest {
+
+ {
+ clusterSize = 2;
+ recreatePerMethod = true;
}
- @AfterMethod protected void destroy() throws Exception {
- for(JChannel ch: Arrays.asList(b, a)) {
- if(ch == null)
- continue;
- RAFT raft=ch.getProtocolStack().findProtocol(RAFT.class);
- raft.log().delete();
- Util.close(ch);
- }
+ @Override
+ protected CounterStateMachine createStateMachine(JChannel ch) {
+ return new CounterStateMachine();
}
+ @Override
+ protected Protocol[] baseProtocolStackForNode(String name) {
+ return new Protocol[] {
+ new SHARED_LOOPBACK(),
+ new SHARED_LOOPBACK_PING(),
+ new NAKACK2(),
+ new UNICAST3(),
+ new STABLE(),
+ new NO_DUPES(),
+ new GMS().setJoinTimeout(1000),
+ new FRAG2(),
+ createNewRaft(name),
+ new REDIRECT()
+ };
+ }
+
+ @Override
+ protected void amendRAFTConfiguration(RAFT raft) {
+ raft.resendInterval(600_000).maxLogSize(1_000_000);
+ }
+
+ @Override
+ protected void afterClusterCreation() throws Exception {
+ super.afterClusterCreation();
+
+ RAFT raft_a=raft(0).setLeaderAndTerm(address(0));
+ RAFT raft_b=raft(1).setLeaderAndTerm(address(0));
+
+ assert raft_a.isLeader();
+ assert !raft_b.isLeader();
+ }
public void testRegularAppend() throws Exception {
+ RaftHandle rha = handle(0);
int prev_value=add(rha, 1);
expect(0, prev_value);
+
+ CounterStateMachine sma = stateMachine(0);
+ CounterStateMachine smb = stateMachine(1);
assert sma.counter() == 1;
assert smb.counter() == 0; // resend_interval is big, so the commit index on B won't get updated
+
+ RAFT raft_a = raft(0);
+ RAFT raft_b = raft(1);
assertIndices(1, 1, 0, raft_a);
assertIndices(1, 0, 0, raft_b);
- assertCommitTableIndeces(b.getAddress(), raft_a, 0, 1, 2);
+ assertCommitTableIndeces(address(1), raft_a, 0, 1, 2);
prev_value=add(rha, 2);
assert prev_value == 1;
assert sma.counter() == 3;
assert smb.counter() == 1; // previous value; B is always lagging one commit behind
- assertCommitTableIndeces(b.getAddress(), raft_a, 1, 2, 3);
+ assertCommitTableIndeces(address(1), raft_a, 1, 2, 3);
prev_value=add(rha, 3);
assert prev_value == 3;
@@ -87,7 +113,7 @@ public void testRegularAppend() throws Exception {
assert smb.counter() == 3; // previous value; B is always lagging one commit behind
assertIndices(3, 3, 0, raft_a);
assertIndices(3, 2, 0, raft_b);
- assertCommitTableIndeces(b.getAddress(), raft_a, 2, 3, 4);
+ assertCommitTableIndeces(address(1), raft_a, 2, 3, 4);
prev_value=add(rha, -3);
assert prev_value == 6;
@@ -95,7 +121,7 @@ public void testRegularAppend() throws Exception {
assert smb.counter() == 6; // previous value; B is always lagging one commit behind
assertIndices(4, 4, 0, raft_a);
assertIndices(4, 3, 0, raft_b);
- assertCommitTableIndeces(b.getAddress(), raft_a, 3, 4, 5);
+ assertCommitTableIndeces(address(1), raft_a, 3, 4, 5);
for(int i=1,prev=3; i <= 1000; i++) {
prev_value=add(rha, 5);
@@ -107,10 +133,10 @@ public void testRegularAppend() throws Exception {
assertIndices(1004, 1004, 0, raft_a);
assertIndices(1004, 1003, 0, raft_b);
- assertCommitTableIndeces(b.getAddress(), raft_a, 1003, 1004, 1005);
+ assertCommitTableIndeces(address(1), raft_a, 1003, 1004, 1005);
long current_term=raft_a.currentTerm(), expected_term;
- raft_a.setLeaderAndTerm(a.getAddress(), expected_term=current_term + 10);
+ raft_a.setLeaderAndTerm(address(0), expected_term=current_term + 10);
for(int i=1; i <= 7; i++)
add(rha, 1);
@@ -120,19 +146,23 @@ public void testRegularAppend() throws Exception {
assertIndices(1011, 1011, expected_term, raft_a);
assertIndices(1011, 1010, expected_term, raft_b);
- assertCommitTableIndeces(b.getAddress(), raft_a, 1010, 1011, 1012);
+ assertCommitTableIndeces(address(1), raft_a, 1010, 1011, 1012);
}
public void testAppendSameElementTwice() throws Exception {
- raft_a.setLeaderAndTerm(a.getAddress(), 20);
+ RAFT raft_a = raft(0);
+ raft_a.setLeaderAndTerm(address(0), 20);
+ RAFT raft_b = raft(1);
+
for(int i=1; i <= 5; i++)
- add(rha, 1);
+ add(handle(0), 1);
+
assertIndices(5, 5, 20, raft_a);
assertIndices(5, 4, 20, raft_b);
- assertCommitTableIndeces(b.getAddress(), raft_a, 4, 5, 6);
- expect(5, sma.counter());
- expect(4, smb.counter());
+ assertCommitTableIndeces(address(1), raft_a, 4, 5, 6);
+ expect(5, stateMachine(0).counter());
+ expect(4, stateMachine(1).counter());
// this will append the same entry, but because index 4 is < last_appended (5), it will not get appended
// Note though that the commit-index will be updated
@@ -140,41 +170,51 @@ public void testAppendSameElementTwice() throws Exception {
Util.waitUntil(5000, 50, () -> raft_b.commitIndex() == 5);
assertIndices(5, 5, 20, raft_a);
assertIndices(5, 5, 20, raft_b);
- assertCommitTableIndeces(b.getAddress(), raft_a, 5, 5, 6);
- assert sma.counter() == 5;
- assert smb.counter() == 5;
+ assertCommitTableIndeces(address(1), raft_a, 5, 5, 6);
+ assert stateMachine(0).counter() == 5;
+ assert stateMachine(1).counter() == 5;
}
public void testAppendBeyondLast() throws Exception {
- raft_a.setLeaderAndTerm(a.getAddress(), 22);
+ RAFT raft_a = raft(0);
+ raft_a.setLeaderAndTerm(address(0), 22);
+
for(int i=1; i <= 5; i++)
- add(rha, 1);
- assert sma.counter() == 5;
- assert smb.counter() == 4; // resend_interval is big, so the commit index on B won't get updated
+ add(handle(0), 1);
+ assert stateMachine(0).counter() == 5;
+ assert stateMachine(1).counter() == 4; // resend_interval is big, so the commit index on B won't get updated
assertIndices(5, 5, 22, raft_a);
- assertIndices(5, 4, 22, raft_b);
- assertCommitTableIndeces(b.getAddress(), raft_a, 4, 5, 6);
+ assertIndices(5, 4, 22, raft(1));
+ assertCommitTableIndeces(address(1), raft_a, 4, 5, 6);
// now append beyond the end
sendAppendEntriesRequest(raft_a, 9, 22, 22, 0);
Util.sleep(1000);
// nothing changed, as request was rejected
- assertCommitTableIndeces(b.getAddress(), raft_a, 4, 5, 6);
+ assertCommitTableIndeces(address(1), raft_a, 4, 5, 6);
}
/** Tests appding with correct prev_index, but incorrect term */
public void testAppendWrongTerm() throws Exception {
- raft_a.setLeaderAndTerm(a.getAddress(), 22);
+ RAFT raft_a = raft(0);
+ raft_a.setLeaderAndTerm(address(0), 22);
+ RAFT raft_b = raft(1);
+
+ RaftHandle rha = handle(0);
for(int i=1; i <= 15; i++) {
if(i % 5 == 0)
- raft_a.setLeaderAndTerm(a.getAddress(), raft_a.currentTerm()+1);
+ raft_a.setLeaderAndTerm(address(0), raft_a.currentTerm()+1);
add(rha, 1);
}
+
+ CounterStateMachine sma = stateMachine(0);
expect(15, sma.counter());
+
+ CounterStateMachine smb = stateMachine(1);
assert smb.counter() == 14; // resend_interval is big, so the commit index on B won't get updated
assertIndices(15, 15, 25, raft_a);
assertIndices(15, 14, 25, raft_b);
- assertCommitTableIndeces(b.getAddress(), raft_a, 14, 15, 16);
+ assertCommitTableIndeces(address(1), raft_a, 14, 15, 16);
// now append entries 16,17 and 18 (all with term=25), but *don't* advance the commit index
for(int i=16; i <= 18; i++)
@@ -184,29 +224,29 @@ public void testAppendWrongTerm() throws Exception {
assert smb.counter() == 14; // resend_interval is big, so the commit index on B won't get updated
assertIndices(15, 15, 25, raft_a);
assertIndices(18, 14, 25, raft_b);
- assertCommitTableIndeces(b.getAddress(), raft_a, 14, 18, 19);
+ assertCommitTableIndeces(address(1), raft_a, 14, 18, 19);
// send a correct index, but incorrect prev_term:
long incorrect_prev_term=24;
long commit_index=raft_a.commitIndex(), prev_index=18;
- raft_a.setLeaderAndTerm(a.getAddress(), 30);
+ raft_a.setLeaderAndTerm(address(0), 30);
sendAppendEntriesRequest(raft_a, prev_index, incorrect_prev_term, 30, commit_index);
Util.sleep(1000); // nothing changed
- assertCommitTableIndeces(b.getAddress(), raft_a, 14, 14, 15);
+ assertCommitTableIndeces(address(1), raft_a, 14, 14, 15);
// now apply the updates on the leader
raft_a.resendInterval(1000);
for(int i=16; i <= 18; i++)
add(rha, 1);
Util.waitUntil(5000, 100, () -> raft_b.lastAppended() == 18);
- assertCommitTableIndeces(b.getAddress(), raft_a, 17, 18, 19);
+ assertCommitTableIndeces(address(1), raft_a, 17, 18, 19);
raft_a.flushCommitTable();
Util.waitUntil(5000, 100, () -> raft_b.commitIndex() == 18);
- assertCommitTableIndeces(b.getAddress(), raft_a, 18, 18, 19);
+ assertCommitTableIndeces(address(1), raft_a, 18, 18, 19);
// compare the log entries from 1-18
for(int i=0; i <= raft_a.lastAppended(); i++) {
@@ -224,24 +264,31 @@ public void testAppendWrongTerm() throws Exception {
/** Tests appends where we change prev_term, so that we'll get an AppendResult with success=false */
public void testIncorrectAppend() throws Exception {
+ RaftHandle rha = handle(0);
int prev_value=add(rha, 1);
assert prev_value == 0;
+
+ CounterStateMachine sma = stateMachine(0);
+ CounterStateMachine smb = stateMachine(1);
assert sma.counter() == 1;
assert smb.counter() == 0; // resend_interval is big, so the commit index on B won't get updated
+
+ RAFT raft_a = raft(0);
+ RAFT raft_b = raft(1);
assertIndices(1, 1, 0, raft_a);
assertIndices(1, 0, 0, raft_b);
- assertCommitTableIndeces(b.getAddress(), raft_a, 0, 1, 2);
+ assertCommitTableIndeces(address(1), raft_a, 0, 1, 2);
- raft_a.setLeaderAndTerm(a.getAddress(), 2);
+ raft_a.setLeaderAndTerm(address(0), 2);
prev_value=add(rha, 1);
assert prev_value == 1;
prev_value=add(rha, 1);
assert prev_value == 2;
assert sma.counter() == 3;
assert smb.counter() == 2; // previous value; B is always lagging one commit behind
- assertCommitTableIndeces(b.getAddress(), raft_a, 2, 3, 4);
+ assertCommitTableIndeces(address(1), raft_a, 2, 3, 4);
- raft_a.setLeaderAndTerm(a.getAddress(), 4);
+ raft_a.setLeaderAndTerm(address(0), 4);
for(int i=1; i <= 3; i++)
add(rha, 1);
@@ -255,17 +302,17 @@ public void testIncorrectAppend() throws Exception {
// 7
LogEntries entries=new LogEntries().add(new LogEntry(5, val));
- AppendResult result=impl.handleAppendEntriesRequest(entries, a.getAddress(), index - 1, 4, 5, 1);
+ AppendResult result=impl.handleAppendEntriesRequest(entries, address(0), index - 1, 4, 5, 1);
assert result.success();
raft_b.currentTerm(5);
index++;
// 8
- result=impl.handleAppendEntriesRequest(entries, a.getAddress(), index-1, 5, 5, 1);
+ result=impl.handleAppendEntriesRequest(entries, address(0), index-1, 5, 5, 1);
assert result.success();
assertIndices(8, 5, 5, raft_b);
- raft_a.setLeaderAndTerm(a.getAddress(), 7);
+ raft_a.setLeaderAndTerm(address(0), 7);
for(int i=1; i <= 2; i++)
add(rha, -1);
@@ -274,23 +321,29 @@ public void testIncorrectAppend() throws Exception {
assert smb.counter() == 5; // resend_interval is big, so the commit index on B won't get updated
assertIndices(8, 8, 7, raft_a);
assertIndices(8, 7, 7, raft_b);
- assertCommitTableIndeces(b.getAddress(), raft_a, 7, 8, 9);
+ assertCommitTableIndeces(address(1), raft_a, 7, 8, 9);
}
/** Tests appending with correct prev_index, but incorrect term */
public void testAppendWrongTermOnlyOneTerm() throws Exception {
-
+ RAFT raft_a = raft(0);
raft_a.resendInterval(1000);
+ raft_a.setLeaderAndTerm(address(0), 22);
+
+ RAFT raft_b = raft(1);
- raft_a.setLeaderAndTerm(a.getAddress(), 22);
+ RaftHandle rha = handle(0);
for(int i=1; i <= 5; i++)
add(rha, 1);
+
+ CounterStateMachine sma = stateMachine(0);
+ CounterStateMachine smb = stateMachine(1);
expect(5, sma.counter());
expect(4, smb.counter()); // resend_interval is big, so the commit index on B won't get updated
assertIndices(5, 5, 22, raft_a);
assertIndices(5, 4, 22, raft_b);
- assertCommitTableIndeces(b.getAddress(), raft_a, 4, 5, 6);
+ assertCommitTableIndeces(address(1), raft_a, 4, 5, 6);
// now append beyond the end
byte[] val=new byte[Integer.BYTES];
@@ -308,13 +361,17 @@ public void testAppendWrongTermOnlyOneTerm() throws Exception {
expect(5, smb.counter()); // resend_interval is big, so the commit index on B won't get updated
assertIndices(5, 5, 22, raft_a);
assertIndices(5, 5, 22, raft_b);
- assertCommitTableIndeces(b.getAddress(), raft_a, 5, 5, 6);
+ assertCommitTableIndeces(address(1), raft_a, 5, 5, 6);
}
public void testSimpleAppendOnFollower() throws Exception {
+ RaftHandle rhb = handle(1);
CompletableFuture f=addAsync(rhb, 5);
assert f != null;
+
+ CounterStateMachine sma = stateMachine(0);
+ CounterStateMachine smb = stateMachine(1);
int prev_value=Bits.readInt(f.get(), 0);
assert prev_value == 0;
assert sma.counter() == 5;
@@ -397,28 +454,6 @@ protected static CompletableFuture addAsync(RaftHandle handle, int delta
return handle.setAsync(buf, 0, buf.length, opts);
}
- protected static JChannel create(String name, long resend_interval, int max_log_size) throws Exception {
- Protocol[] protocols={
- new SHARED_LOOPBACK(),
- new SHARED_LOOPBACK_PING(),
- new NAKACK2(),
- new UNICAST3(),
- new STABLE(),
- new NO_DUPES(),
- new GMS().setJoinTimeout(1000),
- new FRAG2(),
- // new ELECTION().electionMinInterval(100).electionMaxInterval(300).heartbeatInterval(30),
- new RAFT().members(List.of("A", "B", "C")).raftId(name)
- .logPrefix("rafttest-" + name).resendInterval(resend_interval).maxLogSize(max_log_size),
- new REDIRECT()
- };
- return new JChannel(protocols).name(name);
- }
-
- protected static RAFT raft(JChannel ch) {
- return ch.getProtocolStack().findProtocol(RAFT.class);
- }
-
protected static void expect(int expected_value, int actual_value) {
assert actual_value == expected_value : String.format("expected=%d actual=%d", expected_value, actual_value);
}
diff --git a/tests/junit-functional/org/jgroups/tests/SyncElectionTests.java b/tests/junit-functional/org/jgroups/tests/SyncElectionTests.java
index 456d9bb2..5df2e550 100644
--- a/tests/junit-functional/org/jgroups/tests/SyncElectionTests.java
+++ b/tests/junit-functional/org/jgroups/tests/SyncElectionTests.java
@@ -1,31 +1,40 @@
package org.jgroups.tests;
-import org.jgroups.*;
-import org.jgroups.protocols.raft.*;
+import org.jgroups.Address;
+import org.jgroups.EmptyMessage;
+import org.jgroups.Global;
+import org.jgroups.MergeView;
+import org.jgroups.Message;
+import org.jgroups.ObjectMessage;
+import org.jgroups.View;
+import org.jgroups.protocols.raft.AppendEntriesRequest;
+import org.jgroups.protocols.raft.LogEntries;
+import org.jgroups.protocols.raft.LogEntry;
+import org.jgroups.protocols.raft.RAFT;
import org.jgroups.protocols.raft.election.BaseElection;
import org.jgroups.protocols.raft.election.VoteRequest;
import org.jgroups.protocols.raft.election.VoteResponse;
import org.jgroups.raft.testfwk.RaftCluster;
import org.jgroups.raft.testfwk.RaftNode;
-import org.jgroups.raft.util.Utils;
-import org.jgroups.stack.Protocol;
-import org.jgroups.tests.election.BaseElectionTest;
-import org.jgroups.util.ExtendedUUID;
+import org.jgroups.raft.testfwk.RaftTestUtils;
+import org.jgroups.tests.harness.BaseRaftElectionTest;
import org.jgroups.util.ResponseCollector;
-import org.jgroups.util.Util;
-import org.testng.annotations.AfterMethod;
-import org.testng.annotations.BeforeMethod;
-import org.testng.annotations.Test;
import java.util.Arrays;
-import java.util.List;
import java.util.Objects;
-import java.util.concurrent.TimeoutException;
+import java.util.concurrent.TimeUnit;
+import java.util.function.BooleanSupplier;
import java.util.function.Supplier;
import java.util.stream.Collectors;
+import java.util.stream.IntStream;
import java.util.stream.Stream;
-import static org.jgroups.tests.election.BaseElectionTest.ALL_ELECTION_CLASSES_PROVIDER;
+import org.testng.annotations.AfterMethod;
+import org.testng.annotations.BeforeMethod;
+import org.testng.annotations.Test;
+
+import static org.assertj.core.api.Assertions.assertThat;
+import static org.jgroups.tests.harness.BaseRaftElectionTest.ALL_ELECTION_CLASSES_PROVIDER;
/**
* Uses the synchronous test framework to test {@link org.jgroups.protocols.raft.ELECTION}
@@ -33,58 +42,56 @@
* @since 1.0.5
*/
@Test(groups=Global.FUNCTIONAL,singleThreaded=true, dataProvider = ALL_ELECTION_CLASSES_PROVIDER)
-public class SyncElectionTests extends BaseElectionTest {
- protected final Address a,b,c;
- protected final Address[] addrs={a=createAddress("A"), b=createAddress("B"), c=createAddress("C")};
- protected final List mbrs=List.of("A", "B", "C");
- protected final RaftCluster cluster=new RaftCluster();
- protected RAFT[] rafts=new RAFT[3];
- protected BaseElection[] elections=new BaseElection[3];
- protected RaftNode[] nodes=new RaftNode[3];
+public class SyncElectionTests extends BaseRaftElectionTest.ClusterBased {
+
protected int view_id=1;
- @BeforeMethod protected void init() {view_id=1;}
+ {
+ createManually = true;
+ }
+
+ @BeforeMethod
+ protected void init() {
+ view_id=1;
+ }
@AfterMethod
protected void destroy() throws Exception {
- for(int i=nodes.length-1; i >= 0; i--) {
- if(nodes[i] != null) {
- nodes[i].stop();
- nodes[i].destroy();
- nodes[i]=null;
- }
- if(rafts[i] != null) {
- Utils.deleteLog(rafts[i]);
- rafts[i]=null;
- }
- if(elections[i] != null) {
- elections[i].stopVotingThread();
- elections[i]=null;
- }
- }
- cluster.clear();
+ destroyCluster();
}
/** Not really an election, as we only have a single member */
public void testSingletonElection(Class> ignore) throws Exception {
- createNode(0, "A");
- assert !rafts[0].isLeader();
- View view=createView();
+ withClusterSize(3);
+ createCluster(1);
+
+ assertThat(raft(0).isLeader()).isFalse();
+ View view=createView(view_id++, 0);
cluster.handleView(view);
- assert !rafts[0].isLeader();
- Util.waitUntilTrue(2000, 200, () -> rafts[0].isLeader());
- assert !rafts[0].isLeader();
- // assert !elections[0].isVotingThreadRunning();
+
+ assertThat(raft(0).isLeader()).isFalse();
+ assertNotElected(2_000, 0);
+ assertThat(raft(0).isLeader()).isFalse();
+
waitUntilVotingThreadHasStopped();
}
public void testElectionWithTwoMembers(Class> ignore) throws Exception {
- createNode(0, "A");
- createNode(1, "B");
- View view=createView();
+ withClusterSize(3);
+ createCluster(2);
+
+ View view=createView(view_id++, 0, 1);
cluster.handleView(view);
- Util.waitUntilTrue(5000, 100, () -> Arrays.stream(rafts).filter(Objects::nonNull).anyMatch(RAFT::isLeader));
+
+ BooleanSupplier bs = () -> Arrays.stream(nodes())
+ .filter(Objects::nonNull)
+ .map(RaftNode::raft)
+ .anyMatch(RAFT::isLeader);
+ assertThat(RaftTestUtils.eventually(bs, 5_000, TimeUnit.MILLISECONDS))
+ .as("Should elect a leader")
+ .isTrue();
+
assertOneLeader();
System.out.printf("%s\n", print());
waitUntilVotingThreadHasStopped();
@@ -92,16 +99,21 @@ public void testElectionWithTwoMembers(Class> ignore) throws Exception {
/** Tests adding a third member. After {A,B} has formed, this should not change anything */
public void testThirdMember(Class> ignore) throws Exception {
- createNode(0, "A");
- createNode(1, "B");
- View view=createView();
+ withClusterSize(3);
+ createCluster(2);
+
+ View view=createView(view_id++, 0, 1);
cluster.handleView(view);
- Util.waitUntilTrue(5000, 100, () -> Arrays.stream(rafts).filter(Objects::nonNull).anyMatch(RAFT::isLeader));
+
+ waitUntilLeaderElected(5_000, 0, 1);
+
System.out.printf("%s\n", print());
System.out.println("Adding 3rd member:");
- createNode(2, "C");
- view=createView();
+
+ createCluster();
+
+ view=createView(view_id++, 0, 1, 2);
cluster.handleView(view);
assertOneLeader();
System.out.printf("%s\n", print());
@@ -111,17 +123,21 @@ public void testThirdMember(Class> ignore) throws Exception {
/** Tests A -> ABC */
public void testGoingFromOneToThree(Class> ignore) throws Exception {
- createNode(0, "A");
- View view=createView();
+ withClusterSize(3);
+ createCluster(1);
+
+ View view=createView(view_id++, 0);
cluster.handleView(view);
- Util.waitUntilTrue(2000, 200, () -> rafts[0].isLeader());
- assert !rafts[0].isLeader();
- createNode(1, "B");
- createNode(2, "C");
- view=createView();
+ assertNotElected(2_000, 0);
+ assertThat(raft(0).isLeader()).isFalse();
+
+ createCluster();
+ view=createView(view_id++, 0, 1, 2);
cluster.handleView(view);
- Util.waitUntilTrue(5000, 100, () -> Arrays.stream(rafts).filter(Objects::nonNull).anyMatch(RAFT::isLeader));
+
+ waitUntilLeaderElected(5_000, 0, 1, 2);
+
System.out.printf("%s\n", print());
assertOneLeader();
waitUntilVotingThreadHasStopped();
@@ -130,13 +146,14 @@ public void testGoingFromOneToThree(Class> ignore) throws Exception {
/** {} -> {ABC} */
public void testGoingFromZeroToThree(Class> ignore) throws Exception {
- createNode(0, "A");
- createNode(1, "B");
- createNode(2, "C");
- View view=createView();
- // cluster.async(true);
+ withClusterSize(3);
+ createCluster();
+
+ View view=createView(view_id++, 0, 1, 2);
+
cluster.handleView(view);
- Util.waitUntilTrue(5000, 100, () -> Arrays.stream(rafts).filter(Objects::nonNull).anyMatch(RAFT::isLeader));
+ waitUntilLeaderElected(5_000, 0, 1, 2);
+
System.out.printf("%s\n", print());
assertOneLeader();
waitUntilVotingThreadHasStopped();
@@ -146,11 +163,15 @@ public void testGoingFromZeroToThree(Class> ignore) throws Exception {
/** Follower is leaving */
public void testGoingFromThreeToTwo(Class> clazz) throws Exception {
testGoingFromZeroToThree(clazz);
+
int follower=findFirst(false);
- kill(follower);
- View view=createView();
+ close(follower);
+
+ View view=createView(view_id++, 0, 1, 2);
cluster.handleView(view);
- Util.waitUntilTrue(5000, 100, () -> Arrays.stream(rafts).filter(Objects::nonNull).anyMatch(RAFT::isLeader));
+
+ waitUntilLeaderElected(5_000, 0, 1, 2);
+
System.out.printf("%s\n", print());
assertOneLeader();
waitUntilVotingThreadHasStopped();
@@ -159,14 +180,15 @@ public void testGoingFromThreeToTwo(Class> clazz) throws Exception {
public void testGoingFromThreeToOne(Class> clazz) throws Exception {
testGoingFromZeroToThree(clazz);
- int follower=findFirst(false);
- kill(follower);
- follower=findFirst(false);
- kill(follower);
- View view=createView();
+ close(findFirst(false));
+ close(findFirst(false));
+
+ View view=createView(view_id++, 0, 1, 2);
cluster.handleView(view);
- Util.waitUntilTrue(2000, 100, () -> Arrays.stream(rafts).filter(Objects::nonNull).noneMatch(RAFT::isLeader));
+
+ waitUntilStepDown(2_000, 0, 1, 2);
+
System.out.printf("%s\n", print());
waitUntilVotingThreadHasStopped();
assertSameTerm(this::print);
@@ -176,10 +198,15 @@ public void testGoingFromThreeToOne(Class> clazz) throws Exception {
public void testLeaderLeaving(Class> clazz) throws Exception {
testGoingFromZeroToThree(clazz);
int leader=findFirst(true);
- kill(leader);
- View view=createView();
+ close(leader);
+ View view=createView(view_id++, 0, 1, 2);
+
cluster.handleView(view);
- Util.waitUntilTrue(5000, 100, () -> Arrays.stream(rafts).filter(Objects::nonNull).anyMatch(RAFT::isLeader));
+
+ // All nodes but the previous leader.
+ int[] indexes = IntStream.range(0, clusterSize).filter(i -> i != leader).toArray();
+ waitUntilLeaderElected(5_000, indexes);
+
System.out.printf("%s\n", print());
assertOneLeader();
waitUntilVotingThreadHasStopped();
@@ -188,27 +215,26 @@ public void testLeaderLeaving(Class> clazz) throws Exception {
/** Tests a vote where a non-coord has a longer log */
public void testVotingFollowerHasLongerLog(Class> ignore) throws Exception {
- createNode(0, "A");
- createNode(1, "B");
- createNode(2, "C");
+ withClusterSize(3);
+ createCluster();
byte[] data=new byte[4];
int[] terms={0,1,1,1,2,4,4,5,6,6};
for(int i=1; i < terms.length; i++) {
LogEntries entries=new LogEntries().add(new LogEntry(terms[i], data));
- AppendEntriesRequest hdr=new AppendEntriesRequest(a,terms[i],i - 1,terms[i - 1],terms[i],0);
- Message msg=new ObjectMessage(c, entries).putHeader(rafts[2].getId(), hdr);
+ AppendEntriesRequest hdr=new AppendEntriesRequest(address(0),terms[i],i - 1,terms[i - 1],terms[i],0);
+ Message msg=new ObjectMessage(address(2), entries).putHeader(raft(2).getId(), hdr);
cluster.send(msg);
}
- View view=createView();
- // cluster.async(true);
+ View view=createView(view_id++, 0, 1, 2);
cluster.handleView(view);
- Util.waitUntilTrue(5000, 100, () -> Arrays.stream(rafts).filter(Objects::nonNull).anyMatch(RAFT::isLeader));
+ waitUntilLeaderElected(10_000, 0, 1, 2);
System.out.printf("%s\n", print());
assertOneLeader();
- assert !rafts[0].isLeader();
- assert rafts[2].isLeader();
+
+ assertThat(raft(0).isLeader()).isFalse();
+ assertThat(raft(2).isLeader()).isTrue();
waitUntilVotingThreadHasStopped();
assertSameTerm(this::print);
int new_term=terms[terms.length-1]+1;
@@ -217,18 +243,21 @@ public void testVotingFollowerHasLongerLog(Class> ignore) throws Exception {
/** {A}, {B}, {C} -> {A,B,C} */
public void testMerge(Class> ignore) throws Exception {
- createNode(0, "A");
- createNode(1, "B");
- createNode(2, "C");
+ withClusterSize(3);
+ createCluster();
View v1,v2,v3;
- nodes[0].handleView(v1=View.create(a, 1, a));
- nodes[1].handleView(v2=View.create(b, 1, b));
- nodes[2].handleView(v3=View.create(c, 1, c));
- View mv=new MergeView(b, 2, Arrays.asList(b,a,c), Arrays.asList(v2,v1,v3));
- for(RaftNode n: nodes)
+ node(0).handleView(v1=View.create(address(0), 1, address(0)));
+ node(1).handleView(v2=View.create(address(1), 1, address(1)));
+ node(2).handleView(v3=View.create(address(2), 1, address(2)));
+
+ assertNotElected(2_000, 0, 1, 2);
+
+ View mv=new MergeView(address(1), 2, Arrays.asList(address(1), address(0), address(2)), Arrays.asList(v2,v1,v3));
+ for(RaftNode n: nodes())
n.handleView(mv);
- Util.waitUntilTrue(5000, 100, () -> Arrays.stream(rafts).filter(Objects::nonNull).anyMatch(RAFT::isLeader));
+
+ waitUntilLeaderElected(5_000, 0, 1, 2);
System.out.printf("%s\n", print());
assertOneLeader();
waitUntilVotingThreadHasStopped();
@@ -240,24 +269,28 @@ public void testMerge(Class> ignore) throws Exception {
* only A or B can receive a vote response from C, but not both.
*/
public void testMultipleVotes(Class> ignore) throws Exception {
- createNode(0, "A");
- createNode(1, "B");
- createNode(2, "C");
+ withClusterSize(3);
+ createCluster();
+
+ BaseElection[] elections = elections();
+ Address a = address(0), b = address(1), c = address(2);
ResponseCollector votes_a=elections[0].getVotes(), votes_b=elections[1].getVotes();
votes_a.reset(a,b,c);
votes_b.reset(a,b,c);
System.out.println("-- A and B: sending VoteRequests to C");
- long term_a=rafts[0].createNewTerm();
- long term_b=rafts[1].createNewTerm();
+ long term_a=raft(0).createNewTerm();
+ long term_b=raft(1).createNewTerm();
elections[0].down(new EmptyMessage(c).putHeader(elections[0].getId(), new VoteRequest(term_a)));
elections[1].down(new EmptyMessage(c).putHeader(elections[1].getId(), new VoteRequest(term_b)));
System.out.printf("A: vote responses in term %d: %s\n", term_a, votes_a);
System.out.printf("B: vote responses in term %d: %s\n", term_b, votes_b);
int total_rsps=votes_a.numberOfValidResponses() + votes_b.numberOfValidResponses();
- assert total_rsps <= 1 : "A and B both received a vote response from C; this is invalid (Raft $3.4)";
+ assertThat(total_rsps)
+ .as("A and B both received a vote response from C; this is invalid (Raft $3.4)")
+ .isLessThanOrEqualTo(1);
// VoteRequest(term=0): will be dropped by C as C's current_term is 1
term_b--;
@@ -265,7 +298,9 @@ public void testMultipleVotes(Class> ignore) throws Exception {
elections[1].down(new EmptyMessage(c).putHeader(elections[1].getId(), new VoteRequest(term_b)));
System.out.printf("B: vote responses in term %d: %s\n", term_b, votes_b);
total_rsps=votes_b.numberOfValidResponses();
- assert total_rsps == 0 : "B should have received no vote response from C";
+ assertThat(total_rsps)
+ .as("B should have received no vote response from C")
+ .isZero();
// VoteRequest(term=1): will be dropped by C as C already voted (for A) in term 1
term_b++;
@@ -273,7 +308,9 @@ public void testMultipleVotes(Class> ignore) throws Exception {
elections[1].down(new EmptyMessage(c).putHeader(elections[1].getId(), new VoteRequest(term_b)));
System.out.printf("B: vote responses in term %d: %s\n", term_b, votes_b);
total_rsps=votes_b.numberOfValidResponses();
- assert total_rsps == 0 : "B should have received no vote response from C";
+ assertThat(total_rsps)
+ .as("B should have received no vote response from C")
+ .isZero();
// VoteRequest(term=2): C will vote for B, as term=2 is new, and C hasn't yet voted for anyone in term 2
term_b++;
@@ -281,115 +318,116 @@ public void testMultipleVotes(Class> ignore) throws Exception {
elections[1].down(new EmptyMessage(c).putHeader(elections[1].getId(), new VoteRequest(term_b)));
System.out.printf("B: vote responses in term %d: %s\n", term_b, votes_b);
total_rsps=votes_b.numberOfValidResponses();
- assert total_rsps <= 1 : "B should have received a vote response from C";
+ assertThat(total_rsps)
+ .as("B should have received a vote response from C")
+ .isLessThanOrEqualTo(1);
}
public void testIncreasingTermForgetOldLeader(Class> ignore) throws Exception {
- createNode(0, "A");
- createNode(1, "B");
- View view=createView();
+ withClusterSize(2);
+ createCluster();
+ View view=createView(view_id++, 0, 1);
cluster.handleView(view);
- assert Util.waitUntilTrue(5000, 100, () -> Arrays.stream(rafts).filter(Objects::nonNull).anyMatch(RAFT::isLeader));
+
+ waitUntilVotingThreadStops(5_000, 0, 1);
assertOneLeader();
System.out.printf("%s\n", print());
waitUntilVotingThreadHasStopped();
int idx = findFirst(true);
- assert rafts[idx].isLeader();
- assert rafts[idx].role().equals("Leader");
- long term = rafts[idx].currentTerm();
- assert rafts[idx].currentTerm(term + 1) == 1;
- assert rafts[idx].currentTerm() == term + 1;
- assert !rafts[idx].isLeader();
- assert rafts[idx].role().equals("Follower");
+ assertThat(raft(idx).isLeader()).isTrue();
+ assertThat(raft(idx).role()).isEqualTo("Leader");
+
+ long term = raft(idx).currentTerm();
+ assertThat(raft(idx).currentTerm(term + 1)).isOne();
+ assertThat(raft(idx).currentTerm()).isEqualTo(term + 1);
+ assertThat(raft(idx).isLeader()).isFalse();
+ assertThat(raft(idx).role()).isEqualTo("Follower");
}
+ protected void assertNotElected(long timeout, int ... indexes) {
+ BooleanSupplier bs = () -> Arrays.stream(indexes)
+ .mapToObj(this::node)
+ .filter(Objects::nonNull)
+ .map(RaftNode::raft)
+ .anyMatch(RAFT::isLeader);
- protected void waitUntilVotingThreadHasStopped() throws TimeoutException {
- Util.waitUntil(5000, 100, () -> Stream.of(elections)
- .allMatch(el -> el == null || !el.isVotingThreadRunning()),
- this::printVotingThreads);
+ assertThat(RaftTestUtils.eventually(bs, timeout, TimeUnit.MILLISECONDS))
+ .as("Leader should not be elected")
+ .isFalse();
+ }
+
+ protected void waitUntilStepDown(long timeout, int ... indexes) {
+ BooleanSupplier bs = () -> Arrays.stream(indexes)
+ .mapToObj(this::node)
+ .filter(Objects::nonNull)
+ .map(RaftNode::raft)
+ .noneMatch(RAFT::isLeader);
+
+ assertThat(RaftTestUtils.eventually(bs, timeout, TimeUnit.MILLISECONDS))
+ .as("Leader should step down")
+ .isTrue();
+ }
+
+ protected void waitUntilVotingThreadHasStopped() {
+ waitUntilVotingThreadStops(5_000, IntStream.range(0, clusterSize).toArray());
}
// Checks that there is 1 leader in the cluster and the rest are followers
protected void assertOneLeader() {
- assert Stream.of(rafts).filter(r -> r != null && r.isLeader()).count() == 1 : print();
+ long count = Arrays.stream(nodes())
+ .filter(Objects::nonNull)
+ .map(RaftNode::raft)
+ .filter(RAFT::isLeader)
+ .count();
+ assertThat(count).as(this::print).isOne();
}
protected void assertSameTerm(Supplier message) {
long term=-1;
- for(int i=0; i < elections.length; i++) {
- if(elections[i] == null)
+ for (BaseElection election : elections()) {
+ if (election == null)
continue;
- if(term == -1)
- term=elections[i].raft().currentTerm();
- else
- assert term == elections[i].raft().currentTerm() : message.get();
+
+ if (term == -1) term=election.raft().currentTerm();
+ else assertThat(term).as(message).isEqualTo(election.raft().currentTerm());
}
}
protected void assertTerm(int expected_term, Supplier message) {
- for(int i=0; i < elections.length; i++) {
- if(elections[i] == null)
+ for (BaseElection election : elections()) {
+ if (election == null)
continue;
- assert expected_term == elections[i].raft().currentTerm() : message.get();
+ assertThat(expected_term).as(message).isEqualTo(election.raft().currentTerm());
}
}
-
- protected void kill(int index) throws Exception {
- System.out.printf("-- killing node %d (%s)\n", index, elections[index].getAddress());
- cluster.remove(nodes[index].getAddress());
- Utils.deleteLog(rafts[index]);
- nodes[index].stop();
- nodes[index].destroy();
- nodes[index]=null;
- elections[index]=null;
- rafts[index]=null;
- }
-
protected int findFirst(boolean leader) {
- for(int i=rafts.length-1; i >= 0; i--) {
- if(rafts[i] != null && rafts[i].isLeader() == leader)
+ for(int i=clusterSize-1; i >= 0; i--) {
+ RaftNode node = node(i);
+ if (node == null) continue;
+
+ RAFT raft = node.raft();
+ if(raft != null && raft.isLeader() == leader)
return i;
}
return -1;
}
protected String print() {
- return Stream.of(elections).filter(Objects::nonNull)
- .map(el -> String.format("%s: leader=%s, term=%d",
- el.getAddress(), el.raft().leader(), el.raft().currentTerm()))
+ return Stream.of(elections())
+ .filter(Objects::nonNull)
+ .map(el -> String.format("%s: leader=%s, term=%d", el.getAddress(), el.raft().leader(), el.raft().currentTerm()))
.collect(Collectors.joining("\n"));
}
-
- protected String printVotingThreads() {
- return Stream.of(elections).filter(Objects::nonNull)
- .map(e -> String.format("%s: voting thread=%b", e.getAddress(), e.isVotingThreadRunning()))
- .collect(Collectors.joining("\n"));
- }
-
- protected RaftNode createNode(int index, String name) throws Exception {
- rafts[index]=new RAFT().raftId(name).members(mbrs).logPrefix("sync-electiontest-" + name)
- .resendInterval(600_000) // long to disable resending by default
- .stateMachine(new DummyStateMachine())
- .synchronous(true).setAddress(addrs[index]);
- elections[index]=instantiate().raft(rafts[index]).setAddress(addrs[index]);
- RaftNode node=nodes[index]=new RaftNode(cluster, new Protocol[]{elections[index], rafts[index]});
- node.init();
- cluster.add(addrs[index], node);
- node.start();
- return node;
- }
-
- protected View createView() {
- List l=Stream.of(nodes).filter(Objects::nonNull).map(RaftNode::getAddress).collect(Collectors.toList());
- return l.isEmpty()? null : View.create(l.get(0), view_id++, l);
+ @Override
+ protected RaftCluster createNewMockCluster() {
+ return new RaftCluster();
}
- protected static Address createAddress(String name) {
- ExtendedUUID.setPrintFunction(RAFT.print_function);
- return ExtendedUUID.randomUUID(name).put(RAFT.raft_id_key, Util.stringToBytes(name));
+ @Override
+ protected void amendRAFTConfiguration(RAFT raft) {
+ raft.synchronous(true).stateMachine(new DummyStateMachine());
}
}
diff --git a/tests/junit-functional/org/jgroups/tests/SyncElectionWithRestrictionTest.java b/tests/junit-functional/org/jgroups/tests/SyncElectionWithRestrictionTest.java
index 627d797b..fd448dc9 100644
--- a/tests/junit-functional/org/jgroups/tests/SyncElectionWithRestrictionTest.java
+++ b/tests/junit-functional/org/jgroups/tests/SyncElectionWithRestrictionTest.java
@@ -11,26 +11,24 @@
import org.jgroups.protocols.raft.RaftImpl;
import org.jgroups.protocols.raft.election.BaseElection;
import org.jgroups.raft.testfwk.RaftCluster;
-import org.jgroups.raft.testfwk.RaftNode;
-import org.jgroups.raft.util.Utils;
-import org.jgroups.stack.Protocol;
-import org.jgroups.tests.election.BaseElectionTest;
-import org.jgroups.util.ExtendedUUID;
+import org.jgroups.tests.harness.BaseRaftElectionTest;
import org.jgroups.util.Util;
-import org.testng.annotations.AfterMethod;
-import org.testng.annotations.BeforeMethod;
-import org.testng.annotations.Test;
import java.util.ArrayList;
import java.util.Arrays;
import java.util.List;
import java.util.Objects;
-import java.util.concurrent.TimeoutException;
+import java.util.concurrent.TimeUnit;
import java.util.function.BooleanSupplier;
-import java.util.stream.Collectors;
import java.util.stream.Stream;
-import static org.jgroups.tests.election.BaseElectionTest.ALL_ELECTION_CLASSES_PROVIDER;
+import org.testng.annotations.AfterMethod;
+import org.testng.annotations.BeforeMethod;
+import org.testng.annotations.Test;
+
+import static org.assertj.core.api.Assertions.assertThat;
+import static org.jgroups.raft.testfwk.RaftTestUtils.eventually;
+import static org.jgroups.tests.harness.BaseRaftElectionTest.ALL_ELECTION_CLASSES_PROVIDER;
/**
* Uses the synchronous test framework to test {@link ELECTION}. Tests the election restrictions described in 5.4.1
@@ -40,40 +38,25 @@
* @since 1.0.7
*/
@Test(groups=Global.FUNCTIONAL,singleThreaded=true, dataProvider=ALL_ELECTION_CLASSES_PROVIDER)
-public class SyncElectionWithRestrictionTest extends BaseElectionTest {
- protected final Address s1,s2,s3,s4,s5;
- protected final Address[] addrs={s1=createAddress("S1"), s2=createAddress("S2"),
- s3=createAddress("S3"), s4=createAddress("S4"),
- s5=createAddress("S5")};
- protected final List mbrs=List.of("S1", "S2", "S3", "S4", "S5");
- protected final RaftCluster cluster=new RaftCluster();
- protected RAFT[] rafts=new RAFT[5];
- protected BaseElection[] elections=new BaseElection[5];
- protected RaftNode[] nodes=new RaftNode[5];
+public class SyncElectionWithRestrictionTest extends BaseRaftElectionTest.ClusterBased {
+
protected int view_id=1;
protected static final byte[] DATA={1,2,3,4,5};
+ {
+ // Following the paper nomenclature, S1, S2, S3, S4, S5.
+ clusterSize = 5;
+ createManually = true;
+ }
- @BeforeMethod protected void init() {view_id=1;}
+ @BeforeMethod
+ protected void init() {
+ view_id=1;
+ }
@AfterMethod
protected void destroy() throws Exception {
- for(int i=nodes.length-1; i >= 0; i--) {
- if(nodes[i] != null) {
- nodes[i].stop();
- nodes[i].destroy();
- nodes[i]=null;
- }
- if(elections[i] != null) {
- elections[i].stopVotingThread();
- elections[i]=null;
- }
- if(rafts[i] != null) {
- Utils.deleteLog(rafts[i]);
- rafts[i]=null;
- }
- }
- cluster.clear();
+ destroyCluster();
}
@@ -93,14 +76,13 @@ protected void destroy() throws Exception {
public void testScenarioD(Class> ignore) throws Exception {
createScenarioC();
System.out.printf("-- Initial:\n%s\n", printTerms());
- kill(0);
+ close(0);
makeLeader(4);
- View v=View.create(s5, view_id++, s5,s2,s3,s4);
+ View v=createView(view_id++, 4, 1, 2, 3);
cluster.handleView(v);
System.out.printf("-- After killing S1 and making S5 leader:\n%s\n", printTerms());
assertTerms(null, new long[]{1,2}, new long[]{1,2}, new long[]{1}, new long[]{1,3});
- RAFT r5=rafts[4];
- assert Util.waitUntilTrue(2_000, 250, r5::isLeader) : "S5 was not leader";
+ RAFT r5 = waitUntilNodeLeader(4);
r5.flushCommitTable();
System.out.printf("-- After S1 resending messages:\n%s\n\n", printTerms());
long[] expected={1,3};
@@ -122,40 +104,46 @@ public void testScenarioD(Class> ignore) throws Exception {
*/
public void testScenarioE(Class> ignore) throws Exception {
createScenarioC();
- View v=createView();
+ View v=createView(view_id++, 0, 1, 2, 3, 4);
cluster.handleView(v);
+
// append term 4 on S2 and S3:
- RAFT r1=rafts[0];
- assert Util.waitUntilTrue(2_000, 250, r1::isLeader) : "S1 was never leader!";
- r1.flushCommitTable(s2);
- r1.flushCommitTable(s3);
+ RAFT r1=waitUntilNodeLeader(0);
+ System.out.println("-- flushing tables");
+ r1.flushCommitTable(address(1));
+ r1.flushCommitTable(address(2));
System.out.printf("-- Initial:\n%s\n", printTerms());
- kill(0);
- v=View.create(s2, view_id++, s2,s3,s4,s5);
+ close(0);
+ v=createView(view_id++, 1, 2, 3, 4);
cluster.handleView(v);
System.out.printf("-- After killing S1:\n%s\n", printTerms());
assertTerms(null, new long[]{1,2,4}, new long[]{1,2,4}, new long[]{1}, new long[]{1,3});
// start voting to find current leader:
- BaseElection e2=elections[1];
+ BaseElection e2=election(1);
e2.startVotingThread();
- Util.waitUntilTrue(5000, 200, () -> Stream.of(rafts).filter(Objects::nonNull).anyMatch(RAFT::isLeader));
- System.out.printf("-- After the voting phase (either S2 or S3 will be leader):\n%s\n", printTerms());
-
- List leaders=Stream.of(rafts).filter(Objects::nonNull)
- .filter(RAFT::isLeader).map(Protocol::getAddress).collect(Collectors.toList());
- assert leaders.size() == 1 : "Should have a single leader: " + leaders;
- assert leaders.contains(s2) || leaders.contains(s3);
- RAFT leader_raft=rafts[1].isLeader()? rafts[1] : rafts[2];
+ waitUntilVotingThreadStops(5_000, 1, 2, 3, 4);
+ waitUntilLeaderElected(5_000, 1, 2, 3, 4);
+
+ String leaderAndTerms = dumpLeaderAndTerms();
+ System.out.printf("-- After the voting phase (either S2 or S3 will be leader):\n%s\n", leaderAndTerms);
+ assertThat(leaders())
+ .as(leaderAndTerms)
+ .hasSize(1)
+ .containsAnyOf(address(1), address(2));
+
+ RAFT leader_raft=raft(1).isLeader()? raft(1) : raft(2);
+ assertThat(leader_raft)
+ .as(leaderAndTerms)
+ .isNotNull()
+ .satisfies(r -> assertThat(r.isLeader()).isTrue());
// first round adjusts match-index for S4 and deletes term=3 at index 2 on S5
// second round sends term=2 at index 2
// third round sends term=4 at index 3
- //for(int i=1; i <= 3; i++)
- // leader_raft.flushCommitTable();
long[] expected={1,2,4};
- waitUntilTerms(10000,1000,expected,leader_raft::flushCommitTable);
+ waitUntilTerms(10000,1000, expected, leader_raft::flushCommitTable);
assertTerms(null, expected, expected, expected, expected);
System.out.printf("-- final terms:\n%s\n", printTerms());
}
@@ -163,10 +151,10 @@ public void testScenarioE(Class> ignore) throws Exception {
/** Creates scenario C in fig 8 */
protected void createScenarioC() throws Exception {
- for(int i=0; i < mbrs.size(); i++)
- createNode(i, mbrs.get(i));
+ withClusterSize(5);
+ createCluster();
makeLeader(0); // S1 is leader
- Address leader=rafts[0].getAddress();
+ Address leader=raft(0).getAddress();
append(leader, 0, 1, 2, 4);
append(leader, 1, 1,2);
append(leader, 2, 1,2);
@@ -175,12 +163,12 @@ protected void createScenarioC() throws Exception {
}
protected void append(Address leader, int index, int ... terms) {
- RaftImpl impl=rafts[index].impl();
+ RaftImpl impl=raft(index).impl();
for(int i=0; i < terms.length; i++) {
int curr_index=i+1, prev_index=curr_index-1;
int prev_term=prev_index == 0? 0 : terms[prev_index-1];
int curr_term=terms[curr_index-1];
- rafts[index].currentTerm(curr_term);
+ raft(index).currentTerm(curr_term);
LogEntries entries=new LogEntries().add(new LogEntry(curr_term, DATA));
impl.handleAppendEntriesRequest(entries, leader, i, prev_term, curr_term, 0);
}
@@ -189,34 +177,23 @@ protected void append(Address leader, int index, int ... terms) {
protected void assertTerms(long[] ... exp_terms) {
int index=0;
for(long[] expected_terms: exp_terms) {
- RAFT r=rafts[index++];
+ RAFT r=raft(index++);
if(r == null && expected_terms == null)
continue;
long[] actual_terms=terms(r);
- assert Arrays.equals(expected_terms, actual_terms) :
- String.format("%s: expected terms: %s, actual terms: %s", r.getAddress(),
- Arrays.toString(expected_terms), Arrays.toString(actual_terms));
+ assertThat(actual_terms)
+ .as(() -> String.format("%s: expected terms: %s, actual terms: %s", r.getAddress(),
+ Arrays.toString(expected_terms), Arrays.toString(actual_terms)))
+ .isEqualTo(expected_terms);
}
}
protected void waitUntilTerms(long timeout, long interval, long[] expexted_terms, Runnable action) {
- assert waitUntilTrue(timeout, interval,
- () -> Stream.of(rafts).filter(Objects::nonNull)
- .allMatch(r -> Arrays.equals(terms(r),expexted_terms)),
- action) : generateErrorMessage();
- }
-
- private String generateErrorMessage() {
- StringBuilder sb = new StringBuilder("\n");
- for (RAFT raft : rafts) {
- if (raft == null) continue;
-
- sb.append(raft.raftId())
- .append(" -> ")
- .append(Arrays.toString(terms(raft)))
- .append("\n");
- }
- return sb.toString();
+ BooleanSupplier bs = () -> Stream.of(rafts()).filter(Objects::nonNull)
+ .allMatch(r -> Arrays.equals(terms(r),expexted_terms));
+ assertThat(waitUntilTrue(timeout, interval, bs, action))
+ .as(this::dumpLeaderAndTerms)
+ .isTrue();
}
public static boolean waitUntilTrue(long timeout, long interval, BooleanSupplier condition,
@@ -236,22 +213,23 @@ public static boolean waitUntilTrue(long timeout, long interval, BooleanSupplier
/** Make the node at index leader, and everyone else follower (ignores election) */
protected void makeLeader(int index) {
- Address leader=rafts[index].getAddress();
- long term = rafts[index].currentTerm();
- for(int i=0; i < rafts.length; i++) {
- if(rafts[i] == null)
+ Address leader=raft(index).getAddress();
+ long term = raft(index).currentTerm();
+ for(int i=0; i < rafts().length; i++) {
+ if(raft(i) == null)
continue;
- rafts[i].setLeaderAndTerm(leader, term + 1);
+ raft(i).setLeaderAndTerm(leader, term + 1);
}
}
protected String printTerms() {
StringBuilder sb=new StringBuilder(" 1 2 3\n -----\n");
+ List mbrs = new ArrayList<>(getRaftMembers());
for(int i=0; i < mbrs.size(); i++) {
String name=mbrs.get(i);
- RAFT r=rafts[i];
+ RAFT r=raft(i);
if(r == null) {
- sb.append("XX\n");
+ sb.append(name).append(": XX\n");
continue;
}
Log l=r.log();
@@ -278,49 +256,26 @@ protected static long[] terms(RAFT r) {
return list.stream().mapToLong(Long::longValue).toArray();
}
- protected void kill(int index) throws Exception {
- cluster.remove(nodes[index].getAddress());
- nodes[index].stop();
- nodes[index].destroy();
- nodes[index]=null;
- if(elections[index] != null) {
- elections[index].stopVotingThread();
- elections[index]=null;
- }
- if(rafts[index] != null) {
- Utils.deleteLog(rafts[index]);
- rafts[index]=null;
- }
+ private RAFT waitUntilNodeLeader(int node) {
+ RAFT raft = raft(node);
+ assertThat(raft).as("Node at index " + node + " is null").isNotNull();
+
+ Address leaderAddress = raft.getAddress();
+ BooleanSupplier bs = () -> Arrays.stream(rafts())
+ .allMatch(r -> raft(node).currentTerm() == r.currentTerm() && leaderAddress.equals(r.leader()));
+ assertThat(eventually(bs, 5, TimeUnit.SECONDS))
+ .as(this::dumpLeaderAndTerms)
+ .isTrue();
+ return raft;
}
-
- protected RaftNode createNode(int index, String name) throws Exception {
- rafts[index]=new RAFT().raftId(name).members(mbrs).logPrefix("sync-electiontest-restriction-" + name)
- .resendInterval(600_000) // long to disable resending by default
- .stateMachine(new DummyStateMachine())
- .synchronous(true).setAddress(addrs[index]);
- elections[index]=instantiate().raft(rafts[index]).voteTimeout(1_000).setAddress(addrs[index]);
- RaftNode node=nodes[index]=new RaftNode(cluster, new Protocol[]{elections[index], rafts[index]});
- node.init();
- cluster.add(addrs[index], node);
- node.start();
- return node;
- }
-
- protected View createView() {
- List l=Stream.of(nodes).filter(Objects::nonNull).map(RaftNode::getAddress).collect(Collectors.toList());
- return l.isEmpty()? null : View.create(l.get(0), view_id++, l);
+ @Override
+ protected RaftCluster createNewMockCluster() {
+ return new RaftCluster();
}
- protected static Address createAddress(String name) {
- ExtendedUUID.setPrintFunction(RAFT.print_function);
- return ExtendedUUID.randomUUID(name).put(RAFT.raft_id_key, Util.stringToBytes(name));
+ @Override
+ protected void amendRAFTConfiguration(RAFT raft) {
+ raft.synchronous(true).stateMachine(new DummyStateMachine());
}
-
-
- protected void waitUntilVotingThreadHasStopped() throws TimeoutException {
- Util.waitUntil(5000, 100, () -> Stream.of(elections)
- .allMatch(el -> el == null || !el.isVotingThreadRunning()));
- }
-
}
diff --git a/tests/junit-functional/org/jgroups/tests/SyncLeaderCrashTest.java b/tests/junit-functional/org/jgroups/tests/SyncLeaderCrashTest.java
index 7a104b0d..1e15e07e 100644
--- a/tests/junit-functional/org/jgroups/tests/SyncLeaderCrashTest.java
+++ b/tests/junit-functional/org/jgroups/tests/SyncLeaderCrashTest.java
@@ -4,36 +4,28 @@
import org.jgroups.Global;
import org.jgroups.View;
import org.jgroups.protocols.raft.AppendResult;
-import org.jgroups.protocols.raft.ELECTION2;
import org.jgroups.protocols.raft.Log;
import org.jgroups.protocols.raft.LogEntries;
import org.jgroups.protocols.raft.LogEntry;
import org.jgroups.protocols.raft.RAFT;
-import org.jgroups.protocols.raft.election.BaseElection;
import org.jgroups.raft.testfwk.RaftCluster;
-import org.jgroups.raft.testfwk.RaftNode;
import org.jgroups.raft.util.CounterStateMachine;
-import org.jgroups.raft.util.Utils;
-import org.jgroups.stack.Protocol;
-import org.jgroups.tests.election.BaseElectionTest;
+import org.jgroups.tests.harness.BaseRaftElectionTest;
import org.jgroups.util.Bits;
-import org.jgroups.util.ExtendedUUID;
-import org.jgroups.util.Util;
-import org.testng.annotations.AfterMethod;
-import org.testng.annotations.BeforeMethod;
-import org.testng.annotations.Test;
import java.util.ArrayList;
import java.util.Arrays;
import java.util.List;
import java.util.Objects;
import java.util.concurrent.TimeUnit;
-import java.util.concurrent.TimeoutException;
-import java.util.function.BooleanSupplier;
import java.util.stream.Collectors;
import java.util.stream.Stream;
-import static org.jgroups.tests.election.BaseElectionTest.ALL_ELECTION_CLASSES_PROVIDER;
+import org.testng.annotations.AfterMethod;
+import org.testng.annotations.BeforeMethod;
+import org.testng.annotations.Test;
+
+import static org.jgroups.tests.harness.BaseRaftElectionTest.ALL_ELECTION_CLASSES_PROVIDER;
/**
* Tests replaying of requests after leader changes, to advance commit index
@@ -42,69 +34,48 @@
* @since 1.0.7
*/
@Test(groups=Global.FUNCTIONAL,singleThreaded=true, dataProvider = ALL_ELECTION_CLASSES_PROVIDER)
-public class SyncLeaderCrashTest extends BaseElectionTest {
- protected final Address a, b, c;
- protected final Address[] addrs={a=createAddress("A"), b=createAddress("B"),
- c=createAddress("C")};
- protected final List mbrs=List.of("A", "B", "C");
- protected final RaftCluster cluster=new RaftCluster();
- protected RAFT[] rafts=new RAFT[3];
- protected BaseElection[] elections=new BaseElection[3];
- protected RaftNode[] nodes=new RaftNode[3];
- protected CounterStateMachine[] sms;
+public class SyncLeaderCrashTest extends BaseRaftElectionTest.ClusterBased {
protected int view_id=1;
protected final byte[] DATA=new byte[Integer.BYTES];
+ {
+ clusterSize = 3;
+ createManually = true;
+ }
-
- @BeforeMethod protected void init() {
+ @BeforeMethod
+ protected void init() {
view_id=1;
- sms=new CounterStateMachine[]{new CounterStateMachine(), new CounterStateMachine(), new CounterStateMachine()};
Bits.writeInt(1, DATA, 0);
}
@AfterMethod
protected void destroy() throws Exception {
- for(int i=nodes.length-1; i >= 0; i--) {
- if(nodes[i] != null) {
- nodes[i].stop();
- nodes[i].destroy();
- nodes[i]=null;
- }
- if(elections[i] != null) {
- elections[i].stopVotingThread();
- elections[i]=null;
- }
- if(rafts[i] != null) {
- Utils.deleteLog(rafts[i]);
- rafts[i]=null;
- }
- }
- cluster.clear();
+ destroyCluster();
}
public void testsLeaderCrash(Class> ignore) throws Exception {
prepare();
System.out.println("-- Adding requests 5, 6 and 7 to A, B and C (not yet committing them); then crashing A");
- for(RAFT r: rafts) {
+ for(RAFT r: rafts()) {
for(int i=5; i <= 7; i++) {
Log l=r.log();
long prev_term=l.get(i-1).term();
LogEntries entries=new LogEntries().add(new LogEntry(9, DATA));
- AppendResult ar = r.impl().handleAppendEntriesRequest(entries, a,i-1, prev_term, 9, 4);
+ AppendResult ar = r.impl().handleAppendEntriesRequest(entries, address(0),i-1, prev_term, 9, 4);
assert ar != null && ar.success() : String.format("%s failed on %d with %s", r.raftId(), i, ar);
}
}
- kill(0);
- View v=View.create(b, view_id++, b,c);
+ close(0);
+ View v=createView(view_id++, 1, 2);
cluster.handleView(v);
- waitUntilLeaderElected();
+ waitUntilLeaderElected(5_000, 1, 2);
System.out.printf("\n-- Terms after leader A left:\n\n%s\n-- Indices:\n%s\n\n", printTerms(), printIndices(null));
assertIndices(7, 4);
- RAFT leader=Stream.of(rafts).filter(r -> r != null && r.isLeader()).findFirst().orElse(null);
+ RAFT leader=Stream.of(rafts()).filter(r -> r != null && r.isLeader()).findFirst().orElse(null);
assert leader != null;
System.out.printf("-- Leader: %s, commit-table:\n%s\n", leader.getAddress(), leader.commitTable());
@@ -114,16 +85,19 @@ public void testsLeaderCrash(Class> ignore) throws Exception {
assertIndices(7, 7);
System.out.printf("-- State machines:\n%s\n", printStateMachines());
- assert Stream.of(sms).filter(Objects::nonNull).allMatch(sm -> sm.counter() == 7)
+ assert Arrays.stream(rafts())
+ .map(RAFT::stateMachine)
+ .map(sm -> (CounterStateMachine) sm)
+ .filter(Objects::nonNull)
+ .allMatch(sm -> sm.counter() == 7)
: String.format("expected counter of 7, actual:\n%s\n", printStateMachines());
assert leader.requestTableSize() == 0 : String.format("req_table should be 0, but is %d", leader.requestTableSize());
// restart A and see if indixes and state machines match
System.out.println("\n-- Restarting A");
- sms[0]=new CounterStateMachine();
- createNode(0, mbrs.get(0));
- v=View.create(leader.getAddress(), view_id++, b,c,a);
+ createCluster();
+ v=createView(view_id++, 1, 2, 0);
cluster.handleView(v);
leader.flushCommitTable();
@@ -134,24 +108,28 @@ public void testsLeaderCrash(Class> ignore) throws Exception {
assertIndices(7, 7);
System.out.printf("-- State machines:\n%s\n", printStateMachines());
- assert Stream.of(sms).filter(Objects::nonNull).allMatch(sm -> sm.counter() == 7)
+ assert Arrays.stream(rafts())
+ .map(RAFT::stateMachine)
+ .map(sm -> (CounterStateMachine) sm)
+ .filter(Objects::nonNull)
+ .allMatch(sm -> sm.counter() == 7)
: String.format("expected counter of 7, actual:\n%s\n", printStateMachines());
}
/** Creates A,B,C, sends 4 requests and sets commit-index=last-appended to 4 */
void prepare() throws Exception {
- for(int i=0; i < mbrs.size(); i++)
- createNode(i, mbrs.get(i));
+ withClusterSize(3);
+ createCluster();
makeLeader(0); // A is leader
- View v=View.create(a, view_id++, a,b,c);
- cluster.handleView(v);
- waitUntilLeaderElected();
+ cluster.handleView(createView(view_id++, 0, 1, 2));
+ waitUntilVotingThreadStops(5_000, 0, 1, 2);
+ waitUntilLeaderElected(5_000, 0, 1, 2);
long[] terms={2,5,5,7};
- RAFT r=rafts[0];
- assert r.isLeader();
+ RAFT r=raft(0);
+ assert r.isLeader() : dumpLeaderAndTerms();
Address leader = r.leader();
r.setLeaderAndTerm(leader, 2);
r.set(DATA, 0, DATA.length, 5, TimeUnit.SECONDS);
@@ -164,7 +142,7 @@ void prepare() throws Exception {
System.out.printf("terms:\n%s\n", printTerms());
assertTerms(terms, terms, terms);
- rafts[0].flushCommitTable(); // updates the commit index
+ raft(0).flushCommitTable(); // updates the commit index
System.out.printf("-- Indices:\n%s\n", printIndices(null));
assertIndices(4, 4);
}
@@ -179,7 +157,7 @@ protected static byte[] intToByte(int num) {
protected void assertTerms(long[] ... exp_terms) {
int index=0;
for(long[] expected_terms: exp_terms) {
- RAFT r=rafts[index++];
+ RAFT r=raft(index++);
if(r == null && expected_terms == null)
continue;
long[] actual_terms=terms(r);
@@ -190,7 +168,7 @@ protected void assertTerms(long[] ... exp_terms) {
}
protected void assertIndices(int expected_last_appended, int expected_commit) {
- for(RAFT r: rafts) {
+ for(RAFT r: rafts()) {
if(r == null)
continue;
Log l=r.log();
@@ -202,28 +180,30 @@ protected void assertIndices(int expected_last_appended, int expected_commit) {
protected String printIndices(RAFT r) {
if(r == null)
- return Stream.of(rafts).filter(Objects::nonNull).map(this::printIndices).collect(Collectors.joining("\n"));
+ return Stream.of(rafts()).filter(Objects::nonNull).map(this::printIndices).collect(Collectors.joining("\n"));
return String.format("%s: commit=%d, log-commit=%d, last=%d, log-last=%d", r.getAddress(),
r.commitIndex(), r.log().commitIndex(), r.lastAppended(), r.log().lastAppended());
}
/** Make the node at index leader, and everyone else follower (ignores election) */
protected void makeLeader(int index) {
- Address leader=rafts[index].getAddress();
- for(int i=0; i < rafts.length; i++) {
- if(rafts[i] == null)
+ Address leader=raft(index).getAddress();
+ long term = raft(index).currentTerm();
+ for(int i=0; i < rafts().length; i++) {
+ if(raft(i) == null)
continue;
- rafts[i].setLeaderAndTerm(leader);
+ raft(i).setLeaderAndTerm(leader, term + 1);
}
}
protected String printTerms() {
StringBuilder sb=new StringBuilder(" 1 2 3 4 5 6 7\n -------------\n");
+ List mbrs = new ArrayList<>(getRaftMembers());
for(int i=0; i < mbrs.size(); i++) {
String name=mbrs.get(i);
- RAFT r=rafts[i];
+ RAFT r=raft(i);
if(r == null) {
- sb.append("XX\n");
+ sb.append(name).append(": XX\n");
continue;
}
Log l=r.log();
@@ -241,7 +221,7 @@ protected String printTerms() {
}
protected String printStateMachines() {
- return Stream.of(rafts).filter(Objects::nonNull).map(r -> String.format("%s: %s", r.getAddress(), r.stateMachine()))
+ return Stream.of(rafts()).filter(Objects::nonNull).map(r -> String.format("%s: %s", r.getAddress(), r.stateMachine()))
.collect(Collectors.joining("\n"));
}
@@ -255,53 +235,13 @@ protected static long[] terms(RAFT r) {
return list.stream().mapToLong(Long::longValue).toArray();
}
- protected void kill(int index) throws Exception {
- cluster.remove(nodes[index].getAddress());
- nodes[index].stop();
- nodes[index].destroy();
- nodes[index]=null;
- if(elections[index] != null) {
- elections[index].stopVotingThread();
- elections[index]=null;
- }
- if(rafts[index] != null) {
- Utils.deleteLog(rafts[index]);
- rafts[index]=null;
- }
- sms[index]=null;
- }
-
-
- protected RaftNode createNode(int index, String name) throws Exception {
- rafts[index]=new RAFT().raftId(name).members(mbrs).logPrefix("sync-leadercrash-" + name)
- .resendInterval(600_000) // long to disable resending by default
- .stateMachine(sms[index])
- .synchronous(true).setAddress(addrs[index]);
- elections[index]=instantiate().raft(rafts[index]).setAddress(addrs[index]);
- RaftNode node=nodes[index]=new RaftNode(cluster, new Protocol[]{elections[index], rafts[index]});
- node.init();
- cluster.add(addrs[index], node);
- node.start();
- return node;
- }
-
- protected View createView() {
- List l=Stream.of(nodes).filter(Objects::nonNull).map(RaftNode::getAddress).collect(Collectors.toList());
- return l.isEmpty()? null : View.create(l.get(0), view_id++, l);
- }
-
- protected static Address createAddress(String name) {
- ExtendedUUID.setPrintFunction(RAFT.print_function);
- return ExtendedUUID.randomUUID(name).put(RAFT.raft_id_key, Util.stringToBytes(name));
+ @Override
+ protected RaftCluster createNewMockCluster() {
+ return new RaftCluster();
}
-
- protected void waitUntilLeaderElected() throws TimeoutException {
- // ELECTION2 the voting thread has a delayed start, so we wait for a leader elected instead.
- BooleanSupplier supplier = electionClass == ELECTION2.class
- ? () -> Stream.of(elections).anyMatch(el -> el != null && el.raft().isLeader())
- : () -> Stream.of(elections).allMatch(el -> el == null || !el.isVotingThreadRunning());
- Util.waitUntil(5000, 100, supplier);
+ @Override
+ protected void amendRAFTConfiguration(RAFT raft) {
+ raft.synchronous(true).stateMachine(new CounterStateMachine());
}
-
}
diff --git a/tests/junit-functional/org/jgroups/tests/SynchronousTests.java b/tests/junit-functional/org/jgroups/tests/SynchronousTests.java
index 6794eb6c..09bbebd0 100644
--- a/tests/junit-functional/org/jgroups/tests/SynchronousTests.java
+++ b/tests/junit-functional/org/jgroups/tests/SynchronousTests.java
@@ -1,24 +1,39 @@
package org.jgroups.tests;
-import org.jgroups.*;
-import org.jgroups.protocols.raft.*;
+import org.jgroups.Address;
+import org.jgroups.Global;
+import org.jgroups.Message;
+import org.jgroups.ObjectMessage;
+import org.jgroups.View;
+import org.jgroups.protocols.raft.AppendEntriesRequest;
+import org.jgroups.protocols.raft.AppendResult;
+import org.jgroups.protocols.raft.FileBasedLog;
+import org.jgroups.protocols.raft.LevelDBLog;
+import org.jgroups.protocols.raft.Log;
+import org.jgroups.protocols.raft.LogEntries;
+import org.jgroups.protocols.raft.LogEntry;
+import org.jgroups.protocols.raft.RAFT;
+import org.jgroups.protocols.raft.RaftImpl;
import org.jgroups.raft.Options;
import org.jgroups.raft.testfwk.RaftCluster;
import org.jgroups.raft.testfwk.RaftNode;
+import org.jgroups.raft.testfwk.RaftTestUtils;
import org.jgroups.raft.util.CommitTable;
import org.jgroups.raft.util.CounterStateMachine;
-import org.jgroups.raft.util.Utils;
+import org.jgroups.stack.Protocol;
+import org.jgroups.tests.harness.BaseRaftClusterTest;
import org.jgroups.util.Bits;
-import org.jgroups.util.ExtendedUUID;
import org.jgroups.util.Util;
+
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.TimeUnit;
+
import org.testng.annotations.AfterMethod;
import org.testng.annotations.BeforeMethod;
import org.testng.annotations.DataProvider;
import org.testng.annotations.Test;
-import java.util.List;
-import java.util.concurrent.CompletableFuture;
-import java.util.concurrent.TimeUnit;
+import static org.assertj.core.api.Assertions.assertThat;
/**
* Same as {@link RaftTest}, but only a single thread is used to run the tests (no asynchronous processing).
@@ -28,89 +43,99 @@
* @since 1.0.5
*/
@Test(groups=Global.FUNCTIONAL,singleThreaded=true,dataProvider="logProvider")
-public class SynchronousTests {
- protected final Address a=createAddress("A"), b=createAddress("B"), c=createAddress("C");
+public class SynchronousTests extends BaseRaftClusterTest {
protected View view;
- protected final List mbrs=List.of("A", "B","C");
- protected final RaftCluster cluster=new RaftCluster();
- protected RAFT raft_a, raft_b, raft_c;
- protected RaftNode node_a, node_b, node_c;
- protected CounterStateMachine sma, smb, smc;
protected static final int TERM=5;
private Class extends Log> logClass;
+ {
+ clusterSize = 3;
+ recreatePerMethod = true;
+ createManually = true;
+ }
+
+ @Override
+ protected void passDataProviderParameters(Object[] args) {
+ logClass = (Class extends Log>) args[0];
+ }
+
+ @Override
+ protected void amendRAFTConfiguration(RAFT raft) {
+ raft.stateMachine(new CounterStateMachine())
+ .synchronous(true)
+ .resendInterval(600_00);
+ }
+
+ @Override
+ protected String getRaftLogClass() {
+ return logClass.getCanonicalName();
+ }
+
+ @Override
+ protected Protocol[] baseProtocolStackForNode(String name) throws Exception {
+ return new Protocol[] {
+ createNewRaft(name),
+ };
+ }
+
@DataProvider
static Object[][] logProvider() {
return new Object[][] {
- {LevelDBLog.class},
- {FileBasedLog.class}
+ {LevelDBLog.class},
+ {FileBasedLog.class}
};
}
- @BeforeMethod protected void init(Object[] args) throws Exception {
- // args is the arguments from each test method
- logClass = (Class extends Log>) args[0];
- view=View.create(a, 1, a,b);
- raft_a=createRAFT(a, "A", mbrs).stateMachine(sma=new CounterStateMachine()).logClass(logClass.getCanonicalName());
- raft_b=createRAFT(b, "B", mbrs).stateMachine(smb=new CounterStateMachine()).logClass(logClass.getCanonicalName());
- node_a=new RaftNode(cluster, raft_a);
- node_b=new RaftNode(cluster, raft_b);
- node_a.init();
- node_b.init();
- cluster.add(a, node_a).add(b, node_b);
+ @BeforeMethod
+ protected void init() throws Exception {
+ createCluster(2);
+
+ view = createView(1, 0, 1);
cluster.handleView(view);
- node_a.start();
- node_b.start();
- raft_a.setLeaderAndTerm(a, TERM);
- raft_b.setLeaderAndTerm(a, TERM);
+ raft(0).setLeaderAndTerm(address(0), TERM);
+ raft(1).setLeaderAndTerm(address(0), TERM);
}
-
-
- @AfterMethod
+ @AfterMethod(alwaysRun = true)
protected void destroy() throws Exception {
- if(node_c != null) {
- node_c.stop();
- node_c.destroy();
- Utils.deleteLog(raft_c);
- }
- Util.close(node_b, node_a);
- node_b.destroy();
- node_a.destroy();
- Utils.deleteLog(raft_a);
- Utils.deleteLog(raft_b);
- cluster.clear();
+ destroyCluster();
}
public void testSimpleAppend(@SuppressWarnings("unused") Class> logClass) throws Exception {
int prev_value=add(1);
assert prev_value == 0;
- assert sma.counter() == 1;
- assert smb.counter() == 0; // not yet committed
+ assert stateMachine(0).counter() == 1;
+ assert stateMachine(1).counter() == 0; // not yet committed
prev_value=add(1);
assert prev_value == 1;
- assert sma.counter() == 2;
- assert smb.counter() == 1; // not yet committed
+ assert stateMachine(0).counter() == 2;
+ assert stateMachine(1).counter() == 1; // not yet committed
}
public void testRegularAppend(@SuppressWarnings("unused") Class> logClass) throws Exception {
int prev_value=add(1);
expect(0, prev_value);
+
+ CounterStateMachine sma = stateMachine(0);
+ CounterStateMachine smb = stateMachine(1);
assert sma.counter() == 1;
assert smb.counter() == 0; // resend_interval is big, so the commit index on B won't get updated
- assertIndices(1, 1, TERM, raft_a);
- assertIndices(1, 0, TERM, raft_b);
- assertCommitTableIndeces(b, raft_a, 0, 1, 2);
+
+ RAFT raft_a = raft(0);
+ RAFT raft_b = raft(1);
+ assertIndices(1, 1, TERM, raft(0));
+ assertIndices(1, 0, TERM, raft(1));
+ assertCommitTableIndeces(address(1), raft_a, 0, 1, 2);
prev_value=add(2);
assert prev_value == 1;
assert sma.counter() == 3;
assert smb.counter() == 1; // previous value; B is always lagging one commit behind
- assertCommitTableIndeces(b, raft_a, 1, 2, 3);
+ assertCommitTableIndeces(address(1), raft_a, 1, 2, 3);
prev_value=add(3);
assert prev_value == 3;
@@ -118,7 +143,7 @@ public void testRegularAppend(@SuppressWarnings("unused") Class> logClass) thr
assert smb.counter() == 3; // previous value; B is always lagging one commit behind
assertIndices(3, 3, TERM, raft_a);
assertIndices(3, 2, TERM, raft_b);
- assertCommitTableIndeces(b, raft_a, 2, 3, 4);
+ assertCommitTableIndeces(address(1), raft_a, 2, 3, 4);
prev_value=add(-3);
assert prev_value == 6;
@@ -126,7 +151,7 @@ public void testRegularAppend(@SuppressWarnings("unused") Class> logClass) thr
assert smb.counter() == 6; // previous value; B is always lagging one commit behind
assertIndices(4, 4, TERM, raft_a);
assertIndices(4, 3, TERM, raft_b);
- assertCommitTableIndeces(b, raft_a, 3, 4, 5);
+ assertCommitTableIndeces(address(1), raft_a, 3, 4, 5);
for(int i=1,prev=3; i <= 1000; i++) {
prev_value=add(5);
@@ -138,10 +163,10 @@ public void testRegularAppend(@SuppressWarnings("unused") Class> logClass) thr
assertIndices(1004, 1004, TERM, raft_a);
assertIndices(1004, 1003, TERM, raft_b);
- assertCommitTableIndeces(b, raft_a, 1003, 1004, 1005);
+ assertCommitTableIndeces(address(1), raft_a, 1003, 1004, 1005);
long current_term=raft_a.currentTerm(), expected_term;
- raft_a.setLeaderAndTerm(a, expected_term=current_term + 10);
+ raft_a.setLeaderAndTerm(address(0), expected_term=current_term + 10);
for(int i=1; i <= 7; i++)
add(1);
@@ -151,17 +176,24 @@ public void testRegularAppend(@SuppressWarnings("unused") Class> logClass) thr
assertIndices(1011, 1011, expected_term, raft_a);
assertIndices(1011, 1010, expected_term, raft_b);
- assertCommitTableIndeces(b, raft_a, 1010, 1011, 1012);
+ assertCommitTableIndeces(address(1), raft_a, 1010, 1011, 1012);
}
public void testAppendSameElementTwice(@SuppressWarnings("unused") Class> logClass) throws Exception {
- raft_a.setLeaderAndTerm(a, 20);
+ RAFT raft_a = raft(0);
+ raft_a.setLeaderAndTerm(address(0), 20);
+
+ RAFT raft_b = raft(1);
+
for(int i=1; i <= 5; i++)
add(1);
assertIndices(5, 5, 20, raft_a);
assertIndices(5, 4, 20, raft_b);
- assertCommitTableIndeces(b, raft_a, 4, 5, 6);
+ assertCommitTableIndeces(address(1), raft_a, 4, 5, 6);
+
+ CounterStateMachine sma = stateMachine(0);
+ CounterStateMachine smb = stateMachine(1);
expect(5, sma.counter());
expect(4, smb.counter());
@@ -171,40 +203,50 @@ public void testAppendSameElementTwice(@SuppressWarnings("unused") Class> logC
Util.waitUntil(5000, 50, () -> raft_b.commitIndex() == 5);
assertIndices(5, 5, 20, raft_a);
assertIndices(5, 5, 20, raft_b);
- assertCommitTableIndeces(b, raft_a, 5, 5, 6);
+ assertCommitTableIndeces(address(1), raft_a, 5, 5, 6);
assert sma.counter() == 5;
assert smb.counter() == 5;
}
public void testAppendBeyondLast(@SuppressWarnings("unused") Class> logClass) throws Exception {
- raft_a.setLeaderAndTerm(a, 22);
+ RAFT raft_a = raft(0);
+ raft_a.setLeaderAndTerm(address(0), 22);
for(int i=1; i <= 5; i++)
add(1);
+
+ CounterStateMachine sma = stateMachine(0);
+ CounterStateMachine smb = stateMachine(1);
assert sma.counter() == 5;
assert smb.counter() == 4; // resend_interval is big, so the commit index on B won't get updated
assertIndices(5, 5, 22, raft_a);
- assertIndices(5, 4, 22, raft_b);
- assertCommitTableIndeces(b, raft_a, 4, 5, 6);
+ assertIndices(5, 4, 22, raft(1));
+ assertCommitTableIndeces(address(1), raft_a, 4, 5, 6);
// now append beyond the end:
sendAppendEntriesRequest(raft_a, 9, 22, 22, 0);
// nothing changed, as request was rejected
- assertCommitTableIndeces(b, raft_a, 4, 5, 6);
+ assertCommitTableIndeces(address(1), raft_a, 4, 5, 6);
}
/** Tests appding with correct prev_index, but incorrect term */
public void testAppendWrongTerm(@SuppressWarnings("unused") Class> logClass) throws Exception {
- raft_a.setLeaderAndTerm(a, 22);
+ RAFT raft_a = raft(0);
+ raft_a.setLeaderAndTerm(address(0), 22);
+
+ RAFT raft_b = raft(1);
+
for(int i=1; i <= 15; i++) {
if(i % 5 == 0)
- raft_a.setLeaderAndTerm(a, raft_a.currentTerm()+1);
+ raft_a.setLeaderAndTerm(address(0), raft_a.currentTerm()+1);
add(1);
}
+ CounterStateMachine sma = stateMachine(0);
+ CounterStateMachine smb = stateMachine(1);
expect(15, sma.counter());
expect(14, smb.counter()); // resend_interval is big, so the commit index on B won't get updated
assertIndices(15, 15, 25, raft_a);
assertIndices(15, 14, 25, raft_b);
- assertCommitTableIndeces(b, raft_a, 14, 15, 16);
+ assertCommitTableIndeces(address(1), raft_a, 14, 15, 16);
// now append entries 16,17 and 18 (all with term=25), but *don't* advance the commit index
for(int i=16; i <= 18; i++)
@@ -214,24 +256,24 @@ public void testAppendWrongTerm(@SuppressWarnings("unused") Class> logClass) t
assert smb.counter() == 14; // resend_interval is big, so the commit index on B won't get updated
assertIndices(15, 15, 25, raft_a);
assertIndices(18, 14, 25, raft_b);
- assertCommitTableIndeces(b, raft_a, 14, 18, 19);
+ assertCommitTableIndeces(address(1), raft_a, 14, 18, 19);
// send a correct index, but incorrect prev_term:
int incorrect_prev_term=24;
long commit_index=raft_a.commitIndex(), prev_index=18;
- raft_a.setLeaderAndTerm(a, 30);
+ raft_a.setLeaderAndTerm(address(0), 30);
sendAppendEntriesRequest(raft_a, prev_index, incorrect_prev_term, 30, commit_index);
- assertCommitTableIndeces(b, raft_a, 14, 14, 15);
+ assertCommitTableIndeces(address(1), raft_a, 14, 14, 15);
// now apply the updates on the leader
for(int i=16; i <= 18; i++)
addAsync(raft_a,1, Options.create(false));
- raft_a.flushCommitTable(b); // first time: get correct last_appended (18)
- raft_a.flushCommitTable(b); // second time: send missing messages up to and including last_appended (18)
+ raft_a.flushCommitTable(address(1)); // first time: get correct last_appended (18)
+ raft_a.flushCommitTable(address(1)); // second time: send missing messages up to and including last_appended (18)
// assertCommitTableIndeces(b, raft_a, 17, 18, 19);
raft_a.flushCommitTable();
- assertCommitTableIndeces(b, raft_a, 18, 18, 19);
+ assertCommitTableIndeces(address(1), raft_a, 18, 18, 19);
// compare the log entries from 1-18
for(int i=0; i <= raft_a.lastAppended(); i++) {
@@ -248,24 +290,29 @@ public void testAppendWrongTerm(@SuppressWarnings("unused") Class> logClass) t
/** Tests appends where we change prev_term, so that we'll get an AppendResult with success=false */
public void testIncorrectAppend(@SuppressWarnings("unused") Class> logClass) throws Exception {
+ CounterStateMachine sma = stateMachine(0);
+ CounterStateMachine smb = stateMachine(1);
int prev_value=add(1);
assert prev_value == 0;
assert sma.counter() == 1;
assert smb.counter() == 0; // resend_interval is big, so the commit index on B won't get updated
+
+ RAFT raft_a = raft(0);
+ RAFT raft_b = raft(1);
assertIndices(1, 1, 5, raft_a);
assertIndices(1, 0, 5, raft_b);
- assertCommitTableIndeces(b, raft_a, 0, 1, 2);
+ assertCommitTableIndeces(address(1), raft_a, 0, 1, 2);
- raft_a.setLeaderAndTerm(a, 7);
+ raft_a.setLeaderAndTerm(address(0), 7);
prev_value=add(1);
assert prev_value == 1;
prev_value=add(1);
assert prev_value == 2;
assert sma.counter() == 3;
assert smb.counter() == 2; // previous value; B is always lagging one commit behind
- assertCommitTableIndeces(b, raft_a, 2, 3, 4);
+ assertCommitTableIndeces(address(1), raft_a, 2, 3, 4);
- raft_a.setLeaderAndTerm(a, 9);
+ raft_a.setLeaderAndTerm(address(0), 9);
for(int i=1; i <= 3; i++)
add(1);
@@ -279,17 +326,17 @@ public void testIncorrectAppend(@SuppressWarnings("unused") Class> logClass) t
// 7
LogEntries entries=createLogEntries(10, val);
- AppendResult result=impl.handleAppendEntriesRequest(entries, a, index - 1, 9, 10, 1);
+ AppendResult result=impl.handleAppendEntriesRequest(entries, address(0), index - 1, 9, 10, 1);
assert result.success();
raft_b.currentTerm(10);
index++;
// 8
- result=impl.handleAppendEntriesRequest(entries, a, index-1, 10, 10, 1);
+ result=impl.handleAppendEntriesRequest(entries, address(0), index-1, 10, 10, 1);
assert result.success();
assertIndices(8, 5, 10, raft_b);
- raft_a.setLeaderAndTerm(a, 11);
+ raft_a.setLeaderAndTerm(address(0), 11);
for(int i=1; i <= 2; i++)
add(-1);
@@ -298,20 +345,24 @@ public void testIncorrectAppend(@SuppressWarnings("unused") Class> logClass) t
assert smb.counter() == 5; // resend_interval is big, so the commit index on B won't get updated
assertIndices(8, 8, 11, raft_a);
assertIndices(8, 7, 11, raft_b);
- assertCommitTableIndeces(b, raft_a, 7, 8, 9);
+ assertCommitTableIndeces(address(1), raft_a, 7, 8, 9);
}
/** Tests appending with correct prev_index, but incorrect term */
public void testAppendWrongTermOnlyOneTerm(@SuppressWarnings("unused") Class> logClass) throws Exception {
- raft_a.setLeaderAndTerm(a, 22);
+ RAFT raft_a = raft(0);
+ raft_a.setLeaderAndTerm(address(0), 22);
for(int i=1; i <= 5; i++)
add(1);
+
+ CounterStateMachine sma = stateMachine(0);
+ CounterStateMachine smb = stateMachine(1);
expect(5, sma.counter());
expect(4, smb.counter()); // resend_interval is big, so the commit index on B won't get updated
assertIndices(5, 5, 22, raft_a);
- assertIndices(5, 4, 22, raft_b);
- assertCommitTableIndeces(b, raft_a, 4, 5, 6);
+ assertIndices(5, 4, 22, raft(1));
+ assertCommitTableIndeces(address(1), raft_a, 4, 5, 6);
// now append beyond the end
byte[] val=new byte[Integer.BYTES];
@@ -325,24 +376,29 @@ public void testAppendWrongTermOnlyOneTerm(@SuppressWarnings("unused") Class>
5, 23, 25, 0))
.setFlag(Message.TransientFlag.DONT_LOOPBACK); // don't receive my own request
raft_a.getDownProtocol().down(msg);
- raft_a.flushCommitTable(b);
+ raft_a.flushCommitTable(address(1));
expect(5, sma.counter());
expect(5, smb.counter()); // resend_interval is big, so the commit index on B won't get updated
assertIndices(5, 5, 22, raft_a);
- assertIndices(5, 5, 22, raft_b);
- assertCommitTableIndeces(b, raft_a, 5, 5, 6);
+ assertIndices(5, 5, 22, raft(1));
+ assertCommitTableIndeces(address(1), raft_a, 5, 5, 6);
}
public void testSnapshot(@SuppressWarnings("unused") Class> logClass) throws Exception {
+ RAFT raft_b = raft(1);
raft_b.maxLogSize(100);
for(int i=1; i <= 100; i++) {
add(i);
if(raft_b.numSnapshots() > 0)
break;
}
+
+ CounterStateMachine sma = stateMachine(0);
+ CounterStateMachine smb = stateMachine(1);
expect(325, sma.counter());
expect(300, smb.counter());
+ RaftNode node_b = node(1);
node_b.stop();
((CounterStateMachine)raft_b.stateMachine()).reset();
raft_b.stateMachineLoaded(false);
@@ -351,28 +407,34 @@ public void testSnapshot(@SuppressWarnings("unused") Class> logClass) throws E
expect(325, sma.counter());
expect(300, smb.counter());
- raft_a.flushCommitTable(b);
+ raft(0).flushCommitTable(address(1));
expect(325, sma.counter());
expect(325, smb.counter());
}
/** Tests adding C to cluster A,B, transfer of state from A to C */
public void testAddThirdMember(@SuppressWarnings("unused") Class> logClass) throws Exception {
- raft_a.setLeaderAndTerm(a, 20);
+ RAFT raft_a = raft(0);
+ raft_a.setLeaderAndTerm(address(0), 20);
+
+ RAFT raft_b = raft(1);
for(int i=1; i <= 5; i++)
add(i);
+
+ CounterStateMachine sma = stateMachine(0);
+ CounterStateMachine smb = stateMachine(1);
expect(15, sma.counter());
expect(10, smb.counter());
assertIndices(5, 5, 20, raft_a);
assertIndices(5, 4, 20, raft_b);
- assertCommitTableIndeces(b, raft_a, 4, 5, 6);
+ assertCommitTableIndeces(address(1), raft_a, 4, 5, 6);
addMemberC();
- raft_a.flushCommitTable(b);
- raft_a.flushCommitTable(c); // sets C's next-index to 1
- raft_a.flushCommitTable(c); // sends single append, next-index = 2
- raft_a.flushCommitTable(c); // sends messages/commit 2-5
- expect(15, smc.counter());
+ raft_a.flushCommitTable(address(1));
+ raft_a.flushCommitTable(address(2)); // sets C's next-index to 1
+ raft_a.flushCommitTable(address(2)); // sends single append, next-index = 2
+ raft_a.flushCommitTable(address(2)); // sends messages/commit 2-5
+ expect(15, stateMachine(2).counter());
}
@@ -380,20 +442,28 @@ public void testAddThirdMember(@SuppressWarnings("unused") Class> logClass) th
* resend messages missed by C and cannot find them; therefore a snapshot is sent from A -> C
*/
public void testSnapshotOnLeader(@SuppressWarnings("unused") Class> logClass) throws Exception {
- raft_a.setLeaderAndTerm(a, 20);
+ RAFT raft_a = raft(0);
+ raft_a.setLeaderAndTerm(address(0), 20);
addMemberC();
for(int i=1; i <= 5; i++)
add(i);
+
+ CounterStateMachine sma = stateMachine(0);
+ CounterStateMachine smb = stateMachine(1);
+ CounterStateMachine smc = stateMachine(2);
expect(15, sma.counter());
expect(10, smb.counter());
expect(10, smc.counter());
assertIndices(5, 5, 20, raft_a);
+
+ RAFT raft_b = raft(1);
+ RAFT raft_c = raft(2);
assertIndices(5, 4, 20, raft_b);
assertIndices(5, 4, 20, raft_c);
- assertCommitTableIndeces(b, raft_a, 4, 5, 6);
- assertCommitTableIndeces(c, raft_a, 4, 5, 6);
+ assertCommitTableIndeces(address(1), raft_a, 4, 5, 6);
+ assertCommitTableIndeces(address(2), raft_a, 4, 5, 6);
- cluster.dropTrafficTo(c);
+ cluster.dropTrafficTo(address(2));
for(int i=6; i <= 10; i++)
add(i);
expect(55, sma.counter());
@@ -402,35 +472,41 @@ public void testSnapshotOnLeader(@SuppressWarnings("unused") Class> logClass)
assertIndices(10, 10, 20, raft_a);
assertIndices(10, 9, 20, raft_b);
assertIndices(5, 4, 20, raft_c);
- assertCommitTableIndeces(b, raft_a, 9, 10, 11);
- assertCommitTableIndeces(c, raft_a, 4, 5, 6);
+ assertCommitTableIndeces(address(1), raft_a, 9, 10, 11);
+ assertCommitTableIndeces(address(2), raft_a, 4, 5, 6);
// now snapshot the leader at 10, and resume traffic
raft_a.snapshot();
cluster.clearDroppedTraffic();
raft_a.flushCommitTable();
- raft_a.flushCommitTable(c);
+ raft_a.flushCommitTable(address(2));
expect(55, sma.counter());
expect(55, smb.counter());
expect(55, smc.counter());
assertIndices(10, 10, 20, raft_a);
assertIndices(10, 10, 20, raft_b);
assertIndices(10, 10, 20, raft_c);
- assertCommitTableIndeces(b, raft_a, 10, 10, 11);
- assertCommitTableIndeces(c, raft_a, 10, 10, 11);
+ assertCommitTableIndeces(address(1), raft_a, 10, 10, 11);
+ assertCommitTableIndeces(address(2), raft_a, 10, 10, 11);
}
public void testSnapshotOnFollower(@SuppressWarnings("unused") Class> logClass) throws Exception {
- raft_a.setLeaderAndTerm(a, 20);
+ RAFT raft_a = raft(0);
+ raft_a.setLeaderAndTerm(address(0), 20);
for(int i=1; i <= 5; i++)
add(i);
+
+ CounterStateMachine sma = stateMachine(0);
+ CounterStateMachine smb = stateMachine(1);
expect(15, sma.counter());
expect(10, smb.counter());
assertIndices(5, 5, 20, raft_a);
+
+ RAFT raft_b = raft(1);
assertIndices(5, 4, 20, raft_b);
- assertCommitTableIndeces(b, raft_a, 4, 5, 6);
+ assertCommitTableIndeces(address(1), raft_a, 4, 5, 6);
raft_b.snapshot();
assertIndices(5, 4, 20, raft_b);
@@ -443,7 +519,7 @@ public void testSnapshotOnFollower(@SuppressWarnings("unused") Class> logClass
expect(10, smb.counter());
assertIndices(5, 5, 20, raft_a);
assertIndices(5, 4, 20, raft_b);
- assertCommitTableIndeces(b, raft_a, 4, 5, 6);
+ assertCommitTableIndeces(address(1), raft_a, 4, 5, 6);
for(int i=6; i <= 10; i++)
add(i);
@@ -451,26 +527,33 @@ public void testSnapshotOnFollower(@SuppressWarnings("unused") Class> logClass
expect(45, smb.counter());
assertIndices(10, 10, 20, raft_a);
assertIndices(10, 9, 20, raft_b);
- assertCommitTableIndeces(b, raft_a, 9, 10, 11);
+ assertCommitTableIndeces(address(1), raft_a, 9, 10, 11);
- raft_a.flushCommitTable(b);
+ raft_a.flushCommitTable(address(1));
expect(55, smb.counter());
assertIndices(10, 10, 20, raft_b);
- assertCommitTableIndeces(b, raft_a, 10, 10, 11);
+ assertCommitTableIndeces(address(1), raft_a, 10, 10, 11);
}
public void testSnapshotSentToFollower(@SuppressWarnings("unused") Class> logClass) throws Exception {
- raft_a.setLeaderAndTerm(a, 20);
+ RAFT raft_a = raft(0);
+ raft_a.setLeaderAndTerm(address(0), 20);
for(int i=1; i <= 5; i++)
add(i);
+
+ CounterStateMachine sma = stateMachine(0);
+ CounterStateMachine smb = stateMachine(1);
expect(15, sma.counter());
expect(10, smb.counter());
assertIndices(5, 5, 20, raft_a);
+
+ RAFT raft_b = raft(1);
assertIndices(5, 4, 20, raft_b);
- assertCommitTableIndeces(b, raft_a, 4, 5, 6);
+ assertCommitTableIndeces(address(1), raft_a, 4, 5, 6);
+ RaftNode node_b = node(1);
raft_b.stop();
- Utils.deleteLog(raft_b);
+ RaftTestUtils.deleteRaftLog(raft_b);
raft_b.log(null); // required to re-initialize the log
((CounterStateMachine)raft_b.stateMachine()).reset();
raft_b.stateMachineLoaded(false);
@@ -478,21 +561,21 @@ public void testSnapshotSentToFollower(@SuppressWarnings("unused") Class> logC
raft_a.snapshot();
assertIndices(5, 5, 20, raft_a);
- view=View.create(a, view.getViewId().getId()+1, a);
+ view=createView(view.getViewId().getId()+1, 0);
cluster.handleView(view);
- cluster.add(b, node_b);
+ cluster.add(raft_b.getAddress(), node_b);
raft_b.start();
- view=View.create(a, view.getViewId().getId()+1, a,b);
+ view=createView(view.getViewId().getId()+1, 0, 1);
cluster.handleView(view);
- raft_a.flushCommitTable(b); // first flush will set next-index to 1
- raft_a.flushCommitTable(b); // this flush will send the snapshot from A to B
+ raft_a.flushCommitTable(address(1)); // first flush will set next-index to 1
+ raft_a.flushCommitTable(address(1)); // this flush will send the snapshot from A to B
expect(15, sma.counter());
expect(15, smb.counter());
assertIndices(5, 5, 20, raft_a);
assertIndices(5, 5, 20, raft_b);
- assertCommitTableIndeces(b, raft_a, 5, 5, 6);
+ assertCommitTableIndeces(address(1), raft_a, 5, 5, 6);
for(int i=6; i <= 10; i++)
add(i);
@@ -500,14 +583,18 @@ public void testSnapshotSentToFollower(@SuppressWarnings("unused") Class> logC
expect(45, smb.counter());
assertIndices(10, 10, 20, raft_a);
assertIndices(10, 9, 20, raft_b);
- assertCommitTableIndeces(b, raft_a, 9, 10, 11);
+ assertCommitTableIndeces(address(1), raft_a, 9, 10, 11);
}
public void testIgnoreResponse(@SuppressWarnings("unused") Class> logClass) throws Exception {
+ RAFT raft_a = raft(0);
CompletableFuture f=addAsync(raft_a, 5, Options.create(false));
assert f != null;
int prev_value=Bits.readInt(f.get(), 0);
assert prev_value == 0;
+
+ CounterStateMachine sma = stateMachine(0);
+ CounterStateMachine smb = stateMachine(1);
assert sma.counter() == 5;
assert smb.counter() == 0;
@@ -526,16 +613,9 @@ public void testIgnoreResponse(@SuppressWarnings("unused") Class> logClass) th
assert smb.counter() == 10;
}
-
- protected static RAFT createRAFT(Address addr, String name, List members) {
- return new RAFT().raftId(name).members(members).logPrefix("synctest-" + name)
- .resendInterval(600_000) // long to disable resending by default
- .synchronous(true).setAddress(addr);
- }
-
- protected static Address createAddress(String name) {
- ExtendedUUID.setPrintFunction(RAFT.print_function);
- return ExtendedUUID.randomUUID(name).put(RAFT.raft_id_key, Util.stringToBytes(name));
+ @Override
+ protected RaftCluster createNewMockCluster() {
+ return new RaftCluster();
}
protected static byte[] num(int n) {
@@ -546,7 +626,7 @@ protected static byte[] num(int n) {
protected int add(int delta) throws Exception {
byte[] buf=num(delta);
- CompletableFuture f=raft_a.setAsync(buf, 0, buf.length);
+ CompletableFuture f=raft(0).setAsync(buf, 0, buf.length);
byte[] retval=f.get(10, TimeUnit.SECONDS);
return Bits.readInt(retval, 0);
}
@@ -591,10 +671,6 @@ protected static LogEntries createLogEntries(long curr_term, byte[] buf) {
return new LogEntries().add(new LogEntry(curr_term, buf));
}
- protected static LogEntries createLogEntries(int term, byte[] buf, int off, int len) {
- return new LogEntries().add(new LogEntry(term, buf, off, len));
- }
-
protected static void assertCommitTableIndeces(Address member, RAFT r, int commit_index, int match_index, int next_index) {
CommitTable table=r.commitTable();
assert table != null;
@@ -612,13 +688,14 @@ protected static void expect(int expected_value, int actual_value) {
}
protected void addMemberC() throws Exception {
- raft_c=createRAFT(c, "C", mbrs).stateMachine(smc=new CounterStateMachine()).setLeaderAndTerm(a).logClass(logClass.getCanonicalName());
- node_c=new RaftNode(cluster, raft_c);
- node_c.init();
- node_c.start();
- cluster.add(c, node_c);
- view=View.create(a, 2, a,b,c);
+ createCluster();
+ view = createView(2, 0, 1, 2);
cluster.handleView(view);
}
+ private CounterStateMachine stateMachine(int index) {
+ RAFT r = raft(index);
+ assertThat(r).isNotNull();
+ return (CounterStateMachine) r.stateMachine();
+ }
}
diff --git a/tests/junit-functional/org/jgroups/tests/TimeoutTest.java b/tests/junit-functional/org/jgroups/tests/TimeoutTest.java
index c0b0ad1a..6490ef6f 100644
--- a/tests/junit-functional/org/jgroups/tests/TimeoutTest.java
+++ b/tests/junit-functional/org/jgroups/tests/TimeoutTest.java
@@ -2,36 +2,33 @@
import org.jgroups.Global;
import org.jgroups.JChannel;
-import org.jgroups.protocols.raft.ELECTION;
import org.jgroups.protocols.raft.RAFT;
-import org.jgroups.protocols.raft.REDIRECT;
import org.jgroups.raft.blocks.ReplicatedStateMachine;
-import org.jgroups.raft.util.Utils;
-import org.jgroups.stack.Protocol;
-import org.jgroups.util.Util;
-import org.testng.annotations.AfterMethod;
-import org.testng.annotations.Test;
+import org.jgroups.tests.harness.BaseStateMachineTest;
import java.util.Arrays;
-import java.util.List;
-import java.util.function.Predicate;
-import java.util.stream.Collectors;
+import java.util.concurrent.TimeUnit;
+import java.util.function.BooleanSupplier;
import java.util.stream.IntStream;
-import java.util.stream.Stream;
+
+import org.testng.annotations.AfterMethod;
+import org.testng.annotations.Test;
+
+import static org.assertj.core.api.Assertions.assertThat;
+import static org.jgroups.raft.testfwk.RaftTestUtils.eventually;
@Test(groups=Global.FUNCTIONAL,singleThreaded=true)
-public class TimeoutTest {
+public class TimeoutTest extends BaseStateMachineTest> {
protected static final int NUM=200;
- protected JChannel[] channels;
- protected ReplicatedStateMachine[] rsms;
+ {
+ createManually = true;
+ recreatePerMethod = true;
+ }
- @AfterMethod protected void destroy() throws Exception {
- Util.close(channels);
- for(JChannel ch: channels) {
- RAFT raft=ch.getProtocolStack().findProtocol(RAFT.class);
- Utils.deleteLog(raft);
- }
+ @AfterMethod(alwaysRun = true)
+ protected void destroy() throws Exception {
+ destroyCluster();
}
public void testAppendWithSingleNode() throws Exception {
@@ -43,22 +40,23 @@ public void testAppendWith3Nodes() throws Exception {
}
protected void _test(int num) throws Exception {
- channels=createChannels(num);
- rsms=createReplicatedStateMachines(channels);
- for(JChannel ch: channels)
- ch.connect("rsm-cluster");
- Util.waitUntilAllChannelsHaveSameView(10000, 500, channels);
-
- Util.waitUntil(10000, 500,
- () -> Stream.of(channels)
- .map(ch -> ch.getProtocolStack().findProtocol(RAFT.class))
- .anyMatch(r -> ((RAFT)r).isLeader()));
+ withClusterSize(num);
+ createCluster();
+
+ BooleanSupplier bs = () -> Arrays.stream(channels())
+ .map(this::raft)
+ .anyMatch(RAFT::isLeader);
+ assertThat(eventually(bs, 10, TimeUnit.SECONDS))
+ .as("Leader election")
+ .isTrue();
+
ReplicatedStateMachine sm=null;
System.out.println("-- waiting for leader");
- for(int i=0; i < channels.length; i++) {
- RAFT raft=channels[i].getProtocolStack().findProtocol(RAFT.class);
+ for(int i=0; i < channels().length; i++) {
+ RAFT raft=raft(i);
+ assertThat(raft).isNotNull();
if(raft.isLeader()) {
- sm=rsms[i];
+ sm=stateMachine(i);
System.out.printf("-- found leader: %s\n", raft.leader());
break;
}
@@ -77,57 +75,32 @@ protected void _test(int num) throws Exception {
long start=System.currentTimeMillis();
sm.allowDirtyReads(false);
assert sm.get(NUM) == NUM;
- Predicate> converged= r -> {
- try {
- Integer o = r.get(NUM);
- return o != null && o == NUM;
- } catch (Throwable t) {
- throw new AssertionError("Failed with: " + r.raftId(), t);
- }
- };
+
// After reading correctly from the leader with a quorum read, every node should have the same state.
- Util.waitUntil(5_000, 250, () -> Arrays.stream(rsms).allMatch(converged));
+ // We still have to use eventually so the message propagate to ALL nodes, not only majority.
+ assertStateMachineEventuallyMatch(IntStream.range(0, num).toArray());
long time=System.currentTimeMillis()-start;
- System.out.printf("-- it took %d member(s) %d ms to get consistent caches\n", rsms.length, time);
-
- System.out.printf("-- contents:\n%s\n\n",
- Stream.of(rsms).map(r -> String.format("%s: %s", r.channel().getName(), r))
- .collect(Collectors.joining("\n")));
+ System.out.printf("-- it took %d member(s) %d ms to get consistent caches\n", clusterSize, time);
System.out.print("-- verifying contents of state machines:\n");
- for(ReplicatedStateMachine rsm: rsms) {
+ for (int i = 0; i < clusterSize; i++) {
+ ReplicatedStateMachine rsm = stateMachine(i);
System.out.printf("%s: ", rsm.channel().getName());
- for(int i=1; i <= NUM; i++)
- assert rsm.get(i) == i;
+ for(int j=1; j <= NUM; j++)
+ assert rsm.get(j) == j;
System.out.println("OK");
}
}
- protected static JChannel createRaftChannel(List members, String name) throws Exception {
- Protocol[] protocols=Util.getTestStack(
- new ELECTION(),
- new RAFT().members(members).raftId(name).resendInterval(1000).logClass("org.jgroups.protocols.raft.InMemoryLog"),
- new REDIRECT());
- return new JChannel(protocols).name(name);
+ @Override
+ protected void amendRAFTConfiguration(RAFT raft) {
+ raft.resendInterval(1_000);
}
- protected static JChannel[] createChannels(int num) throws Exception {
- List members=IntStream.range(0, num)
- .mapToObj(n -> String.valueOf((char)('A' + n))).collect(Collectors.toList());
-
- JChannel[] ret=new JChannel[num];
- for(int i=0; i < num; i++) {
- ret[i]=createRaftChannel(members, String.valueOf((char)('A' + i)));
- }
- return ret;
- }
-
- protected static ReplicatedStateMachine[] createReplicatedStateMachines(JChannel[] chs) {
- ReplicatedStateMachine[] ret=new ReplicatedStateMachine[chs.length];
- for(int i=0; i < ret.length; i++) {
- ret[i]=new ReplicatedStateMachine<>(chs[i]);
- ret[i].timeout(2000).allowDirtyReads(true);
- }
- return ret;
+ @Override
+ protected ReplicatedStateMachine createStateMachine(JChannel ch) {
+ ReplicatedStateMachine rsm = new ReplicatedStateMachine<>(ch);
+ rsm.timeout(2_000).allowDirtyReads(true);
+ return rsm;
}
}
diff --git a/tests/junit-functional/org/jgroups/tests/VoteTest.java b/tests/junit-functional/org/jgroups/tests/VoteTest.java
index 466b3ae6..3bdef2f3 100644
--- a/tests/junit-functional/org/jgroups/tests/VoteTest.java
+++ b/tests/junit-functional/org/jgroups/tests/VoteTest.java
@@ -3,24 +3,24 @@
import org.jgroups.Address;
import org.jgroups.Global;
import org.jgroups.JChannel;
-import org.jgroups.protocols.DISCARD;
-import org.jgroups.protocols.TP;
import org.jgroups.protocols.raft.RAFT;
-import org.jgroups.protocols.raft.REDIRECT;
-import org.jgroups.raft.util.Utils;
-import org.jgroups.stack.ProtocolStack;
-import org.jgroups.tests.election.BaseElectionTest;
+import org.jgroups.tests.harness.BaseRaftElectionTest;
+import org.jgroups.tests.harness.RaftAssertion;
import org.jgroups.util.Util;
-import org.testng.annotations.AfterMethod;
-import org.testng.annotations.Test;
-import java.util.Arrays;
-import java.util.Collections;
-import java.util.List;
import java.util.concurrent.TimeUnit;
+import java.util.function.BooleanSupplier;
+import java.util.function.Supplier;
+import java.util.stream.Collectors;
import java.util.stream.Stream;
-import static org.jgroups.tests.election.BaseElectionTest.ALL_ELECTION_CLASSES_PROVIDER;
+import org.assertj.core.api.Assertions;
+import org.testng.annotations.AfterMethod;
+import org.testng.annotations.Test;
+
+import static org.assertj.core.api.Assertions.assertThat;
+import static org.jgroups.raft.testfwk.RaftTestUtils.eventually;
+import static org.jgroups.tests.harness.BaseRaftElectionTest.ALL_ELECTION_CLASSES_PROVIDER;
/**
* Tests that a member cannot vote twice. Issue: https://github.com/belaban/jgroups-raft/issues/24
@@ -28,30 +28,26 @@
* @since 0.2
*/
@Test(groups=Global.FUNCTIONAL,singleThreaded=true, dataProvider = ALL_ELECTION_CLASSES_PROVIDER)
-public class VoteTest extends BaseElectionTest {
- protected JChannel[] channels;
- protected RAFT[] rafts;
- protected static final String CLUSTER=VoteTest.class.getSimpleName();
-
- @AfterMethod protected void destroy() {
- if(channels != null)
- close(channels);
+public class VoteTest extends BaseRaftElectionTest.ChannelBased {
+
+ {
+ createManually = true;
+ }
+
+ @AfterMethod
+ protected void destroy() throws Exception {
+ destroyCluster();
}
/** Start a member not in {A,B,C} -> expects an exception */
public void testStartOfNonMember(Class> ignore) throws Exception {
- JChannel non_member=null;
- try {
- non_member=create("X", Arrays.asList("A", "B"));
- assert false : "Starting a non-member should throw an exception";
- }
- catch(Exception e) {
- System.out.println("received exception as expected: " + e);
- }
- finally {
- close(non_member);
- }
+ withClusterSize(2);
+ JChannel non_member=createDisconnectedChannel("X");
+ Assertions.assertThatThrownBy(() -> non_member.connect(clusterName()))
+ .isInstanceOf(IllegalStateException.class)
+ .hasMessageStartingWith("raft-id X is not listed in members");
+ close(non_member);
}
@@ -63,255 +59,126 @@ public void testStartOfNonMember(Class> ignore) throws Exception {
*/
@Deprecated
public void testMemberVotesTwice(Class> ignore) throws Exception {
- init("A", "B", "C", "D");
- Util.waitUntilAllChannelsHaveSameView(10000, 500, channels);
-
- Util.close(channels[2], channels[3]); // close C and D
- Util.waitUntilAllChannelsHaveSameView(10000, 500, channels[0], channels[1]); // A and B: {A,B}
+ withClusterSize(4);
+ createCluster();
- RAFT raft=channels[0].getProtocolStack().findProtocol(RAFT.class);
+ // close C and D
+ close(2);
+ close(3);
- try {
- raft.set(new byte[]{'b', 'e', 'l', 'a'}, 0, 4, 500, TimeUnit.MILLISECONDS);
- assert false : "the change should have failed as we don't have a majority of 3 to commit it";
- }
- catch(IllegalStateException ex) {
- System.out.println("Caught an exception as expected, trying to commit a change: " + ex);
- }
+ // A and B: {A,B}
+ Util.waitUntilAllChannelsHaveSameView(10000, 500, channel(0), channel(1));
+ RaftAssertion.assertLeaderlessOperationThrows(() -> raft(0).set(new byte[]{'b', 'e', 'l', 'a'}, 0, 4, 500, TimeUnit.MILLISECONDS));
// close B and create a new B'
- System.out.printf("restarting %s\n", channels[1].name());
- Util.close(channels[1]);
-
- channels[1]=create("B", Arrays.asList("A", "B", "C", "D"));
- Util.waitUntilAllChannelsHaveSameView(10000, 500, channels[0], channels[1]);
+ System.out.printf("restarting %s\n", channel(1).name());
+ close(1);
+ createCluster(1);
// Try the change again: we have votes from A and B from before the non-leader was restarted. Now B was
// restarted, but it cannot vote again in the same term, so we still only have 2 votes!
- try {
- raft.set(new byte[]{'b', 'e', 'l', 'a'}, 0, 4, 500, TimeUnit.MILLISECONDS);
- assert false : "the change should have failed as we don't have a majority of 3 to commit it";
- }
- catch(IllegalStateException ex) {
- System.out.println("Caught an exception as expected, trying to commit a change: " + ex);
- }
+ RaftAssertion.assertLeaderlessOperationThrows(() -> raft(0).set(new byte[]{'b', 'e', 'l', 'a'}, 0, 4, 500, TimeUnit.MILLISECONDS));
// now start C. as we have a majority now (A,B,C), the change should succeed
System.out.println("starting C");
- channels[2]=create("C", Arrays.asList("A", "B", "C", "D"));
- Util.waitUntilAllChannelsHaveSameView(10000, 500, channels[0], channels[1], channels[2]);
+ createCluster(1);
// wait until we have a leader (this may take a few ms)
- Util.waitUntil(10000, 500,
- () -> Stream.of(channels).filter(JChannel::isConnected)
- .anyMatch(c -> ((RAFT)c.getProtocolStack().findProtocol(RAFT.class)).isLeader()));
+ waitUntilLeaderElected(10_000, 0, 1, 2);
// need to set this again, as the leader might have changed
- raft=null;
- for(JChannel c: channels) {
- if(!c.isConnected())
- continue;
- RAFT r=c.getProtocolStack().findProtocol(RAFT.class);
- if(r.isLeader()) {
- raft=r;
- break;
- }
- }
- assert raft != null;
+ RAFT raft=leader();
+ assertThat(raft).isNotNull();
// This time, we should succeed
raft.set(new byte[]{'b', 'e', 'l', 'a'}, 0, 4, 500, TimeUnit.MILLISECONDS);
- Util.waitUntil(10000, 500, () -> Stream.of(channels).filter(JChannel::isConnected)
- .map(c -> (RAFT)c.getProtocolStack().findProtocol(RAFT.class))
- .allMatch(r -> r.commitIndex() == 1 && r.lastAppended() == 1));
- for(JChannel ch: channels) {
- if(!ch.isConnected())
- continue;
- RAFT r=ch.getProtocolStack().findProtocol(RAFT.class);
- System.out.printf("%s: append-index=%d, commit-index=%d\n", ch.getAddress(), r.lastAppended(), r.commitIndex());
- }
+ BooleanSupplier bs = () -> Stream.of(actualChannels())
+ .filter(JChannel::isConnected)
+ .map(this::raft)
+ .allMatch(r -> r.commitIndex() == 1 && r.lastAppended() == 1);
+ Supplier message = () -> Stream.of(actualChannels())
+ .map(this::raft)
+ .map(r -> String.format("%s: append-index=%d, commit-index=%d\n", r.getAddress(), r.lastAppended(), r.commitIndex()))
+ .collect(Collectors.joining(System.lineSeparator()));
+
+ assertThat(eventually(bs, 10, TimeUnit.SECONDS)).as(message).isTrue();
}
/** Membership=A, member=A: should become leader immediately */
public void testSingleMember(Class> ignore) throws Exception {
- channels=new JChannel[]{create("A", Collections.singletonList("A"))};
- rafts=new RAFT[]{raft(channels[0])};
- Address leader=leader(10000, 500, channels);
+ withClusterSize(1);
+ createCluster();
+
+ waitUntilLeaderElected(10_000, 0);
+ Address leader=leaderAddress();
System.out.println("leader = " + leader);
- assert leader != null;
- assert leader.equals(channels[0].getAddress());
+ assertThat(leader).isNotNull();
+ assertThat(leader).isEqualTo(channel(0).getAddress());
}
/** {A,B,C} with leader A. Then B and C leave: A needs to become Follower */
- public void testLeaderGoingBacktoFollower(Class> ignore) throws Exception {
- init("A", "B", "C");
- Util.waitUntilAllChannelsHaveSameView(10000, 500, channels);
- JChannel leader_ch=getLeader(10000, 500, channels);
- RAFT raft=raft(leader_ch);
- System.out.printf("leader is %s\n", leader_ch);
+ public void testLeaderGoingBackToFollower(Class> ignore) throws Exception {
+ withClusterSize(3);
+ createCluster();
+
+ waitUntilLeaderElected(5_000, 0, 1, 2);
+ RAFT raft=leader();
+ assertThat(raft).isNotNull();
+
+ System.out.printf("leader is %s\n", raft.raftId());
System.out.println("closing non-leaders:");
- // Stream.of(channels).filter(c -> !c.getAddress().equals(leader_ch.getAddress())).forEach(JChannel::close);
- for(JChannel ch: channels) {
- if(ch.getAddress().equals(leader_ch.getAddress()))
+ JChannel[] channels = channels();
+ for (int i = 0; i < channels.length; i++) {
+ JChannel ch = channels[i];
+ if(ch.getAddress().equals(raft.getAddress()))
continue;
- Util.close(ch);
+ close(i);
}
Util.waitUntil(5000, 500, () -> !raft.isLeader());
- assert raft.leader() == null;
+ assertThat(raft.leader()).isNull();
}
/** {A,B,C,D}: A is leader and A, B, C and D have leader=A. When C and D are closed, both A, B and C must
* have leader set to null, as there is no majority (3) any longer */
public void testNullLeader(Class> ignore) throws Exception {
- init("A", "B", "C", "D");
- Util.waitUntilAllChannelsHaveSameView(10000, 500, channels);
+ withClusterSize(4);
+ createCluster();
// assert we have a leader
- Util.waitUntil(10000, 500,
- () -> Stream.of(channels)
- .map(c -> (RAFT)c.getProtocolStack().findProtocol(RAFT.class))
- .allMatch((RAFT r) -> r.leader() != null));
- Util.close(channels[2], channels[3]); // close C and D, now everybody should have a null leader
- Util.waitUntilAllChannelsHaveSameView(10_000, 500, channels[0], channels[1]);
- Util.waitUntil(10000, 500,
- () -> Stream.of(channels).filter(JChannel::isConnected)
- .map(VoteTest::raft)
- .allMatch((RAFT r) -> r.leader() == null),
- this::printLeaders);
+ waitUntilLeaderElected(10_000, 0, 1, 2, 3);
+
+ // close C and D, now everybody should have a null leader
+ close(3);
+ close(2);
+
+ Util.waitUntilAllChannelsHaveSameView(10_000, 250, channel(0), channel(1));
+ BooleanSupplier bs = () -> Stream.of(actualChannels())
+ .filter(JChannel::isConnected)
+ .map(this::raft)
+ .allMatch((RAFT r) -> r.leader() == null);
+ assertThat(eventually(bs, 10, TimeUnit.SECONDS)).as(this::printLeaders).isTrue();
System.out.printf("channels:\n%s", printLeaders());
}
-
- protected void init(String ... nodes) throws Exception {
- channels=new JChannel[nodes.length];
- rafts=new RAFT[nodes.length];
- for(int i=0; i < nodes.length; i++) {
- channels[i]=create(nodes[i], Arrays.asList(nodes));
- rafts[i]=raft(channels[i]);
- }
- }
-
protected String printLeaders() {
StringBuilder sb=new StringBuilder("\n");
- for(JChannel ch: channels) {
+ for(JChannel ch: actualChannels()) {
if(!ch.isConnected())
sb.append(String.format("%s: not connected\n", ch.getName()));
else {
- RAFT raft=ch.getProtocolStack().findProtocol(RAFT.class);
+ RAFT raft=raft(ch);
sb.append(String.format("%s: leader=%s\n", ch.getName(), raft.leader()));
}
}
return sb.toString();
}
-
- protected JChannel create(String name, List mbrs) throws Exception {
- RAFT raft=new RAFT().members(mbrs).raftId(name).stateMachine(new DummyStateMachine())
- .logClass("org.jgroups.protocols.raft.InMemoryLog").logPrefix(name + "-" + CLUSTER);
- JChannel ch=new JChannel(Util.getTestStack(instantiate(), raft, new REDIRECT())).name(name);
- ch.connect(CLUSTER);
- return ch;
- }
-
-
- protected static Address leader(long timeout, long interval, JChannel ... channels) {
- long target_time=System.currentTimeMillis() + timeout;
- while(System.currentTimeMillis() <= target_time) {
- for(JChannel ch : channels) {
- if(ch.isConnected() && raft(ch).isLeader())
- return raft(ch).leader();
- }
- Util.sleep(interval);
- }
- return null;
- }
-
- protected static JChannel getLeader(long timeout, long interval, JChannel ... channels) {
- long target_time=System.currentTimeMillis() + timeout;
- while(System.currentTimeMillis() <= target_time) {
- for(JChannel ch : channels) {
- if(ch.isConnected() && raft(ch).isLeader())
- return ch;
- }
- Util.sleep(interval);
- }
- return null;
- }
-
-
- protected static JChannel nonLeader(JChannel ... channels) {
- return Stream.of(channels).filter(c -> c.isConnected() && !raft(c).leader().equals(c.getAddress()))
- .filter(c -> c.getProtocolStack().findProtocol(DISCARD.class) == null)
- .findFirst().orElse(null);
- }
-
- protected static void discardAll(JChannel ... channels) throws Exception {
- for(JChannel ch: channels)
- ch.getProtocolStack().insertProtocol(new DISCARD().discardAll(true), ProtocolStack.Position.ABOVE, TP.class);
- }
-
- protected static void assertSameLeader(Address leader, JChannel... channels) {
- for(JChannel ch: channels)
- assert leader.equals(raft(ch).leader());
- }
-
-
-
- protected static void assertCommitIndex(long timeout, long interval, int expected_commit, JChannel... channels) {
- long target_time=System.currentTimeMillis() + timeout;
- while(System.currentTimeMillis() <= target_time) {
- boolean all_ok=true;
- for(JChannel ch: channels) {
- RAFT raft=raft(ch);
- if(expected_commit != raft.commitIndex())
- all_ok=false;
- }
- if(all_ok)
- break;
- Util.sleep(interval);
- }
- for(JChannel ch: channels) {
- RAFT raft=raft(ch);
- System.out.printf("%s: members=%s, last-applied=%d, commit-index=%d\n", ch.getAddress(), raft.members(),
- raft.lastAppended(), raft.commitIndex());
- assert raft.commitIndex() == expected_commit : String.format("%s: last-applied=%d, commit-index=%d",
- ch.getAddress(), raft.lastAppended(), raft.commitIndex());
- }
- }
-
protected RAFT raft(Address addr) {
return raft(channel(addr));
}
- protected JChannel channel(Address addr) {
- for(JChannel ch: channels) {
- if(ch.getAddress() != null && ch.getAddress().equals(addr))
- return ch;
- }
- return null;
- }
-
- protected static RAFT raft(JChannel ch) {
- return ch.getProtocolStack().findProtocol(RAFT.class);
- }
-
- protected static void close(JChannel... channels) {
- for(JChannel ch: channels) {
- if(ch == null)
- continue;
- ProtocolStack stack=ch.getProtocolStack();
- stack.removeProtocol(DISCARD.class);
- RAFT raft=stack.findProtocol(RAFT.class);
- try {
- Utils.deleteLog(raft);
- }
- catch(Exception ignored) {}
- Util.close(ch);
- }
- }
-
}
diff --git a/tests/junit-functional/org/jgroups/tests/blocks/CounterTest.java b/tests/junit-functional/org/jgroups/tests/blocks/CounterTest.java
index 95326d89..58339840 100644
--- a/tests/junit-functional/org/jgroups/tests/blocks/CounterTest.java
+++ b/tests/junit-functional/org/jgroups/tests/blocks/CounterTest.java
@@ -1,9 +1,21 @@
package org.jgroups.tests.blocks;
-import static org.testng.AssertJUnit.assertEquals;
-import static org.testng.AssertJUnit.assertFalse;
-import static org.testng.AssertJUnit.assertNull;
-import static org.testng.AssertJUnit.assertTrue;
+import org.jgroups.Global;
+import org.jgroups.blocks.atomic.AsyncCounter;
+import org.jgroups.blocks.atomic.CounterFunction;
+import org.jgroups.blocks.atomic.CounterView;
+import org.jgroups.blocks.atomic.SyncCounter;
+import org.jgroups.protocols.raft.RAFT;
+import org.jgroups.raft.Options;
+import org.jgroups.raft.blocks.CounterService;
+import org.jgroups.raft.blocks.RaftAsyncCounter;
+import org.jgroups.raft.blocks.RaftSyncCounter;
+import org.jgroups.tests.harness.BaseRaftChannelTest;
+import org.jgroups.tests.harness.BaseRaftElectionTest;
+import org.jgroups.util.CompletableFutures;
+import org.jgroups.util.LongSizeStreamable;
+import org.jgroups.util.SizeStreamable;
+import org.jgroups.util.Util;
import java.io.DataInput;
import java.io.DataOutput;
@@ -14,74 +26,49 @@
import java.util.Objects;
import java.util.concurrent.CompletableFuture;
import java.util.concurrent.CompletionStage;
+import java.util.concurrent.TimeUnit;
import java.util.concurrent.atomic.AtomicInteger;
import java.util.concurrent.atomic.AtomicLong;
+import java.util.function.BooleanSupplier;
import java.util.function.Function;
+import java.util.function.Supplier;
import java.util.stream.Collectors;
import java.util.stream.Stream;
-import org.jgroups.Global;
-import org.jgroups.JChannel;
-import org.jgroups.blocks.atomic.AsyncCounter;
-import org.jgroups.blocks.atomic.CounterFunction;
-import org.jgroups.blocks.atomic.CounterView;
-import org.jgroups.blocks.atomic.SyncCounter;
-import org.jgroups.protocols.raft.ELECTION;
-import org.jgroups.protocols.raft.FileBasedLog;
-import org.jgroups.protocols.raft.RAFT;
-import org.jgroups.protocols.raft.REDIRECT;
-import org.jgroups.raft.Options;
-import org.jgroups.raft.blocks.CounterService;
-import org.jgroups.raft.blocks.RaftAsyncCounter;
-import org.jgroups.raft.blocks.RaftSyncCounter;
-import org.jgroups.raft.util.Utils;
-import org.jgroups.util.CompletableFutures;
-import org.jgroups.util.LongSizeStreamable;
-import org.jgroups.util.SizeStreamable;
-import org.jgroups.util.Util;
-import org.testng.annotations.AfterClass;
-import org.testng.annotations.BeforeClass;
import org.testng.annotations.Test;
+import static org.assertj.core.api.Assertions.assertThat;
+import static org.jgroups.raft.testfwk.RaftTestUtils.eventually;
+import static org.testng.AssertJUnit.assertEquals;
+import static org.testng.AssertJUnit.assertFalse;
+import static org.testng.AssertJUnit.assertNull;
+import static org.testng.AssertJUnit.assertTrue;
+
/**
* {@link AsyncCounter} and {@link SyncCounter} test.
*/
@Test(groups = Global.FUNCTIONAL, singleThreaded = true)
-public class CounterTest {
-
- private static final String CLUSTER = "_counter_test_";
+public class CounterTest extends BaseRaftChannelTest {
- protected JChannel a, b, c;
protected CounterService service_a, service_b, service_c;
- @AfterClass(alwaysRun = true)
- public void afterMethod() {
- for (JChannel ch : Arrays.asList(c, b, a)) {
- Util.close(ch);
- RAFT raft = ch.getProtocolStack().findProtocol(RAFT.class);
- try {
- Utils.deleteLog(raft);
- } catch (Exception ignored) {
- }
- }
+ {
+ clusterSize = 3;
}
- @BeforeClass(alwaysRun = true)
- public void init() throws Exception {
- List members = Arrays.asList("a", "b", "c");
-
- a = createChannel(0, members).connect(CLUSTER);
- b = createChannel(1, members).connect(CLUSTER);
- c = createChannel(2, members).connect(CLUSTER);
-
- Util.waitUntilAllChannelsHaveSameView(1000, 500, a, b, c);
+ @Override
+ protected void afterClusterCreation() {
+ service_a = new CounterService(channel(0)).allowDirtyReads(false);
+ service_b = new CounterService(channel(1)).allowDirtyReads(false);
+ service_c = new CounterService(channel(2)).allowDirtyReads(false);
- service_a = new CounterService(a).allowDirtyReads(false);
- service_b = new CounterService(b).allowDirtyReads(false);
- service_c = new CounterService(c).allowDirtyReads(false);
+ RAFT[] rafts = Arrays.stream(channels())
+ .map(this::raft)
+ .toArray(RAFT[]::new);
- Util.waitUntilTrue(10_000, 500, () -> Stream.of(a, b, c)
- .map(CounterTest::getRaft).allMatch(r -> r.leader() != null));
+ // Need to wait until ALL nodes have the leader.
+ // Otherwise, REDIRECT might fail because of not knowing the leader.
+ BaseRaftElectionTest.waitUntilAllHaveLeaderElected(rafts, 15_000);
}
public void testIncrement() {
@@ -359,16 +346,22 @@ public void testConcurrentCas() {
public void testDelete() throws Exception {
List counters = createAsyncCounters("to-delete");
for (AsyncCounter counter : counters) {
- System.out.println("-- name: " + counter.getName());
assertEquals(0, counter.sync().get());
}
assert counters.size() == 3 && counters.stream().allMatch(Objects::nonNull);
- for (CounterService service : Arrays.asList(service_a, service_b, service_c)) {
- String info = service.printCounters();
- assert info.contains("to-delete") : String.format("%s failed\n%s", service.raftId(), info);
- }
+ // We create the counter from each server, but the request is redirected to the leader.
+ // In some occasions, with 3 nodes, we can end up committing only on the same 2 of the three in all requests.
+ // When this happens, we retrieve the printCounters locally, without a consensus read, so the counter is not locally created yet.
+ BooleanSupplier bs = () -> Stream.of(service_a, service_b, service_c)
+ .allMatch(service -> service.printCounters().contains("to-delete"));
+ Supplier message = () -> Stream.of(service_a, service_b, service_c)
+ .map(service -> String.format("%s: %s", service.raftId(), service.printCounters()))
+ .collect(Collectors.joining(System.lineSeparator()));
+ assertThat(eventually(bs, 10, TimeUnit.SECONDS))
+ .as(message)
+ .isTrue();
// blocks until majority
service_a.deleteCounter("to-delete");
@@ -484,18 +477,6 @@ private static RaftAsyncCounter createCounter(String name, CounterService counte
return CompletableFutures.join(counterService.getOrCreateAsyncCounter(name, 0));
}
- private static JChannel createChannel(int id, final List members) throws Exception {
- String name = members.get(id);
- ELECTION election = new ELECTION();
- RAFT raft = new RAFT().members(members).raftId(members.get(id)).logClass(FileBasedLog.class.getCanonicalName()).logPrefix(name + "-" + CLUSTER);
- //noinspection resource
- return new JChannel(Util.getTestStack(election, raft, new REDIRECT())).name(name);
- }
-
- private static RAFT getRaft(JChannel ch) {
- return ch.getProtocolStack().findProtocol(RAFT.class);
- }
-
public static class GetAndAddFunction implements CounterFunction, SizeStreamable {
long delta;
diff --git a/tests/junit-functional/org/jgroups/tests/election/BaseElectionTest.java b/tests/junit-functional/org/jgroups/tests/election/BaseElectionTest.java
deleted file mode 100644
index 3622ee2d..00000000
--- a/tests/junit-functional/org/jgroups/tests/election/BaseElectionTest.java
+++ /dev/null
@@ -1,33 +0,0 @@
-package org.jgroups.tests.election;
-
-import org.jgroups.protocols.raft.ELECTION;
-import org.jgroups.protocols.raft.ELECTION2;
-import org.jgroups.protocols.raft.election.BaseElection;
-import org.testng.annotations.BeforeMethod;
-import org.testng.annotations.DataProvider;
-
-public class BaseElectionTest {
- public static final String ALL_ELECTION_CLASSES_PROVIDER = "all-election-classes";
-
- protected Class extends BaseElection> electionClass;
-
- public BaseElection instantiate() throws Exception {
- assert electionClass != null : "Election class not set";
- // The default constructor is always available.
- return electionClass.getDeclaredConstructor().newInstance();
- }
-
- @BeforeMethod(alwaysRun = true)
- @SuppressWarnings("unchecked")
- protected void setElectionClass(Object[] args) {
- electionClass = (Class extends BaseElection>) args[0];
- }
-
- @DataProvider(name = ALL_ELECTION_CLASSES_PROVIDER)
- protected static Object[][] electionClasses() {
- return new Object[][] {
- {ELECTION.class},
- {ELECTION2.class},
- };
- }
-}
diff --git a/tests/junit-functional/org/jgroups/tests/harness/AbstractRaftTest.java b/tests/junit-functional/org/jgroups/tests/harness/AbstractRaftTest.java
new file mode 100644
index 00000000..fc6352c4
--- /dev/null
+++ b/tests/junit-functional/org/jgroups/tests/harness/AbstractRaftTest.java
@@ -0,0 +1,413 @@
+package org.jgroups.tests.harness;
+
+import org.jgroups.protocols.raft.ELECTION;
+import org.jgroups.protocols.raft.InMemoryLog;
+import org.jgroups.protocols.raft.RAFT;
+import org.jgroups.protocols.raft.election.BaseElection;
+import org.jgroups.stack.Protocol;
+import org.jgroups.tests.DummyStateMachine;
+
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.List;
+import java.util.Set;
+
+import org.testng.ITestContext;
+import org.testng.annotations.AfterClass;
+import org.testng.annotations.AfterMethod;
+import org.testng.annotations.BeforeClass;
+import org.testng.annotations.BeforeMethod;
+
+/**
+ * The base class for the Raft tests.
+ *
+ * This class is abstract and does not enforce which type of cluster/communication is utilized between the nodes. It
+ * requires the implementation of the creation, destruction, and communication pieces instead. Therefore, nodes can
+ * utilize {@link org.jgroups.JChannel}, or {@link org.jgroups.raft.testfwk.MockRaftCluster} without implementing
+ * everything again. Some of the utilities provided:
+ *
+ *
+ * - Entry point during the lifecycle to customize configurations.
+ * - A default configuration to utilize and share between the tests.
+ * - Utilities functions to access the channels and protocols.
+ *
+ *
+ * With a central base class, it is easier to apply changes instead of creating a boiler plate code copied through
+ * many test classes. There is also subclasses specialized for different scenarios.
+ *
+ *
+ * When utilizing the base class with a custom data provider factory, to receive the arguments properly, the
+ * {@link #passDataProviderParameters(Object[])} needs to be overridden.
+ *
+ *
+ *
+ *
+ * Configuration
+ *
+ * The basic configuration in the class has:
+ *
+ * - {@link #clusterSize}: The number of nodes in the test.
+ * - {@link #recreatePerMethod}: If the cluster is created and strip-down per method execution.
+ * - {@link #createManually}: If the lifecycle management of creating and destroying is manual.
+ *
+ *
+ *
+ * Trace logging
+ *
+ * There is also a configuration by command-line arguments to enable tracing during the tests. To enable trace level
+ * globally, to all protocols in the stack, pass the variable {@link #ENABLE_GLOBAL_TRACE}, that is:
+ *
+ *
+ * $ mvn clean test -Dorg.jgroups.test.trace
+ *
+ *
+ * We also provide an alias with
+ *
+ *
+ * $ mvn clean test -Dtrace
+ *
+ *
+ * Any of the above will enable trace to all protocols. The output is not directed to the standard output, it can be
+ * found at the generated reports. To instead enable trace for specific classes, is necessary to use the complete
+ * variable {@link #ENABLE_TRACE_CLASSES}, passing the classes fully qualified name:
+ *
+ *
+ * $ mvn clean test -Dorg.jgroups.test.trace="org.jgroups.protocols.raft.RAFT"
+ *
+ *
+ * This will enable trace level only for the {@link RAFT} class. It is possible to pass multiple classes separated
+ * by comma.
+ *
+ *
+ * @since 1.0.13
+ */
+public abstract class AbstractRaftTest {
+
+ /**
+ * Enable trace log level to all classes in the protocol stack.
+ */
+ public static final String ENABLE_GLOBAL_TRACE = "org.jgrops.test.trace";
+
+ /**
+ * Enable trace log level to the specific list of classes given in FQN form and comma separated.
+ */
+ public static final String ENABLE_TRACE_CLASSES = "org.jgroups.test.traces";
+
+ // Configurable parameters for sub-classes.
+
+ /**
+ * Defines the size of the cluster, i.e., the number of nodes.
+ */
+ protected int clusterSize = 1;
+
+ /**
+ * Defines if the cluster should be cleared and re-created on a per-method basis.
+ */
+ protected boolean recreatePerMethod = false;
+
+ /**
+ * Defines if the cluster creation should happen manually. Therefore, both creation and clear happens manually.
+ */
+ protected boolean createManually = false;
+
+ /**
+ * {@link #clusterSize}.
+ *
+ * @param size: The cluster size to create.
+ * @return This test class.
+ */
+ protected final T withClusterSize(int size) {
+ this.clusterSize = size;
+ return self();
+ }
+
+ /**
+ * {@link #recreatePerMethod}.
+ *
+ * @param value: The value to use.
+ * @return This test class.
+ */
+ protected final T withRecreatePerMethod(boolean value) {
+ this.recreatePerMethod = value;
+ return self();
+ }
+
+ /**
+ * {@link #createManually}.
+ *
+ * @param value: The value to use.
+ * @return This test class.
+ */
+ protected final T withManuallyCreate(boolean value) {
+ this.createManually = value;
+ return self();
+ }
+
+ @BeforeClass(alwaysRun = true)
+ protected final void initialize(ITestContext ctx) throws Exception {
+ if (!createManually && !recreatePerMethod) createCluster();
+ }
+
+ @BeforeMethod(alwaysRun = true)
+ protected final void initializeMethod(ITestContext ctx, Object[] args) throws Exception {
+ passDataProviderParameters(args);
+ if (!createManually && recreatePerMethod) createCluster();
+ }
+
+ @AfterClass(alwaysRun = true)
+ protected final void teardown() throws Exception {
+ if (!createManually && !recreatePerMethod) destroyCluster();
+ }
+
+ @AfterMethod(alwaysRun = true)
+ protected final void teardownMethod() throws Exception {
+ if (!createManually && recreatePerMethod) destroyCluster();
+ }
+
+ @SuppressWarnings("unchecked")
+ protected final T self() {
+ return (T) this;
+ }
+
+ /**
+ * Creates the cluster using the provided configuration of size but limiting the number of nodes to create.
+ *
+ * This method limits the number of nodes created at once. Calling the method multiple times is safe, and the
+ * maximum {@link #clusterSize} is never violated.
+ *
+ *
+ * @param limit: The number of nodes to create at once. Never creates more than {@link #clusterSize} nodes.
+ * @throws Exception: If an error occur while creating the cluster.
+ */
+ protected abstract void createCluster(int limit) throws Exception;
+
+ /**
+ * Destroy the cluster and clears any open resource.
+ *
+ * The implementors must invoke the appropriate {@link #beforeClusterDestroy()} and {@link #afterClusterDestroy()}
+ * methods to notify listeners.
+ *
+ *
+ * @throws Exception: If failed clearing the resources.
+ */
+
+ /**
+ * Destroy the cluster.
+ *
+ * This method clears all resources allocated to the cluster. The cluster should be recreated again for use.
+ *
+ *
+ * @throws Exception: If an error happens while clearing the resources.
+ */
+ protected abstract void destroyCluster() throws Exception;
+
+ /**
+ * Creates the protocol stack to utilize during the tests.
+ *
+ * @param name: The name of node requesting the stack.
+ * @return The complete protocol stack.
+ * @throws Exception: If an error occur while creating the stack.
+ */
+ protected abstract Protocol[] baseProtocolStackForNode(String name) throws Exception;
+
+ /**
+ * Apply the log level configuration.
+ *
+ * Check {@link AbstractRaftTest} about how to configure the log level. Note that, a log instance might be static,
+ * and since the tests run in parallel, this configuration might leak between tests!
+ *
+ *
+ * @param stack: The protocol stack in use.
+ */
+ protected final void applyTraceConfiguration(Protocol[] stack) {
+ String[] traces = Arrays.stream(System.getProperty(ENABLE_TRACE_CLASSES, "").split(","))
+ .filter(s -> !s.isEmpty())
+ .toArray(String[]::new);
+ boolean globalTrace = Boolean.parseBoolean(System.getProperty(ENABLE_GLOBAL_TRACE, "false"));
+
+ if (globalTrace) System.out.println("Enabling trace on all classes");
+ else if (traces.length > 0) System.out.printf("Enabling trace on classes: %s%n", Arrays.toString(traces));
+
+ Set classes = Set.of(traces);
+ for (Protocol protocol : stack) {
+ if (globalTrace || classes.contains(protocol.getClass().getName())) {
+ protocol.level("trace");
+ }
+ }
+ }
+
+ /**
+ * Creates the cluster using the provided configuration of size.
+ *
+ * This method can be invoked multiple times without harm in the tests.
+ *
+ *
+ *
+ * The lifecycle methods are invoked when appropriately. The cluster lifecycle methods are always invoked.
+ *
+ *
+ * @throws Exception: If an error occur while creating the cluster.
+ */
+ protected final void createCluster() throws Exception {
+ beforeClusterCreation();
+ createCluster(clusterSize);
+ afterClusterCreation();
+ }
+
+ /**
+ * The cluster name to connect the channels. Defaults to the test class name.
+ *
+ * @return The cluster name.
+ */
+ protected String clusterName() {
+ return getClass().getSimpleName();
+ }
+
+ /**
+ * Creates a new {@link RAFT} instance.
+ *
+ * The default configuration has configured:
+ *
+ *
+ * - The raft ID.
+ * - The members IDs, based on the offset from
'A'
and the {@link #clusterSize} .
+ * - The log to use with {@link #getRaftLogClass()}.
+ * - The log prefix, the raft ID concat with the cluster name.
+ *
+ * This method is final, to update any configuration, override the {@link #amendRAFTConfiguration(RAFT)}. The
+ * method is invoked before returning.
+ *
+ *
+ * @param raftId: The ID of the node.
+ * @return A new and properly configured {@link RAFT} instance.
+ */
+ protected final RAFT createNewRaft(String raftId) {
+ RAFT r = new RAFT()
+ .raftId(raftId)
+ .members(getRaftMembers())
+ .logClass(getRaftLogClass())
+ .stateMachine(new DummyStateMachine())
+ .logPrefix(String.format("%s-%s", raftId, clusterName()));
+ amendRAFTConfiguration(r);
+ return r;
+ }
+
+ /**
+ * Creates the election algorithm and invokes the entry point to decorate.
+ *
+ * This method is final, to define which election algorithm to use, override {@link #createNewElection()}.
+ *
+ *
+ * @return A new and properly configured {@link BaseElection} concrete implementation.
+ * @throws Exception: If failed creating the election instance.
+ */
+ protected final BaseElection createNewElectionAndDecorate() throws Exception {
+ BaseElection be = createNewElection();
+ amendBaseElectionConfiguration(be);
+ return be;
+ }
+
+ /**
+ * Creates the election algorithm to use.
+ *
+ * The default election implementation is {@link ELECTION}. This method is not final and could be override to
+ * create a different instance. There is also the {@link #amendBaseElectionConfiguration(BaseElection)} to change
+ * the protocol configuration.
+ *
+ *
+ * @return A new and properly configured {@link BaseElection} concrete implementation.
+ */
+ protected BaseElection createNewElection() throws Exception {
+ return new ELECTION();
+ }
+
+ /**
+ * Retrieves the members utilized in the {@link RAFT} configuration.
+ *
+ * This implementation offset from 'A'
up to {@link #clusterSize} members. That is, for a size of
+ * 2
, creates a list with 'A', 'B'
.
+ *
+ *
+ * @return The list of members in the cluster.
+ */
+ protected final Collection getRaftMembers() {
+ List members = new ArrayList<>(clusterSize);
+ for (int i = 0; i < clusterSize; i++) {
+ members.add(Character.toString('A' + i));
+ }
+
+ return members;
+ }
+
+ /**
+ * The {@link org.jgroups.protocols.raft.Log} to use in {@link RAFT}.
+ * Defaults to {@link InMemoryLog}.
+ *
+ * @return The log class to use.
+ */
+ protected String getRaftLogClass() {
+ return InMemoryLog.class.getName();
+ }
+
+ /**
+ * Entrypoint to change configurations of the {@link RAFT} instance before start.
+ *
+ * This method is invoked every time a new {@link RAFT} instance is created.
+ *
+ *
+ * @param raft: The {@link RAFT} instance with basic configuration.
+ * @see #createNewRaft(String)
+ */
+ protected void amendRAFTConfiguration(RAFT raft) { }
+
+ /**
+ * Entrypoint to change configuration of the {@link BaseElection} instance before start.
+ *
+ * This method is invoked every time a new {@link BaseElection} instance is created, the actual concrete type
+ * might vary depending on the test.
+ *
+ *
+ * @param election: The instance with base configuration.
+ * @see #createNewElection()
+ */
+ protected void amendBaseElectionConfiguration(BaseElection election) { }
+
+ /**
+ * Necessary when utilizing the base class with a data provider factory.
+ *
+ * @param args: The data provider arguments.
+ */
+ protected void passDataProviderParameters(Object[] args) { }
+
+ /**
+ * Entrypoint executed before the cluster is created.
+ * This method might be invoked multiple times during a single test.
+ *
+ * @throws Exception: If an exception happens during the execution.
+ */
+ protected void beforeClusterCreation() throws Exception { }
+
+ /**
+ * Entrypoint executed before destroying the cluster.
+ *
+ * @throws Exception: If an exception happens during the execution.
+ */
+ protected void beforeClusterDestroy() throws Exception { }
+
+ /**
+ * Entrypoint executed after the cluster is created and running.
+ * This method can execute multiple times during a single test, and executes after all channels have the same view.
+ *
+ * @throws Exception: If an exception happens during execution.
+ * @see #beforeClusterCreation()
+ */
+ protected void afterClusterCreation() throws Exception { }
+
+ /**
+ * Entrypoint executed after the cluster is destroyed.
+ *
+ * @throws Exception: If an exception happens during execution.
+ */
+ protected void afterClusterDestroy() throws Exception { }
+}
diff --git a/tests/junit-functional/org/jgroups/tests/harness/BaseRaftChannelTest.java b/tests/junit-functional/org/jgroups/tests/harness/BaseRaftChannelTest.java
new file mode 100644
index 00000000..dd745bec
--- /dev/null
+++ b/tests/junit-functional/org/jgroups/tests/harness/BaseRaftChannelTest.java
@@ -0,0 +1,367 @@
+package org.jgroups.tests.harness;
+
+import org.jgroups.Address;
+import org.jgroups.JChannel;
+import org.jgroups.protocols.raft.RAFT;
+import org.jgroups.protocols.raft.REDIRECT;
+import org.jgroups.protocols.raft.election.BaseElection;
+import org.jgroups.raft.testfwk.RaftTestUtils;
+import org.jgroups.stack.Protocol;
+import org.jgroups.util.Util;
+
+import java.util.ArrayList;
+import java.util.List;
+
+/**
+ * The base class for tests that utilize {@link JChannel} for communication.
+ *
+ * This class already provides the mechanism to retrieve and access the protocols in the {@link JChannel}. Creating a
+ * cluster waits until all the nodes receive the same view. The cluster can be resized dynamically during a test
+ * execution, growing to accommodate more nodes, but not shrinking.
+ *
+ */
+public class BaseRaftChannelTest extends AbstractRaftTest {
+
+ private JChannel[] channels;
+
+ @Override
+ protected final void createCluster(int limit) throws Exception {
+ System.out.printf("%s: creating cluster '%s' with size %d (limited %d) %n", getClass(), clusterName(), clusterSize, limit);
+
+ if (this.channels == null) this.channels = new JChannel[clusterSize];
+ if (this.channels.length < clusterSize) {
+ JChannel[] updated = new JChannel[clusterSize];
+ System.arraycopy(channels, 0, updated, 0, channels.length);
+ channels = updated;
+ }
+
+ for (int i = 0; i < clusterSize; i++) {
+
+ // The creation can be invoked multiple times to resize the cluster and add new members.
+ if (channels[i] != null) continue;
+
+ String name = Character.toString('A' + i);
+ channels[i] = createChannel(name);
+ limit -= 1;
+
+ if (limit == 0) break;
+ }
+
+ Util.waitUntilAllChannelsHaveSameView(10_000, 100, actualChannels());
+ System.out.printf("%s: current cluster %s of size %d view is:%n%s%n", getClass(), clusterName(), clusterSize, printCurrentCluster());
+ }
+
+ /**
+ * Creates a new {@link JChannel} using the configurations and attached to the lifecycle entrypoint.
+ *
+ * @param name: The {@link JChannel} name.
+ * @return A new instance of the {@link JChannel}.
+ * @throws Exception: If failed to create the channel.
+ */
+ protected JChannel createChannel(String name) throws Exception {
+ JChannel ch = createDisconnectedChannel(name);
+ beforeChannelConnection(ch);
+ ch.connect(clusterName());
+ afterChannelConnection(ch);
+ return ch;
+ }
+
+ /**
+ * Creates a disconnected {@link JChannel}.
+ *
+ * @param name: The {@link JChannel} name.
+ * @return A disconnected new instance of the {@link JChannel}.
+ * @throws Exception: If failed to create the channel.
+ */
+ protected JChannel createDisconnectedChannel(String name) throws Exception {
+ Protocol[] stack = baseProtocolStackForNode(name);
+
+ applyTraceConfiguration(stack);
+
+ JChannel ch = new JChannel(stack);
+ ch.name(name);
+ return ch;
+ }
+
+ /**
+ * Destroy the complete cluster.
+ *
+ * This iterates the {@link #channels} in reverse order to avoid any major disruption. Each channel is closed and
+ * set to null. This deletes the state information of the {@link RAFT} instance, deleting the log. All the
+ * created resources are cleared after this method executes.
+ *
+ *
+ * @throws Exception: If an error happens while clearing the resources.
+ */
+ @Override
+ protected final void destroyCluster() throws Exception {
+ beforeClusterDestroy();
+
+ System.out.printf("%s: destroying cluster %s with size %d%n", getClass(), clusterName(), clusterSize);
+ if (channels != null) {
+ for (int i = clusterSize - 1; i >= 0; i--) {
+ close(i);
+ }
+ }
+
+ this.channels = null;
+ afterClusterDestroy();
+ }
+
+ /**
+ * Close the {@link JChannel} in the given index.
+ *
+ * The {@link JChannel} is closed and the {@link RAFT} state is deleted.
+ *
+ *
+ * @param index: The channel to close.
+ * @throws Exception: If an error happens while closing the resources.
+ */
+ protected final void close(int index) throws Exception {
+ JChannel ch = channel(index);
+ if (ch == null) return;
+
+ channels[index] = null;
+ close(ch);
+ }
+
+ /**
+ * Closes the given channel and deletes the {@link RAFT} state.
+ *
+ * @param ch: The {@link JChannel} to close.
+ * @throws Exception: If an error happens while closing the resources.
+ */
+ protected final void close(JChannel ch) throws Exception {
+ if (ch == null) return;
+
+ if (channels != null) {
+ int i = 0;
+ for (JChannel curr : channels) {
+ if (curr == ch) {
+ channels[i] = null;
+ break;
+ }
+ i++;
+ }
+ }
+
+ // Always close the channel before deleting the log.
+ Util.close(ch);
+ RaftTestUtils.deleteRaftLog(raft(ch));
+ }
+
+ /**
+ * The stack to utilize in the node during the test with a channel.
+ *
+ * By default, besides decorating with the protocols from {@link Util#getTestStack(Protocol...)}, we use:
+ *
+ * - An election algorithm {@link BaseElection}.
+ * - The {@link RAFT} protocol.
+ * - The {@link REDIRECT} protocol.
+ *
+ *
+ *
+ * @param name: The node's name creating the stack.
+ * @return The complete stack in the correct order.
+ */
+ @Override
+ protected Protocol[] baseProtocolStackForNode(String name) throws Exception {
+ return Util.getTestStack(createNewElectionAndDecorate(), createNewRaft(name), new REDIRECT());
+ }
+
+ /**
+ * Get the created channel in the provided index.
+ *
+ * @param index: The index of the channel to retrieve.
+ * @return The channel in the specified position. Might return null
if the channel was closed.
+ */
+ protected final JChannel channel(int index) {
+ assert index < channels.length : "Index out of bounds, maximum is " + (clusterSize - 1);
+ return channels[index];
+ }
+
+ /**
+ * Get the {@link JChannel} with the given address.
+ *
+ * @param addr: Channel address to match.
+ * @return The channel with the given address, or null
, otherwise.
+ */
+ protected final JChannel channel(Address addr) {
+ for(JChannel ch: channels) {
+ if(ch.getAddress() != null && ch.getAddress().equals(addr))
+ return ch;
+ }
+ return null;
+ }
+
+ /**
+ * Get the created {@link RAFT} in the provided index.
+ *
+ * @param index: The index of the RAFT instance to retrieve.
+ * @return The {@link RAFT} in the specified position or null
.
+ * @see #channel(int)
+ */
+ protected final RAFT raft(int index) {
+ JChannel ch = channel(index);
+ return ch == null ? null : raft(ch);
+ }
+
+ /**
+ * The {@link RAFT} instance in the provided {@link JChannel}.
+ *
+ * @param ch: The channel to retrieve the protocol.
+ * @return The RAFT protocol instance or null
.
+ */
+ protected final RAFT raft(JChannel ch) {
+ return RaftTestUtils.raft(ch);
+ }
+
+ /**
+ * Search the first node identified as leader.
+ * As per the Raft specification, the algorithm can have more than one leader in different terms. If this method
+ * is invoked during a network split or during an election round, it could have fuzzy results. It is better suited
+ * to use when a stable topology is in place.
+ *
+ * @return The first node identified as an elected leader.
+ */
+ protected final RAFT leader() {
+ for (JChannel ch : channels) {
+ if (ch == null) continue;
+
+ RAFT r = raft(ch);
+ if (r.isLeader()) return r;
+ }
+
+ return null;
+ }
+
+ /**
+ * Finds the first node identified as leader and retrieves the address.
+ *
+ * @return The leader address or null
.
+ */
+ protected final Address leaderAddress() {
+ RAFT r = leader();
+ return r == null ? null : r.getAddress();
+ }
+
+ /**
+ * Iterate over all nodes and return the ones which identify itself as leader.
+ *
+ * @return All nodes which sees itself as leader.
+ * @see #leader()
+ */
+ protected final List leaders() {
+ List leaders = new ArrayList<>();
+ for (JChannel ch : channels) {
+ if (ch == null) continue;
+
+ RAFT r = raft(ch);
+ if (r.isLeader()) leaders.add(r);
+ }
+
+ return leaders;
+ }
+
+ /**
+ * Generates a message with all members view of the current term and leader.
+ *
+ * @return A string with a cluster-wide view of the cluster.
+ */
+ protected final String dumpLeaderAndTerms() {
+ List members = new ArrayList<>(getRaftMembers());
+ StringBuilder builder = new StringBuilder("Cluster: ")
+ .append(clusterName())
+ .append(System.lineSeparator());
+
+ for (int i = 0; i < clusterSize; i++) {
+ String member = members.get(i);
+ RAFT raft = raft(i);
+
+ if (raft == null) {
+ builder.append(member)
+ .append(" is null");
+ } else {
+ builder.append(member)
+ .append(" -> ")
+ .append(raft.currentTerm())
+ .append(" and leader ")
+ .append(raft.leader());
+ }
+
+ builder.append(System.lineSeparator());
+ }
+ return builder.toString();
+ }
+
+ protected final String printCurrentCluster() {
+ List members = new ArrayList<>(getRaftMembers());
+
+ StringBuilder sb = new StringBuilder();
+ for (int i = 0; i < clusterSize; i++) {
+ JChannel ch = channel(i);
+ if (ch == null) {
+ sb.append(members.get(i)).append(" is null");
+ } else {
+ sb.append(members.get(i))
+ .append(": ")
+ .append(ch.getViewAsString());
+ }
+ sb.append(System.lineSeparator());
+ }
+ return sb.toString();
+ }
+
+ /**
+ * Get the complete array of channels.
+ * During the execution, as channels are closed, the array might contain null
entries.
+ *
+ * @return The complete array of channels.
+ */
+ protected final JChannel[] channels() {
+ return channels;
+ }
+
+ /**
+ * Iterate over all channels and remove any null
entries.
+ *
+ * @return The array of currently active channels.
+ */
+ protected final JChannel[] actualChannels() {
+ List c = new ArrayList<>(channels.length);
+ for (JChannel ch : channels) {
+ if (ch != null) c.add(ch);
+ }
+
+ return c.toArray(new JChannel[0]);
+ }
+
+ /**
+ * Retrieve the address of the channel by index.
+ *
+ * @param index: The channel index to retrieve the address.
+ * @return The channel address.
+ */
+ protected final Address address(int index) {
+ return channel(index).getAddress();
+ }
+
+ /**
+ * Entrypoint executed before the {@link JChannel} connects to the cluster.
+ * This method is invoked for every new {@link JChannel} instance created. The channel is already configured at
+ * this point and contains the configured protocol stack.
+ *
+ * @param ch: The configured channel which is connecting.
+ * @throws Exception: If an exception happens during the execution.
+ */
+ protected void beforeChannelConnection(JChannel ch) throws Exception { }
+
+ /**
+ * Entrypoint executed after the {@link JChannel} connects to the cluster.
+ *
+ * @param ch: The channel connected instance.
+ * @throws Exception: If an exception happens during the execution.
+ * @see #beforeChannelConnection(JChannel)
+ */
+ protected void afterChannelConnection(JChannel ch) throws Exception { }
+}
diff --git a/tests/junit-functional/org/jgroups/tests/harness/BaseRaftClusterTest.java b/tests/junit-functional/org/jgroups/tests/harness/BaseRaftClusterTest.java
new file mode 100644
index 00000000..196163d8
--- /dev/null
+++ b/tests/junit-functional/org/jgroups/tests/harness/BaseRaftClusterTest.java
@@ -0,0 +1,299 @@
+package org.jgroups.tests.harness;
+
+import org.jgroups.Address;
+import org.jgroups.View;
+import org.jgroups.protocols.raft.RAFT;
+import org.jgroups.protocols.raft.election.BaseElection;
+import org.jgroups.raft.testfwk.MockRaftCluster;
+import org.jgroups.raft.testfwk.RaftNode;
+import org.jgroups.raft.testfwk.RaftTestUtils;
+import org.jgroups.stack.Protocol;
+import org.jgroups.util.ExtendedUUID;
+import org.jgroups.util.Util;
+
+import java.util.Arrays;
+import java.util.List;
+import java.util.Objects;
+import java.util.stream.Collectors;
+
+/**
+ * Base class test utilizing {@link MockRaftCluster} instances to mock a cluster.
+ *
+ *
+ * This class implements the base utilities to retrieve the cluster members ({@link RaftNode}), access the cluster,
+ * and the stack protocols.
+ *
+ *
+ * @param : The type of the cluster instances.
+ * @since 1.0.13
+ * @see AbstractRaftTest
+ * @see MockRaftCluster
+ */
+public abstract class BaseRaftClusterTest extends AbstractRaftTest {
+
+ /**
+ * Keep track of the cluster members.
+ */
+ private RaftNode[] nodes;
+
+ /**
+ * The actual cluster instance. Visible to subclasses to interact.
+ */
+ protected T cluster;
+
+ /**
+ * Creates a new {@link MockRaftCluster} instance.
+ *
+ * @return A new {@link MockRaftCluster} instance. In case null
is returned, the cluster is not created.
+ */
+ protected abstract T createNewMockCluster();
+
+ @Override
+ protected final void createCluster(int limit) throws Exception {
+ System.out.printf("-- Creating mock cluster %s with size %s limited to %d%n", clusterName(), clusterSize, limit);
+
+ if (cluster == null) {
+ if ((cluster = createNewMockCluster()) == null) {
+ System.out.println("-- No cluster instance created!");
+ return;
+ }
+ }
+
+ // Maybe resizing would be necessary when dynamically expanding the cluster.
+ if (nodes == null) nodes = new RaftNode[clusterSize];
+
+ while (limit > 0 && cluster.size() < clusterSize) {
+ String name = Character.toString('A' + cluster.size());
+ Address address = createAddress(name);
+
+ RaftNode node = createNode(name, address);
+ node.init();
+ nodes[cluster.size()] = node;
+ cluster.add(address, node);
+ node.start();
+
+ limit -= 1;
+ }
+ }
+
+ @Override
+ protected final void destroyCluster() throws Exception {
+ beforeClusterDestroy();
+
+ System.out.printf("-- Destroying mock cluster %s named %s with size %d%n", cluster, clusterName(), clusterSize);
+
+ if (nodes == null) return;
+
+ for (int i = clusterSize - 1; i >= 0; i--) {
+ close(i);
+ }
+
+ cluster.clear();
+ cluster = null;
+ nodes = null;
+
+ afterClusterDestroy();
+ }
+
+ /**
+ * Close the member identified by the index.
+ *
+ * This method removes the {@link RaftNode} from the active member list, subsequent calls to {@link #node(int)}
+ * will return null
. Also removes the member from the actual cluster view and deletes the
+ * {@link RAFT} persisted state.
+ *
+ *
+ * @param index: The index to retrieve the {@link RaftNode}.
+ * @throws Exception: If an error happens while clearing the resources.
+ * @see BaseRaftChannelTest#close(int)
+ */
+ protected final void close(int index) throws Exception {
+ RaftNode node = node(index);
+
+ if (node == null) return;
+
+ cluster.remove(node.getAddress());
+
+ node.stop();
+ node.destroy();
+
+ // Always stop everything **before** deleting the log.
+ RaftTestUtils.deleteRaftLog(node.raft());
+
+ nodes[index] = null;
+ }
+
+ @Override
+ protected Protocol[] baseProtocolStackForNode(String name) throws Exception {
+ return new Protocol[] {
+ createNewElectionAndDecorate(),
+ createNewRaft(name),
+ };
+ }
+
+ /**
+ * Retrieves a {@link RaftNode} by index.
+ *
+ * @param index: Index to retrieve the member.
+ * @return A {@link RaftNode} instance or null
.
+ * @throws AssertionError: in case the {@param index} is greater that the {@link #clusterSize} or the cluster
+ * was not created.
+ */
+ protected final RaftNode node(int index) {
+ assert index < clusterSize : "Index out of bounds, maximum " + clusterSize;
+ assert nodes != null : "Cluster not created!";
+ return nodes[index];
+ }
+
+ /**
+ * Retrieves the {@link RAFT} instance from the node identified by the index.
+ *
+ * @param index: Index to retrieve the cluster member.
+ * @return A {@link RAFT} instance of null
in case the member is not present.
+ * @see #node(int)
+ */
+ protected final RAFT raft(int index) {
+ RaftNode node = node(index);
+ return node == null ? null : node.raft();
+ }
+
+ /**
+ * Retrieves the {@link Address} of the member identified by the index.
+ *
+ * @param index: Index to retrieve the cluster member.
+ * @return The member's {@link Address} or null
if it is not present in the cluster.
+ */
+ protected final Address address(int index) {
+ RaftNode node = node(index);
+ return node.raft().getAddress();
+ }
+
+ /**
+ * Retrieve all cluster members.
+ *
+ *
+ * The array might contain null
entries.
+ *
+ *
+ * @return All the cluster {@link RaftNode}s.
+ */
+ protected final RaftNode[] nodes() {
+ return nodes;
+ }
+
+ /**
+ * Retrieve all the {@link RAFT}s in the cluster.
+ *
+ * @return The existing {@link RAFT} instances. There is no null
entries.
+ */
+ protected final RAFT[] rafts() {
+ return Arrays.stream(nodes)
+ .filter(Objects::nonNull)
+ .map(RaftNode::raft)
+ .toArray(RAFT[]::new);
+ }
+
+ /**
+ * Generates a message identifying the term and leader of each cluster member.
+ *
+ * @return A string containing a cluster-wide view of leader and term.
+ */
+ protected String dumpLeaderAndTerms() {
+ StringBuilder sb = new StringBuilder();
+ for (int i = 0; i < nodes.length; i++) {
+ RaftNode node = nodes[i];
+
+ if (node == null) {
+ sb.append(Character.toString('A' + i))
+ .append(" is null")
+ .append(System.lineSeparator());
+ continue;
+ }
+
+ RAFT raft = node.raft();
+ sb.append(raft.raftId())
+ .append(": (")
+ .append(raft.currentTerm())
+ .append(") with leader ")
+ .append(raft.leader())
+ .append(System.lineSeparator());
+ }
+ return sb.toString();
+ }
+
+ /**
+ * Retrieves the {@link RAFT} leader of each active node in the cluster.
+ *
+ * This method also removes the null
entries.
+ *
+ *
+ * @return Non-null leader of each cluster member.
+ */
+ protected final List leaders() {
+ return Arrays.stream(nodes)
+ .filter(Objects::nonNull)
+ .map(RaftNode::raft)
+ .map(RAFT::leader)
+ .filter(Objects::nonNull)
+ .distinct()
+ .collect(Collectors.toList());
+ }
+
+ /**
+ * Creates a new {@link View}.
+ *
+ * The method creates a new view containing all the active members identified by the indexes. If a member for
+ * a given index is not present in the cluster, it is not added to the final view.
+ *
+ *
+ *
+ * The index on the first entry, i.e., indexes[0]
, is the view coordinator.
+ *
+ *
+ * @param id: The {@link View} id.
+ * @param indexes: The cluster members to add to the view.
+ * @return A new {@link View} instance.
+ * @throws AssertionError if no members identified by the indexes is present in the cluster.
+ * @see #node(int)
+ */
+ protected final View createView(long id, int ... indexes) {
+ List addresses = Arrays.stream(indexes)
+ .distinct()
+ .mapToObj(this::node)
+ .filter(Objects::nonNull)
+ .map(RaftNode::getAddress)
+ .collect(Collectors.toList());
+
+ assert !addresses.isEmpty() : "No members left to create a view!";
+ return View.create(addresses.get(0), id, addresses);
+ }
+
+ private RaftNode createNode(String name, Address address) throws Exception {
+ Protocol[] stack = baseProtocolStackForNode(name);
+ applyTraceConfiguration(stack);
+
+ RAFT r = findProtocol(stack, RAFT.class);
+ BaseElection be = findProtocol(stack, BaseElection.class);
+
+ assert r != null : "RAFT never found!";
+ r.setAddress(address);
+
+ // Some tests use cluster without election.
+ if (be != null) be.raft(r).setAddress(address);
+
+ return new RaftNode(cluster, stack);
+ }
+
+ private Address createAddress(String name) {
+ ExtendedUUID.setPrintFunction(RAFT.print_function);
+ return ExtendedUUID.randomUUID(name).put(RAFT.raft_id_key, Util.stringToBytes(name));
+ }
+
+ private P findProtocol(Protocol[] stack, Class
clazz) {
+ for (Protocol protocol : stack) {
+ if (clazz.isAssignableFrom(protocol.getClass()))
+ return (P) protocol;
+ }
+ return null;
+ }
+}
diff --git a/tests/junit-functional/org/jgroups/tests/harness/BaseRaftElectionTest.java b/tests/junit-functional/org/jgroups/tests/harness/BaseRaftElectionTest.java
new file mode 100644
index 00000000..c7d88e60
--- /dev/null
+++ b/tests/junit-functional/org/jgroups/tests/harness/BaseRaftElectionTest.java
@@ -0,0 +1,362 @@
+package org.jgroups.tests.harness;
+
+import org.jgroups.JChannel;
+import org.jgroups.protocols.raft.ELECTION;
+import org.jgroups.protocols.raft.ELECTION2;
+import org.jgroups.protocols.raft.RAFT;
+import org.jgroups.protocols.raft.election.BaseElection;
+import org.jgroups.raft.testfwk.MockRaftCluster;
+import org.jgroups.raft.testfwk.RaftNode;
+import org.jgroups.raft.testfwk.RaftTestUtils;
+
+import java.util.Arrays;
+import java.util.Objects;
+import java.util.concurrent.TimeUnit;
+import java.util.function.BooleanSupplier;
+import java.util.function.Supplier;
+import java.util.stream.Collectors;
+
+import org.testng.annotations.DataProvider;
+
+import static org.assertj.core.api.Assertions.assertThat;
+
+/**
+ * Base class for election tests.
+ *
+ * This class itself is final, but it has two inner classes for tests based either on channels or the mock cluster.
+ * The {@link ChannelBased} or {@link ClusterBased}, respectively. Subclasses must choose which approach to use and
+ * extend one of the inner classes.
+ *
+ *
+ *
+ * This class contains some utilities for election, like retrieving the protocol or waiting for something related to
+ * the election mechanism. There is no methods required for implementation.
+ *
+ *
+ * @since 1.0.13
+ * @see AbstractRaftTest
+ */
+public final class BaseRaftElectionTest {
+
+ /**
+ * A data provider to utilize with the {@link org.testng.annotations.Test} annotation.
+ *
+ * This data provider pass as argument the classes which implement an election algorithm.
+ *
+ */
+ public static final String ALL_ELECTION_CLASSES_PROVIDER = "all-election-classes";
+
+ private static Object[][] electionClasses() {
+ return new Object[][] {
+ {ELECTION.class},
+ {ELECTION2.class},
+ };
+ }
+
+ /**
+ * Asserts that something eventually evaluates to true
.
+ *
+ * @param timeout: The timeout in milliseconds to wait.
+ * @param bs: The expression to evaluate.
+ * @param message: The error message in case of failure.
+ */
+ private static void assertWaitUntil(long timeout, BooleanSupplier bs, Supplier message) {
+ assertThat(RaftTestUtils.eventually(bs, timeout, TimeUnit.MILLISECONDS))
+ .as(message)
+ .isTrue();
+ }
+
+ /**
+ * Wait until the voting thread stops running on all election instances.
+ *
+ * @param elections: The elections classes to verify.
+ * @param timeout: Time in milliseconds to wait.
+ */
+ private static void waitUntilVotingThreadStop(BaseElection[] elections, long timeout) {
+ // Wait until the voting thread stops.
+ // In theory, the voting thread runs only on the view coordinator, nevertheless we iterate over all member.
+ BooleanSupplier bs = () -> Arrays.stream(elections)
+ .filter(Objects::nonNull)
+ .noneMatch(BaseElection::isVotingThreadRunning);
+ Supplier message = () -> Arrays.stream(elections)
+ .filter(Objects::nonNull)
+ .map(e -> e.raft().raftId() + " voting running? " + e.isVotingThreadRunning())
+ .collect(Collectors.joining(System.lineSeparator()));
+
+ assertWaitUntil(timeout, bs, message);
+ }
+
+ /**
+ * Wait and assert a leader is elected in the given {@link RAFT}s.
+ *
+ * This method waits until a leader is elected between the given {@link RAFT}s instances. A leader is elected once:
+ *
+ * - A majority of members see the same leader;
+ * - A majority of members have the same term;
+ * - The leader seen by everyone is in the {@link RAFT} instances.
+ *
+ *
+ *
+ * @param rafts: The instances to check for an elected leader.
+ * @param timeout: The timeout in milliseconds to wait for the election.
+ */
+ public static void waitUntilLeaderElected(RAFT[] rafts, long timeout) {
+ Supplier message = () -> Arrays.stream(rafts)
+ .map(r -> String.format("%s: %d -> %s", r.raftId(), r.currentTerm(), r.leader()))
+ .collect(Collectors.joining(System.lineSeparator()))
+ + "\n with a majority of " + rafts[0].majority();
+ assertWaitUntil(timeout, checkRaftLeader(rafts), message);
+ }
+
+ /**
+ * Blocks until a new leader is elected between the provided instances and is received by all participants.
+ *
+ * @param rafts: Instances to check.
+ * @param timeout: The timeout in milliseconds to wait.
+ * @see #waitUntilLeaderElected(RAFT[], long)
+ */
+ public static void waitUntilAllHaveLeaderElected(RAFT[] rafts, long timeout) {
+ waitUntilLeaderElected(rafts, timeout);
+ BooleanSupplier bs = () -> Arrays.stream(rafts)
+ .filter(Objects::nonNull)
+ .allMatch(r -> r.leader() != null);
+ Supplier message = () -> Arrays.stream(rafts)
+ .map(r -> String.format("%s: %d -> %s", r.raftId(), r.currentTerm(), r.leader()))
+ .collect(Collectors.joining(System.lineSeparator()));
+ assertWaitUntil(timeout, bs, message);
+ }
+
+ private static BooleanSupplier checkRaftLeader(RAFT[] rafts) {
+ int majority = rafts[0].majority();
+
+ assertThat(rafts).hasSizeGreaterThanOrEqualTo(majority);
+
+ // Check that a majority of nodes have the same leader.
+ BooleanSupplier sameLeader = () -> Arrays.stream(rafts)
+ .map(RAFT::leader)
+ .filter(Objects::nonNull)
+ .count() >= majority;
+
+ // Check that a majority of members have the same term.
+ BooleanSupplier sameTerm = () -> {
+ long currentTerm = Arrays.stream(rafts)
+ .map(RAFT::currentTerm)
+ .max(Long::compareTo)
+ .orElseThrow();
+ long counter = 0;
+ for (RAFT raft : rafts) {
+ if (raft.currentTerm() == currentTerm) counter++;
+ }
+ return counter >= majority;
+ };
+
+ // Check that the node seen as leader is present in the cluster and sees itself as leader.
+ // This is necessary when checking for a new leader between elections. Say we had a leader, stop some nodes,
+ // and check again. The nodes could see the past leader still and evaluate to true.
+ BooleanSupplier electedIsLeader = () -> Arrays.stream(rafts)
+ .map(RAFT::leader)
+ .filter(Objects::nonNull)
+ .distinct()
+ .map(l -> Arrays.stream(rafts)
+ .filter(r -> r.getAddress().equals(l))
+ .findFirst())
+ .allMatch(o -> {
+ if (o.isEmpty()) return false;
+ RAFT r = o.get();
+ return r.isLeader();
+ });
+
+ return () -> sameLeader.getAsBoolean()
+ && sameTerm.getAsBoolean()
+ && electedIsLeader.getAsBoolean();
+ }
+
+ /**
+ * Internal class utilized to instantiate the concrete {@link BaseElection} instance.
+ *
+ * This internal class receives the data provider argument from {@link #ALL_ELECTION_CLASSES_PROVIDER} and
+ * creates a new instance.
+ *
+ */
+ private static class ClassHandler {
+ protected Class extends BaseElection> electionClass;
+
+ @SuppressWarnings("unchecked")
+ protected void setElectionClass(Object[] args) {
+ electionClass = (Class extends BaseElection>) args[0];
+ }
+
+ public BaseElection instantiate() throws Exception {
+ assert electionClass != null : "Election class not set";
+ // The default constructor is always available.
+ return electionClass.getDeclaredConstructor().newInstance();
+ }
+ }
+
+ /**
+ * Base class for election tests based on {@link JChannel}.
+ *
+ * This class builds upon {@link BaseRaftChannelTest} and offer utilities specific to the election process.
+ *
+ */
+ public static class ChannelBased extends BaseRaftChannelTest {
+
+ private final ClassHandler handler = new ClassHandler();
+
+ @Override
+ protected void passDataProviderParameters(Object[] args) {
+ handler.setElectionClass(args);
+ }
+
+ @Override
+ protected BaseElection createNewElection() throws Exception {
+ return handler.instantiate();
+ }
+
+ /**
+ * Stop the voting thread manually.
+ */
+ protected final void stopVotingThread() {
+ for (JChannel channel : channels()) {
+ if (channel == null) continue;
+
+ BaseElection election = election(channel);
+ election.stopVotingThread();
+ }
+ }
+
+ /**
+ * Retrieves the concrete {@link BaseElection} implementation in the protocol stack.
+ *
+ * @param ch: Channel to retrieve the election protocol.
+ * @return The {@link BaseElection} concrete implementation or null
, if not found.
+ */
+ protected final BaseElection election(JChannel ch) {
+ return ch.getProtocolStack().findProtocol(handler.electionClass);
+ }
+
+ /**
+ * Wait until a leader is elected in the cluster.
+ *
+ * This methods blocks and waits until all the requested members in the cluster have an elected leader.
+ *
+ *
+ * @param timeout: Time in milliseconds to wait for an election.
+ * @param indexes: The index of the cluster members.
+ * @see #waitUntilLeaderElected(RAFT[], long)
+ */
+ protected final void waitUntilLeaderElected(long timeout, int ... indexes) {
+ RAFT[] rafts = Arrays.stream(indexes)
+ .mapToObj(this::raft)
+ .filter(Objects::nonNull)
+ .toArray(RAFT[]::new);
+ BaseRaftElectionTest.waitUntilLeaderElected(rafts, timeout);
+ }
+
+ /**
+ * Wait until the voting thread stops running on all election instances.
+ *
+ * @param timeout: Time in milliseconds to wait.
+ * @param indexes: The index of the cluster members to check.
+ */
+ protected final void waitUntilVotingThreadStops(long timeout, int ... indexes) {
+ BaseElection[] elections = Arrays.stream(indexes)
+ .mapToObj(this::channel)
+ .filter(Objects::nonNull)
+ .map(this::election)
+ .toArray(BaseElection[]::new);
+ waitUntilVotingThreadStop(elections, timeout);
+ }
+
+ @DataProvider(name = ALL_ELECTION_CLASSES_PROVIDER)
+ protected static Object[][] electionClasses() {
+ return BaseRaftElectionTest.electionClasses();
+ }
+ }
+
+ /**
+ * Base class for election tests based on a {@link MockRaftCluster}.
+ *
+ * This class builds on the base {@link BaseRaftClusterTest} and offer utilities specific to the election process.
+ *
+ *
+ * @param : The type of {@link MockRaftCluster}.
+ */
+ public abstract static class ClusterBased extends BaseRaftClusterTest {
+
+ private final ClassHandler handler = new ClassHandler();
+
+ @Override
+ protected void passDataProviderParameters(Object[] args) {
+ handler.setElectionClass(args);
+ }
+
+ @Override
+ protected BaseElection createNewElection() throws Exception {
+ return handler.instantiate();
+ }
+
+ /**
+ * Retrieves the concrete {@link BaseElection} algorithm from the cluster member.
+ *
+ * @param index: The member index in the cluster.
+ * @return The concrete election algorithm.
+ */
+ protected final BaseElection election(int index) {
+ RaftNode node = node(index);
+ return node.election();
+ }
+
+ /**
+ * Retrieves the election protocol of all cluster members.
+ *
+ * @return The election protocol of all members in the cluster. There is no null
entries.
+ */
+ protected BaseElection[] elections() {
+ return Arrays.stream(nodes())
+ .filter(Objects::nonNull)
+ .map(RaftNode::election)
+ .toArray(BaseElection[]::new);
+
+ }
+
+ /**
+ * Wait until a leader is elected in the cluster.
+ *
+ * This methods blocks and waits until all the requested members in the cluster have an elected leader.
+ *
+ *
+ * @param timeout: Time in milliseconds to wait for an election.
+ * @param indexes: The index of the cluster members.
+ * @see #waitUntilLeaderElected(RAFT[], long)
+ */
+ protected void waitUntilLeaderElected(long timeout, int ... indexes) {
+ RAFT[] rafts = Arrays.stream(indexes)
+ .mapToObj(this::node)
+ .filter(Objects::nonNull)
+ .map(RaftNode::raft)
+ .toArray(RAFT[]::new);
+ BaseRaftElectionTest.waitUntilLeaderElected(rafts, timeout);
+ }
+
+ /**
+ * Wait until the voting thread stops running on all election instances.
+ *
+ * @param timeout: Time in milliseconds to wait.
+ * @param indexes: The index of the cluster members to check.
+ */
+ protected void waitUntilVotingThreadStops(long timeout, int ... indexes) {
+ BaseElection[] elections = Arrays.stream(indexes)
+ .mapToObj(this::node)
+ .filter(Objects::nonNull)
+ .map(RaftNode::election)
+ .toArray(BaseElection[]::new);
+ waitUntilVotingThreadStop(elections, timeout);
+ }
+
+ @DataProvider(name = ALL_ELECTION_CLASSES_PROVIDER)
+ protected static Object[][] electionClasses() {
+ return BaseRaftElectionTest.electionClasses();
+ }
+ }
+}
diff --git a/tests/junit-functional/org/jgroups/tests/harness/BaseStateMachineTest.java b/tests/junit-functional/org/jgroups/tests/harness/BaseStateMachineTest.java
new file mode 100644
index 00000000..2095ed2b
--- /dev/null
+++ b/tests/junit-functional/org/jgroups/tests/harness/BaseStateMachineTest.java
@@ -0,0 +1,159 @@
+package org.jgroups.tests.harness;
+
+import org.jgroups.JChannel;
+import org.jgroups.protocols.raft.RAFT;
+import org.jgroups.raft.RaftHandle;
+import org.jgroups.raft.StateMachine;
+
+import java.util.Arrays;
+import java.util.Objects;
+import java.util.concurrent.TimeUnit;
+import java.util.function.BooleanSupplier;
+
+import static org.assertj.core.api.Assertions.assertThat;
+import static org.jgroups.raft.testfwk.RaftTestUtils.eventually;
+
+/**
+ * Base class for tests that use a {@link StateMachine}.
+ *
+ *
+ * This class provides base functionalities to assert on the {@link StateMachine}, and easy methods to retrieve the
+ * instances.
+ *
+ *
+ * Contract
+ *
+ *
+ * All the subclasses must override the {@link #createStateMachine(JChannel)} method. This is invoked everytime
+ * a new node is added to the cluster. A method to the super is necessary when overriding
+ * {@link #beforeClusterCreation()} or {@link #beforeChannelConnection(JChannel)}. These methods are responsible for
+ * populating the {@link RaftHandle}s and {@link StateMachine}s.
+ *
+ *
+ * @param : The type of the {@link StateMachine} the test uses.
+ * @since 1.0.13
+ * @see BaseRaftChannelTest
+ */
+public class BaseStateMachineTest extends BaseRaftChannelTest {
+
+ // Keep track of all handles in the cluster
+ private RaftHandle[] handles;
+
+ @Override
+ protected void beforeClusterCreation() throws Exception {
+ this.handles = new RaftHandle[clusterSize];
+ }
+
+ /**
+ * Timeout when verifying if the state machines match.
+ */
+ protected long matchTimeout = 10_000;
+
+ @Override
+ protected void beforeChannelConnection(JChannel ch) {
+ int i = 0;
+ for (JChannel channel : actualChannels()) {
+ if (channel == ch) break;
+ i += 1;
+ }
+ handles[i] = new RaftHandle(ch, createStateMachine(ch));
+ }
+
+ /**
+ * Creates the {@link StateMachine} instance to use during the tests.
+ *
+ * Warning: All subclasses must override this method.
+ *
+ *
+ * @param ch: The channel to retrieve any information to create the state machine instance.
+ * @return A {@link StateMachine} instance to use in the tests.
+ */
+ protected T createStateMachine(JChannel ch) {
+ throw new IllegalStateException("Unknown state machine");
+ }
+
+ /**
+ * Retrieve the {@link StateMachine} in the given index.
+ *
+ * @param index: Index to retrieve the {@link RAFT} to retrieve the {@link StateMachine}.
+ * @return The {@link StateMachine} cast to type T.
+ * @throws AssertionError in case the {@link RAFT} instance for the index does not exist.
+ */
+ protected final T stateMachine(int index) {
+ RAFT r = raft(index);
+
+ assert r != null : "RAFT should not be null!";
+ return stateMachine(r);
+ }
+
+ /**
+ * Retrieves the {@link StateMachine} in use in the {@link RAFT} {@param r} instance.
+ *
+ * @param r: {@link RAFT} instance to retrieve the {@link StateMachine}.
+ * @return The {@link StateMachine} cast to type T.
+ */
+ protected final T stateMachine(RAFT r) {
+ // noinspection unchecked
+ return (T) r.stateMachine();
+ }
+
+ protected final RaftHandle handle(int index) {
+ return handles[index];
+ }
+
+ /**
+ * Assert that all the {@link StateMachine} in the given {@param indexes} are matching.
+ * First, verify that all members have a matching commit index, meaning everything was applied. Then, verify
+ * the actual state machines.
+ *
+ * @param indexes: Indexes to verify. The {@link JChannel} for each index must exist.
+ */
+ protected final void assertStateMachineEventuallyMatch(int ... indexes) {
+ // Verifies the commit indexes of all nodes.
+ BooleanSupplier commitIndexVerify = () -> Arrays.stream(indexes)
+ .mapToObj(this::raft)
+ .map(Objects::requireNonNull)
+ .map(RAFT::commitIndex)
+ .distinct()
+ .count() == 1;
+
+ assertThat(eventually(commitIndexVerify, matchTimeout, TimeUnit.MILLISECONDS))
+ .as(generateErrorMessage())
+ .isTrue();
+ System.out.println(dumpStateMachines(indexes));
+
+ // Verify the state machines match.
+ BooleanSupplier matchStateMachineVerify = () -> Arrays.stream(indexes)
+ .mapToObj(this::stateMachine)
+ .distinct()
+ .count() == 1;
+ assertThat(eventually(matchStateMachineVerify, matchTimeout, TimeUnit.MILLISECONDS))
+ .as(generateErrorMessage() + " where " + dumpStateMachines(indexes))
+ .isTrue();
+ }
+
+ private String generateErrorMessage() {
+ StringBuilder sb = new StringBuilder();
+ RAFT leader = leader();
+
+ assertThat(leader)
+ .as("Expecting an elected leader")
+ .isNotNull();
+
+ sb.append(leader.raftId())
+ .append(": commit-index=").append(leader.commitIndex())
+ .append(leader.dumpCommitTable());
+ return sb.toString();
+ }
+
+ private String dumpStateMachines(int ... indexes) {
+ StringBuilder sb = new StringBuilder();
+ for (int i : indexes) {
+ sb.append(raft(i).raftId())
+ .append(" -> ")
+ .append(stateMachine(i))
+ .append("\n");
+ }
+ return sb.toString();
+ }
+}
diff --git a/tests/junit-functional/org/jgroups/tests/harness/RaftAssertion.java b/tests/junit-functional/org/jgroups/tests/harness/RaftAssertion.java
new file mode 100644
index 00000000..e379a153
--- /dev/null
+++ b/tests/junit-functional/org/jgroups/tests/harness/RaftAssertion.java
@@ -0,0 +1,49 @@
+package org.jgroups.tests.harness;
+
+import java.util.concurrent.ExecutionException;
+import java.util.concurrent.TimeoutException;
+import java.util.function.Supplier;
+
+import org.assertj.core.api.Assertions;
+
+import static org.assertj.core.api.Assertions.assertThat;
+
+public final class RaftAssertion {
+
+ private RaftAssertion() { }
+
+ public static void assertLeaderlessOperationThrows(ThrowingRunnable operation) {
+ assertLeaderlessOperationThrows(operation, "Running operation without a leader.");
+ }
+
+ public static void assertLeaderlessOperationThrows(ThrowingRunnable operation, String message) {
+ assertLeaderlessOperationThrows(operation, () -> message);
+ }
+
+ public static void assertLeaderlessOperationThrows(ThrowingRunnable operation, Supplier message) {
+ Assertions.assertThatThrownBy(operation::run)
+ .as(message)
+ .satisfiesAnyOf(
+ // In case the leader already received the view update and stepped down.
+ tc -> assertThat(tc)
+ .isInstanceOf(IllegalStateException.class)
+ .hasMessageContaining("I'm not the leader "),
+
+ // In case the request is sent before the leader step down.
+ // We could update this so when the leader step down it cancel requests.
+ tc -> assertThat(tc).isInstanceOf(TimeoutException.class),
+
+ // The request was sent but failed.
+ tc -> assertThat(tc).isInstanceOf(ExecutionException.class)
+ .cause()
+ .isInstanceOf(IllegalStateException.class)
+ .hasMessageContaining("I'm not the leader ")
+ );
+ }
+
+ @FunctionalInterface
+ public interface ThrowingRunnable {
+
+ void run() throws Throwable;
+ }
+}
diff --git a/tests/junit-functional/org/jgroups/tests/utils/JUnitXMLReporter.java b/tests/junit-functional/org/jgroups/tests/utils/JUnitXMLReporter.java
index b7e89520..ef83e0e0 100644
--- a/tests/junit-functional/org/jgroups/tests/utils/JUnitXMLReporter.java
+++ b/tests/junit-functional/org/jgroups/tests/utils/JUnitXMLReporter.java
@@ -1,23 +1,43 @@
package org.jgroups.tests.utils;
-import org.jgroups.util.*;
-import org.testng.IConfigurationListener2;
-import org.testng.ITestContext;
-import org.testng.ITestListener;
-import org.testng.ITestResult;
-
-import java.io.*;
-import java.util.*;
+import org.jgroups.util.Bits;
+import org.jgroups.util.Streamable;
+import org.jgroups.util.Util;
+
+import java.io.DataInput;
+import java.io.DataInputStream;
+import java.io.DataOutput;
+import java.io.DataOutputStream;
+import java.io.File;
+import java.io.FileInputStream;
+import java.io.FileNotFoundException;
+import java.io.FileOutputStream;
+import java.io.FileReader;
+import java.io.FileWriter;
+import java.io.IOException;
+import java.io.PrintStream;
+import java.io.Reader;
+import java.io.Writer;
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.List;
+import java.util.Map;
+import java.util.Properties;
import java.util.concurrent.ConcurrentHashMap;
import java.util.concurrent.ConcurrentMap;
+import org.testng.ITestContext;
+import org.testng.ITestNGMethod;
+import org.testng.ITestResult;
+import org.testng.internal.IResultListener;
+
/**
* Listener generating XML output suitable to be processed by JUnitReport.
* Copied from TestNG (www.testng.org) and modified
*
* @author Bela Ban
*/
-public class JUnitXMLReporter implements ITestListener, IConfigurationListener2 {
+public class JUnitXMLReporter implements IResultListener {
protected String output_dir=null;
protected static final String XML_DEF="";
@@ -82,6 +102,8 @@ public void onTestStart(ITestResult result) {
/** Invoked each time a test method succeeds */
public void onTestSuccess(ITestResult tr) {
+ if (stdout.get() != null)
+ stdout.get().println("\n\n------------- SUCCESS: " + getMethodName(tr) + " -----------");
onTestCompleted(tr, "OK: ", old_stdout);
}
@@ -92,6 +114,8 @@ public void onTestFailedButWithinSuccessPercentage(ITestResult tr) {
/** Invoked each time a test method fails */
public void onTestFailure(ITestResult tr) {
+ if (stdout.get() != null)
+ stdout.get().println("\n\n------------- FAILURE: " + getMethodName(tr) + " -----------");
onTestCompleted(tr, "FAIL: ",old_stderr);
}
@@ -102,9 +126,16 @@ public void onTestSkipped(ITestResult tr) {
public void beforeConfiguration(ITestResult tr) {
setupStreams(tr, false);
+ if (stdout.get() != null)
+ stdout.get().printf("------ %s invoking: %s%n", getConfigurationMethodType(tr.getMethod()), tr.getMethod().getQualifiedName());
}
public void onConfigurationSuccess(ITestResult tr) {
+ if (stdout.get() != null) {
+ stdout.get().printf("------ %s done: %s%n", getConfigurationMethodType(tr.getMethod()), tr.getMethod().getQualifiedName());
+ if (tr.getMethod().isAfterClassConfiguration() || tr.getMethod().isAfterMethodConfiguration())
+ stdout.get().println("\n=======================\n");
+ }
closeStreams();
}
@@ -147,7 +178,7 @@ protected void setupStreams(ITestResult result, boolean printMethodName) {
if(stderr.get() == null)
stderr.set(new PrintStream(new FileOutputStream(_stderr, true)));
if(printMethodName)
- stdout.get().println("\n\n------------- " + getMethodName(result) + " -----------");
+ stdout.get().println("\n\n------------- STARTING: " + getMethodName(result) + " -----------");
}
catch(IOException e) {
error(e.toString());
@@ -217,6 +248,16 @@ else if(params[0] != null)
return method_name;
}
+ private String getConfigurationMethodType(ITestNGMethod method) {
+ if (method.isBeforeClassConfiguration()) return "Before class";
+ if (method.isAfterClassConfiguration()) return "After class";
+
+ if (method.isBeforeMethodConfiguration()) return "Before method";
+ if (method.isAfterMethodConfiguration()) return "After method";
+
+ return "";
+ }
+
/** Generate the XML report from all the test results */
protected void generateReports() throws IOException {