Replace all users with a call to command().
JIRA: CONTROLLER-2137
Change-Id: Ie21c2cd0de59058ce3bb7f3c42ae133dc9fbac2d
Signed-off-by: Robert Varga <robert.varga@pantheon.tech>
jsonWriter.beginObject().name("Entries");
jsonWriter.beginArray();
for (var entry : entries) {
- final var data = entry.getData();
+ final var data = entry.command();
if (data instanceof CommitTransactionPayload payload) {
final var candidate = payload.getCandidate().candidate();
writeNode(jsonWriter, candidate);
// We persist the ClusterConfig but not the ReplicatedLogEntry to avoid gaps in the journal indexes
// on recovery if data persistence is later enabled.
if (!delegate.isRecoveryApplicable() && obj instanceof ReplicatedLogEntry entry
- && entry.getData() instanceof ClusterConfig serverConfig) {
+ && entry.command() instanceof ClusterConfig serverConfig) {
method.invoke(enabledStorage, serverConfig, p -> callback.accept(obj));
} else {
method.invoke(delegate, obj, callback);
final long startTime = System.nanoTime();
final var entry = applyState.getReplicatedLogEntry();
- final var payload = entry.getData();
+ final var payload = entry.command();
if (LOG.isDebugEnabled()) {
LOG.debug("{}: Applying state for log index {} data {}", memberId(), entry.index(), payload);
}
logEntry.index(), logEntry.size());
}
- final var data = logEntry.getData();
- if (isMigratedSerializable(data)) {
+ final var command = logEntry.command();
+ if (isMigratedSerializable(command)) {
hasMigratedDataRecovered = true;
}
- if (data instanceof ClusterConfig clusterConfig) {
+ if (command instanceof ClusterConfig clusterConfig) {
context.updatePeerIds(clusterConfig);
}
if (context.getPersistenceProvider().isRecoveryApplicable()) {
replicatedLog().append(logEntry);
- } else if (!(data instanceof ClusterConfig)) {
+ } else if (!(command instanceof ClusterConfig)) {
dataRecoveredWithPersistenceDisabled = true;
}
}
private void batchRecoveredLogEntry(final ReplicatedLogEntry logEntry) {
initRecoveryTimers();
- if (logEntry.getData() instanceof ClusterConfig) {
+ if (logEntry.command() instanceof ClusterConfig) {
// FIXME: explain why ClusterConfig is special
return;
}
cohort.startLogRecoveryBatch(batchSize);
}
- cohort.appendRecoveredLogEntry(logEntry.getData());
+ cohort.appendRecoveredLogEntry(logEntry.command());
if (++currentRecoveryBatchCount >= batchSize) {
endCurrentLogRecoveryBatch();
}
private static boolean isMigratedPayload(final ReplicatedLogEntry repLogEntry) {
- return isMigratedSerializable(repLogEntry.getData());
+ return isMigratedSerializable(repLogEntry.command());
}
private static boolean isMigratedSerializable(final Object message) {
import org.opendaylight.controller.cluster.raft.messages.AddServer;
import org.opendaylight.controller.cluster.raft.messages.AddServerReply;
import org.opendaylight.controller.cluster.raft.messages.ChangeServersVotingStatus;
-import org.opendaylight.controller.cluster.raft.messages.Payload;
import org.opendaylight.controller.cluster.raft.messages.RemoveServer;
import org.opendaylight.controller.cluster.raft.messages.RemoveServerReply;
import org.opendaylight.controller.cluster.raft.messages.ServerChangeReply;
}
private boolean onApplyState(final ApplyState applyState) {
- Payload data = applyState.getReplicatedLogEntry().getData();
- if (data instanceof ClusterConfig) {
+ if (applyState.getReplicatedLogEntry().command() instanceof ClusterConfig) {
currentOperationState.onApplyState(applyState);
return true;
}
// sure it's meant for us.
if (operationContext.getContextId().equals(applyState.getIdentifier())) {
LOG.info("{}: {} has been successfully replicated to a majority of followers", memberId(),
- applyState.getReplicatedLogEntry().getData());
+ applyState.getReplicatedLogEntry().command());
timer.cancel();
operationComplete(operationContext, null);
*/
@NonNullByDefault
public interface ReplicatedLogEntry extends LogEntry {
- /**
- * Returns the payload/data to be replicated.
- *
- * @return the payload/data
- */
- @Deprecated(forRemoval = true)
- default Payload getData() {
- return command();
- }
-
@Override
Payload command();
// Tracker is missing, this means that we switched behaviours between replicate and applystate
// and became the leader again,. We still want to apply this as a local modification because
// we have resumed leadership with that log entry having been committed.
- if (entry.getData() instanceof IdentifiablePayload<?> identifiable) {
+ if (entry.command() instanceof IdentifiablePayload<?> identifiable) {
return new ApplyState(null, identifiable.getIdentifier(), entry);
}
// If the first entry's size exceeds the max data size threshold, it will be returned from the call above. If
// that is the case, then we need to slice it into smaller chunks.
- if (entries.size() != 1 || entries.getFirst().getData().serializedSize() <= maxDataSize) {
+ if (entries.size() != 1 || entries.getFirst().command().serializedSize() <= maxDataSize) {
// Don't need to slice.
return entries;
}
final var firstEntry = entries.getFirst();
- LOG.debug("{}: Log entry size {} exceeds max payload size {}", logName, firstEntry.getData().size(),
+ LOG.debug("{}: Log entry size {} exceeds max payload size {}", logName, firstEntry.command().size(),
maxDataSize);
// If an AppendEntries has already been serialized for the log index then reuse the
for (int i = addEntriesFrom; i < numLogEntries; i++) {
final var entry = entries.get(i);
- LOG.debug("{}: Append entry to log {}", logName, entry.getData());
+ LOG.debug("{}: Append entry to log {}", logName, entry.command());
replLog.appendAndPersist(entry, appendAndPersistCallback, false);
shouldCaptureSnapshot.compareAndSet(false, replLog.shouldCaptureSnapshot(entry.index()));
- if (entry.getData() instanceof ClusterConfig serverConfiguration) {
+ if (entry.command() instanceof ClusterConfig serverConfiguration) {
context.updatePeerIds(serverConfiguration);
}
}
out.writeInt(entries.size());
for (var e : entries) {
WritableObjects.writeLongs(out, e.index(), e.term());
- out.writeObject(e.getData());
+ out.writeObject(e.command());
}
out.writeObject(appendEntries.leaderAddress());
for (ReplicatedLogEntry e: appendEntries.entries) {
out.writeLong(e.index());
out.writeLong(e.term());
- out.writeObject(e.getData());
+ out.writeObject(e.command());
}
out.writeObject(appendEntries.leaderAddress);
LE(final SimpleReplicatedLogEntry logEntry) {
index = logEntry.index();
term = logEntry.term();
- data = logEntry.getData();
+ data = logEntry.command();
}
@Override
out.writeInt(unAppliedEntries.size());
for (var e : unAppliedEntries) {
WritableObjects.writeLongs(out, e.index(), e.term());
- out.writeObject(e.getData());
+ out.writeObject(e.command());
}
out.writeObject(snapshot.getState());
final byte[] bytes = baos.toByteArray();
if (LOG.isDebugEnabled()) {
- final var data = replicatedLogEntry.getData();
+ final var data = replicatedLogEntry.command();
LOG.debug("Estimated serialized size {}, data size {} for payload: {}. Actual serialized size: {}",
estimatedSerializedSize, data.size(), data, bytes.length);
}
try (var oos = new ObjectOutputStream(entryCompress.encodeOutput(dos))) {
for (var entry : unappliedEntries) {
oos.writeLong(entry.term());
- oos.writeObject(entry.getData());
+ oos.writeObject(entry.command());
}
}
}
for (int i = 0; i < expJournal.size(); i++) {
final var expected = expJournal.get(i);
final var actual = journal.get(i);
- verifyReplicatedLogEntry(expected, actual.term(), actual.index(), actual.getData());
+ verifyReplicatedLogEntry(expected, actual.term(), actual.index(), actual.command());
}
}
protected void verifyReplicatedLogEntry(final ReplicatedLogEntry replicatedLogEntry, final long expTerm,
final long expIndex, final Payload payload) {
- assertEquals("ReplicatedLogEntry getTerm", expTerm, replicatedLogEntry.term());
- assertEquals("ReplicatedLogEntry getIndex", expIndex, replicatedLogEntry.index());
- assertEquals("ReplicatedLogEntry getData", payload, replicatedLogEntry.getData());
+ assertEquals(expTerm, replicatedLogEntry.term());
+ assertEquals(expIndex, replicatedLogEntry.index());
+ assertEquals(payload, replicatedLogEntry.command());
}
protected String testActorPath(final String id) {
@Test
void testIndexOperations() {
// check if the values returned are correct, with snapshotIndex = -1
- assertEquals("B", replicatedLogImpl.get(1).getData().toString());
- assertEquals("D", replicatedLogImpl.last().getData().toString());
+ assertEquals("B", replicatedLogImpl.get(1).command().toString());
+ assertEquals("D", replicatedLogImpl.last().command().toString());
assertEquals(3, replicatedLogImpl.lastIndex());
assertEquals(2, replicatedLogImpl.lastTerm());
assertEquals(2, replicatedLogImpl.getFrom(2).size());
// check the values after the snapshot.
// each index value passed in the test is the logical index (log entry index)
// which gets mapped to the list's physical index
- assertEquals("D", replicatedLogImpl.get(3).getData().toString());
- assertEquals("D", replicatedLogImpl.last().getData().toString());
+ assertEquals("D", replicatedLogImpl.get(3).command().toString());
+ assertEquals("D", replicatedLogImpl.last().command().toString());
assertNull(replicatedLogImpl.get(1));
assertEquals(3, replicatedLogImpl.lastIndex());
assertEquals(2, replicatedLogImpl.lastTerm());
// check their values as well
assertEquals(5, replicatedLogImpl.size());
- assertEquals("D", replicatedLogImpl.get(3).getData().toString());
- assertEquals("E", replicatedLogImpl.get(4).getData().toString());
- assertEquals("H", replicatedLogImpl.last().getData().toString());
+ assertEquals("D", replicatedLogImpl.get(3).command().toString());
+ assertEquals("E", replicatedLogImpl.get(4).command().toString());
+ assertEquals("H", replicatedLogImpl.last().command().toString());
assertEquals(3, replicatedLogImpl.lastTerm());
assertEquals(7, replicatedLogImpl.lastIndex());
assertTrue(replicatedLogImpl.isPresent(7));
void testGetFromWithMax() {
var from = replicatedLogImpl.getFrom(0, 1, ReplicatedLog.NO_MAX_SIZE);
assertEquals(1, from.size());
- assertEquals("A", from.get(0).getData().toString());
+ assertEquals("A", from.get(0).command().toString());
from = replicatedLogImpl.getFrom(0, 20, ReplicatedLog.NO_MAX_SIZE);
assertEquals(4, from.size());
- assertEquals("A", from.get(0).getData().toString());
- assertEquals("B", from.get(1).getData().toString());
- assertEquals("C", from.get(2).getData().toString());
- assertEquals("D", from.get(3).getData().toString());
+ assertEquals("A", from.get(0).command().toString());
+ assertEquals("B", from.get(1).command().toString());
+ assertEquals("C", from.get(2).command().toString());
+ assertEquals("D", from.get(3).command().toString());
// Pre-calculate sizing information for use with capping
final int sizeB = from.get(1).serializedSize();
from = replicatedLogImpl.getFrom(1, 2, ReplicatedLog.NO_MAX_SIZE);
assertEquals(2, from.size());
- assertEquals("B", from.get(0).getData().toString());
- assertEquals("C", from.get(1).getData().toString());
+ assertEquals("B", from.get(0).command().toString());
+ assertEquals("C", from.get(1).command().toString());
from = replicatedLogImpl.getFrom(1, 3, sizeB + sizeC);
assertEquals(2, from.size());
- assertEquals("B", from.get(0).getData().toString());
- assertEquals("C", from.get(1).getData().toString());
+ assertEquals("B", from.get(0).command().toString());
+ assertEquals("C", from.get(1).command().toString());
from = replicatedLogImpl.getFrom(1, 3, sizeB + sizeC + sizeD);
assertEquals(3, from.size());
- assertEquals("B", from.get(0).getData().toString());
- assertEquals("C", from.get(1).getData().toString());
- assertEquals("D", from.get(2).getData().toString());
+ assertEquals("B", from.get(0).command().toString());
+ assertEquals("C", from.get(1).command().toString());
+ assertEquals("D", from.get(2).command().toString());
from = replicatedLogImpl.getFrom(1, 2, sizeB + sizeC + sizeD);
assertEquals(2, from.size());
- assertEquals("B", from.get(0).getData().toString());
- assertEquals("C", from.get(1).getData().toString());
+ assertEquals("B", from.get(0).command().toString());
+ assertEquals("C", from.get(1).command().toString());
replicatedLogImpl.append(new SimpleReplicatedLogEntry(4, 2, new MockPayload("12345")));
from = replicatedLogImpl.getFrom(4, 2, 2);
assertEquals(1, from.size());
- assertEquals("12345", from.get(0).getData().toString());
+ assertEquals("12345", from.get(0).command().toString());
}
@Test
long lastTerm = 0;
for (int i = 0; i < numEntries; i++) {
final var entry = replicatedLogImpl.getAtPhysicalIndex(i);
- map.put(entry.index(), entry.getData().toString());
+ map.put(entry.index(), entry.command().toString());
lastIndex = entry.index();
lastTerm = entry.term();
}
// message is forwarded to the followers.
expectFirstMatching(follower1CollectorActor, AppendEntries.class, ae ->
- ae.getEntries().size() == 1 && ae.getEntries().get(0).index() == 1
- && ae.getEntries().get(0).getData().equals(payload1));
+ ae.getEntries().size() == 1 && ae.getEntries().getFirst().index() == 1
+ && ae.getEntries().getFirst().command().equals(payload1));
expectFirstMatching(follower2CollectorActor, AppendEntries.class, ae ->
- ae.getEntries().size() == 1 && ae.getEntries().get(0).index() == 1
- && ae.getEntries().get(0).getData().equals(payload1));
+ ae.getEntries().size() == 1 && ae.getEntries().getFirst().index() == 1
+ && ae.getEntries().getFirst().command().equals(payload1));
verifyApplyJournalEntries(leaderCollectorActor, 1);
// message is forwarded to the followers.
expectFirstMatching(follower1CollectorActor, AppendEntries.class, ae ->
- ae.getEntries().size() == 1 && ae.getEntries().get(0).index() == 1
- && ae.getEntries().get(0).getData().equals(payload1));
+ ae.getEntries().size() == 1 && ae.getEntries().getFirst().index() == 1
+ && ae.getEntries().getFirst().command().equals(payload1));
expectFirstMatching(follower2CollectorActor, AppendEntries.class, ae ->
- ae.getEntries().size() == 1 && ae.getEntries().get(0).index() == 1
- && ae.getEntries().get(0).getData().equals(payload1));
+ ae.getEntries().size() == 1 && ae.getEntries().getFirst().index() == 1
+ && ae.getEntries().getFirst().command().equals(payload1));
verifyApplyJournalEntries(leaderCollectorActor, 1);
@Before
public void setup() {
- doReturn(PERSISTENT_PAYLOAD).when(mockPersistentLogEntry).getData();
- doReturn(NON_PERSISTENT_PAYLOAD).when(mockNonPersistentLogEntry).getData();
+ doReturn(PERSISTENT_PAYLOAD).when(mockPersistentLogEntry).command();
+ doReturn(NON_PERSISTENT_PAYLOAD).when(mockNonPersistentLogEntry).command();
provider = new PersistenceControl(mockDisabledStorage, mockEnabledStorage);
}
inOrder.verify(mockCohort).startLogRecoveryBatch(5);
for (int i = 0; i < replicatedLog.size() - 1; i++) {
- inOrder.verify(mockCohort).appendRecoveredLogEntry(replicatedLog.get(i).getData());
+ inOrder.verify(mockCohort).appendRecoveredLogEntry(replicatedLog.get(i).command());
}
inOrder.verify(mockCohort).applyCurrentLogRecoveryBatch();
inOrder.verify(mockCohort).startLogRecoveryBatch(5);
- inOrder.verify(mockCohort).appendRecoveredLogEntry(replicatedLog.get(replicatedLog.size() - 1).getData());
+ inOrder.verify(mockCohort).appendRecoveredLogEntry(replicatedLog.get(replicatedLog.size() - 1).command());
inOrder.verifyNoMoreInteractions();
}
inOrder.verify(mockCohort).startLogRecoveryBatch(anyInt());
for (int i = 0; i < replicatedLog.size(); i++) {
- inOrder.verify(mockCohort).appendRecoveredLogEntry(replicatedLog.get(i).getData());
+ inOrder.verify(mockCohort).appendRecoveredLogEntry(replicatedLog.get(i).command());
}
inOrder.verify(mockCohort).applyCurrentLogRecoveryBatch();
private static void verifyServerConfigurationPayloadEntry(final ReplicatedLog log, final ServerInfo... expected) {
ReplicatedLogEntry logEntry = log.get(log.lastIndex());
- final var payload = assertInstanceOf(ClusterConfig.class, logEntry.getData());
+ final var payload = assertInstanceOf(ClusterConfig.class, logEntry.command());
assertEquals("Server config", Set.of(expected), Set.copyOf(payload.serverInfo()));
}
for (int i = 0; i < fromLog.size(); i++) {
final var entry = fromLog.get(i);
- getState().add(entry.getData());
+ getState().add(entry.command());
toLog.append(entry);
}
final var log = leaderActor.underlyingActor().getReplicatedLog();
assertEquals("Leader last index", 5, log.lastIndex());
- assertEquals(List.of(payload4, payload5), List.of(log.get(4).getData(), log.get(5).getData()));
+ assertEquals(List.of(payload4, payload5), List.of(log.get(4).command(), log.get(5).command()));
}
private void reinstateLeaderActor() {
assertEquals("Next index", 3, log.lastIndex());
//assertEquals("Entry 2", entries.get(0), log.get(2));
- assertEquals("Entry 1 data", "one", log.get(1).getData().toString());
+ assertEquals("Entry 1 data", "one", log.get(1).command().toString());
// Check that the entry at index 2 has the new data
assertEquals("Entry 2", entries.get(0), log.get(2));
assertEquals("Snapshot getLastAppliedIndex", 1, snapshot.getLastAppliedIndex());
assertEquals("Snapshot getLastTerm", 1, snapshot.getLastTerm());
assertEquals("Snapshot getLastIndex", 1, snapshot.getLastIndex());
- assertEquals("Snapshot state", List.of(entries.get(0).getData(), entries.get(1).getData()),
+ assertEquals("Snapshot state", List.of(entries.get(0).command(), entries.get(1).command()),
MockRaftActor.fromState(snapshot.getState()));
}
assertEquals("Snapshot getLastAppliedIndex", 2, snapshot.getLastAppliedIndex());
assertEquals("Snapshot getLastTerm", 1, snapshot.getLastTerm());
assertEquals("Snapshot getLastIndex", 2, snapshot.getLastIndex());
- assertEquals("Snapshot state", List.of(entries.get(0).getData(), entries.get(1).getData(),
- entries.get(2).getData()), MockRaftActor.fromState(snapshot.getState()));
+ assertEquals("Snapshot state", List.of(entries.get(0).command(), entries.get(1).command(),
+ entries.get(2).command()), MockRaftActor.fromState(snapshot.getState()));
assertEquals("Journal size", 0, followerRaftActor.get().getReplicatedLog().size());
assertEquals("Snapshot index", 2, followerRaftActor.get().getReplicatedLog().getSnapshotIndex());
assertEquals("Last index", 2, followerLog.lastIndex());
assertEquals("Last applied index", 2, followerLog.getLastApplied());
assertEquals("Commit index", 2, followerLog.getCommitIndex());
- assertEquals("State", List.of(entries.get(0).getData(), entries.get(1).getData(), entries.get(2).getData()),
+ assertEquals("State", List.of(entries.get(0).command(), entries.get(1).command(), entries.get(2).command()),
followerRaftActor.get().getState());
}
assertEquals("Snapshot getLastAppliedIndex", 0, snapshot.getLastAppliedIndex());
assertEquals("Snapshot getLastTerm", 1, snapshot.getLastTerm());
assertEquals("Snapshot getLastIndex", 2, snapshot.getLastIndex());
- assertEquals("Snapshot state", List.of(entries.get(0).getData()),
+ assertEquals("Snapshot state", List.of(entries.get(0).command()),
MockRaftActor.fromState(snapshot.getState()));
}
assertEquals("Entries size", 1, appendEntries.getEntries().size());
assertEquals("Entry getIndex", lastIndex + 1, appendEntries.getEntries().get(0).index());
assertEquals("Entry getTerm", term, appendEntries.getEntries().get(0).term());
- assertEquals("Entry payload", "foo", appendEntries.getEntries().get(0).getData().toString());
+ assertEquals("Entry payload", "foo", appendEntries.getEntries().get(0).command().toString());
assertEquals("Commit Index", lastIndex, log.getCommitIndex());
}
assertEquals("Entries size", 1, appendEntries.getEntries().size());
assertEquals("Entry getIndex", newIndex, appendEntries.getEntries().get(0).index());
assertEquals("Entry getTerm", newTerm, appendEntries.getEntries().get(0).term());
- assertEquals("Entry payload", "foo", appendEntries.getEntries().get(0).getData().toString());
+ assertEquals("Entry payload", "foo", appendEntries.getEntries().get(0).command().toString());
// The follower replies with success. The leader should now update the commit index to the new index
// as per ยง5.4.1 "once an entry from the current term is committed by counting replicas, then all
assertEquals("Entries size", 1, appendEntries.getEntries().size());
assertEquals("Entry getIndex", lastIndex + 1, appendEntries.getEntries().get(0).index());
assertEquals("Entry getTerm", term, appendEntries.getEntries().get(0).term());
- assertEquals("Entry payload", "foo", appendEntries.getEntries().get(0).getData().toString());
+ assertEquals("Entry payload", "foo", appendEntries.getEntries().get(0).command().toString());
assertEquals("Commit Index", lastIndex + 1, log.getCommitIndex());
}
}
ApplyState last = applyStateList.get((int) newLogIndex - 1);
- assertEquals("getData", data, last.getReplicatedLogEntry().getData());
+ assertEquals("getData", data, last.getReplicatedLogEntry().command());
assertEquals("getIdentifier", identifier, last.getIdentifier());
}
assertEquals("Log entries size", 2, appendEntries.getEntries().size());
assertEquals("First entry index", 1, appendEntries.getEntries().get(0).index());
- assertEquals("First entry data", leadersSecondLogEntry.getData(),
- appendEntries.getEntries().get(0).getData());
+ assertEquals("First entry data", leadersSecondLogEntry.command(),
+ appendEntries.getEntries().get(0).command());
assertEquals("Second entry index", 2, appendEntries.getEntries().get(1).index());
- assertEquals("Second entry data", leadersThirdLogEntry.getData(),
- appendEntries.getEntries().get(1).getData());
+ assertEquals("Second entry data", leadersThirdLogEntry.command(),
+ appendEntries.getEntries().get(1).command());
FollowerLogInformation followerInfo = leader.getFollower(FOLLOWER_ID);
assertEquals("getNextIndex", 3, followerInfo.getNextIndex());
- List<ApplyState> applyStateList = MessageCollectorActor.expectMatching(followerActor, ApplyState.class, 2);
+ final var applyStateList = MessageCollectorActor.expectMatching(followerActor, ApplyState.class, 2);
ApplyState applyState = applyStateList.get(0);
assertEquals("Follower's first ApplyState index", 1, applyState.getReplicatedLogEntry().index());
assertEquals("Follower's first ApplyState term", 1, applyState.getReplicatedLogEntry().term());
- assertEquals("Follower's first ApplyState data", leadersSecondLogEntry.getData(),
- applyState.getReplicatedLogEntry().getData());
+ assertEquals("Follower's first ApplyState data", leadersSecondLogEntry.command(),
+ applyState.getReplicatedLogEntry().command());
applyState = applyStateList.get(1);
assertEquals("Follower's second ApplyState index", 2, applyState.getReplicatedLogEntry().index());
assertEquals("Follower's second ApplyState term", 1, applyState.getReplicatedLogEntry().term());
- assertEquals("Follower's second ApplyState data", leadersThirdLogEntry.getData(),
- applyState.getReplicatedLogEntry().getData());
+ assertEquals("Follower's second ApplyState data", leadersThirdLogEntry.command(),
+ applyState.getReplicatedLogEntry().command());
followerLog = followerActorContext.getReplicatedLog();
assertEquals("Follower's commit index", 2, followerLog.getCommitIndex());
assertEquals("Log entries size", 2, appendEntries.getEntries().size());
assertEquals("First entry index", 0, appendEntries.getEntries().get(0).index());
- assertEquals("First entry data", leadersFirstLogEntry.getData(),
- appendEntries.getEntries().get(0).getData());
+ assertEquals("First entry data", leadersFirstLogEntry.command(),
+ appendEntries.getEntries().get(0).command());
assertEquals("Second entry index", 1, appendEntries.getEntries().get(1).index());
- assertEquals("Second entry data", leadersSecondLogEntry.getData(),
- appendEntries.getEntries().get(1).getData());
+ assertEquals("Second entry data", leadersSecondLogEntry.command(),
+ appendEntries.getEntries().get(1).command());
FollowerLogInformation followerInfo = leader.getFollower(FOLLOWER_ID);
assertEquals("getNextIndex", 2, followerInfo.getNextIndex());
ApplyState applyState = applyStateList.get(0);
assertEquals("Follower's first ApplyState index", 0, applyState.getReplicatedLogEntry().index());
assertEquals("Follower's first ApplyState term", 1, applyState.getReplicatedLogEntry().term());
- assertEquals("Follower's first ApplyState data", leadersFirstLogEntry.getData(),
- applyState.getReplicatedLogEntry().getData());
+ assertEquals("Follower's first ApplyState data", leadersFirstLogEntry.command(),
+ applyState.getReplicatedLogEntry().command());
applyState = applyStateList.get(1);
assertEquals("Follower's second ApplyState index", 1, applyState.getReplicatedLogEntry().index());
assertEquals("Follower's second ApplyState term", 1, applyState.getReplicatedLogEntry().term());
- assertEquals("Follower's second ApplyState data", leadersSecondLogEntry.getData(),
- applyState.getReplicatedLogEntry().getData());
+ assertEquals("Follower's second ApplyState data", leadersSecondLogEntry.command(),
+ applyState.getReplicatedLogEntry().command());
final var followerLog = followerActorContext.getReplicatedLog();
assertEquals("Follower's commit index", 1, followerLog.getCommitIndex());
assertEquals("First entry index", 0, appendEntries.getEntries().get(0).index());
assertEquals("First entry term", 2, appendEntries.getEntries().get(0).term());
- assertEquals("First entry data", leadersFirstLogEntry.getData(),
- appendEntries.getEntries().get(0).getData());
+ assertEquals("First entry data", leadersFirstLogEntry.command(),
+ appendEntries.getEntries().get(0).command());
assertEquals("Second entry index", 1, appendEntries.getEntries().get(1).index());
assertEquals("Second entry term", 2, appendEntries.getEntries().get(1).term());
- assertEquals("Second entry data", leadersSecondLogEntry.getData(),
- appendEntries.getEntries().get(1).getData());
+ assertEquals("Second entry data", leadersSecondLogEntry.command(),
+ appendEntries.getEntries().get(1).command());
FollowerLogInformation followerInfo = leader.getFollower(FOLLOWER_ID);
assertEquals("getNextIndex", 2, followerInfo.getNextIndex());
ApplyState applyState = applyStateList.get(0);
assertEquals("Follower's first ApplyState index", 0, applyState.getReplicatedLogEntry().index());
assertEquals("Follower's first ApplyState term", 2, applyState.getReplicatedLogEntry().term());
- assertEquals("Follower's first ApplyState data", leadersFirstLogEntry.getData(),
- applyState.getReplicatedLogEntry().getData());
+ assertEquals("Follower's first ApplyState data", leadersFirstLogEntry.command(),
+ applyState.getReplicatedLogEntry().command());
applyState = applyStateList.get(1);
assertEquals("Follower's second ApplyState index", 1, applyState.getReplicatedLogEntry().index());
assertEquals("Follower's second ApplyState term", 2, applyState.getReplicatedLogEntry().term());
- assertEquals("Follower's second ApplyState data", leadersSecondLogEntry.getData(),
- applyState.getReplicatedLogEntry().getData());
+ assertEquals("Follower's second ApplyState data", leadersSecondLogEntry.command(),
+ applyState.getReplicatedLogEntry().command());
final var followerLog = followerActorContext.getReplicatedLog();
assertEquals("Follower's commit index", 1, followerLog.getCommitIndex());
assertEquals("Log entries size", 2, appendEntries.getEntries().size());
assertEquals("First entry index", 0, appendEntries.getEntries().get(0).index());
- assertEquals("First entry data", leadersFirstLogEntry.getData(),
- appendEntries.getEntries().get(0).getData());
+ assertEquals("First entry data", leadersFirstLogEntry.command(),
+ appendEntries.getEntries().get(0).command());
assertEquals("Second entry index", 1, appendEntries.getEntries().get(1).index());
- assertEquals("Second entry data", leadersSecondLogEntry.getData(),
- appendEntries.getEntries().get(1).getData());
+ assertEquals("Second entry data", leadersSecondLogEntry.command(),
+ appendEntries.getEntries().get(1).command());
appendEntries = appendEntriesList.get(1);
assertEquals("getLeaderCommit", leaderCommitIndex, appendEntries.getLeaderCommit());
assertEquals("Log entries size", 2, appendEntries.getEntries().size());
assertEquals("First entry index", 2, appendEntries.getEntries().get(0).index());
- assertEquals("First entry data", leadersThirdLogEntry.getData(),
- appendEntries.getEntries().get(0).getData());
+ assertEquals("First entry data", leadersThirdLogEntry.command(),
+ appendEntries.getEntries().get(0).command());
assertEquals("Second entry index", 3, appendEntries.getEntries().get(1).index());
- assertEquals("Second entry data", leadersFourthLogEntry.getData(),
- appendEntries.getEntries().get(1).getData());
+ assertEquals("Second entry data", leadersFourthLogEntry.command(),
+ appendEntries.getEntries().get(1).command());
FollowerLogInformation followerInfo = leader.getFollower(FOLLOWER_ID);
assertEquals("getNextIndex", 4, followerInfo.getNextIndex());
private static void assertReplicatedLogEntry(final ReplicatedLogEntry expected, final ReplicatedLogEntry actual) {
assertEquals(expected.index(), actual.index());
assertEquals(expected.term(), actual.term());
- assertEquals(expected.getData().toString(), actual.getData().toString());
+ assertEquals(expected.command().toString(), actual.command().toString());
}
}
*/
package org.opendaylight.controller.cluster.raft.persisted;
-import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.assertNotNull;
+import static org.junit.jupiter.api.Assertions.assertEquals;
+import static org.junit.jupiter.api.Assertions.assertInstanceOf;
import java.io.NotSerializableException;
import org.apache.pekko.actor.ExtendedActorSystem;
import org.apache.pekko.testkit.javadsl.TestKit;
-import org.junit.Test;
+import org.junit.jupiter.api.Test;
import org.opendaylight.controller.cluster.raft.MockRaftActorContext;
/**
*
* @author Thomas Pantelis
*/
-public class SimpleReplicatedLogEntrySerializerTest {
-
+class SimpleReplicatedLogEntrySerializerTest {
@Test
- public void testToAndFromBinary() throws NotSerializableException {
- SimpleReplicatedLogEntry expected = new SimpleReplicatedLogEntry(0, 1,
- new MockRaftActorContext.MockPayload("A"));
+ void testToAndFromBinary() throws NotSerializableException {
+ final var expected = new SimpleReplicatedLogEntry(0, 1, new MockRaftActorContext.MockPayload("A"));
- final ExtendedActorSystem system = (ExtendedActorSystem) ExtendedActorSystem.create("test");
- final Object deserialized;
+ final var system = (ExtendedActorSystem) ExtendedActorSystem.create("test");
+ final SimpleReplicatedLogEntry actual;
try {
- final SimpleReplicatedLogEntrySerializer serializer = new SimpleReplicatedLogEntrySerializer(system);
+ final var serializer = new SimpleReplicatedLogEntrySerializer(system);
final byte[] bytes = serializer.toBinary(expected);
- deserialized = serializer.fromBinary(bytes, SimpleReplicatedLogEntry.class);
+ actual = assertInstanceOf(SimpleReplicatedLogEntry.class,
+ serializer.fromBinary(bytes, SimpleReplicatedLogEntry.class));
} finally {
TestKit.shutdownActorSystem(system);
}
- assertNotNull("fromBinary returned null", deserialized);
- assertEquals("fromBinary return type", SimpleReplicatedLogEntry.class, deserialized.getClass());
-
- SimpleReplicatedLogEntry actual = (SimpleReplicatedLogEntry)deserialized;
- assertEquals("getTerm", expected.term(), actual.term());
- assertEquals("getIndex", expected.index(), actual.index());
- assertEquals("getData", expected.getData(), actual.getData());
+ assertEquals(expected.term(), actual.term());
+ assertEquals(expected.index(), actual.index());
+ assertEquals(expected.command(), actual.command());
}
}
*/
package org.opendaylight.controller.cluster.raft.persisted;
-import static org.junit.Assert.assertEquals;
+import static org.junit.jupiter.api.Assertions.assertEquals;
import org.apache.commons.lang3.SerializationUtils;
-import org.junit.Test;
+import org.junit.jupiter.api.Test;
import org.opendaylight.controller.cluster.raft.MockRaftActorContext;
/**
*
* @author Thomas Pantelis
*/
-public class SimpleReplicatedLogEntryTest {
+class SimpleReplicatedLogEntryTest {
@Test
- public void testSerialization() {
+ void testSerialization() {
final var expected = new SimpleReplicatedLogEntry(0, 1, new MockRaftActorContext.MockPayload("A"));
final var bytes = SerializationUtils.serialize(expected);
assertEquals(218, bytes.length);
final var cloned = SerializationUtils.<SimpleReplicatedLogEntry>deserialize(bytes);
- assertEquals("getTerm", expected.term(), cloned.term());
- assertEquals("getIndex", expected.index(), cloned.index());
- assertEquals("getData", expected.getData(), cloned.getData());
+ assertEquals(expected.term(), cloned.term());
+ assertEquals(expected.index(), cloned.index());
+ assertEquals(expected.command(), cloned.command());
}
}