Browse Source

MINOR; Add accessor methods to OffsetAndEpoch (#12770)

Accessor are preferred over fields because they compose better with Java's
lambda syntax.

Reviewers: Jason Gustafson <jason@confluent.io>
pull/12774/head
José Armando García Sancio 2 years ago committed by GitHub
parent
commit
d0ff869718
No known key found for this signature in database
GPG Key ID: 4AEE18F83AFDEB23
  1. 4
      metadata/src/main/java/org/apache/kafka/image/MetadataDelta.java
  2. 4
      metadata/src/test/java/org/apache/kafka/image/MetadataImageTest.java
  3. 4
      metadata/src/test/java/org/apache/kafka/image/writer/RaftSnapshotWriterTest.java
  4. 4
      metadata/src/test/java/org/apache/kafka/metadata/RecordTestUtils.java
  5. 6
      metadata/src/test/java/org/apache/kafka/metalog/LocalLogManager.java
  6. 30
      raft/src/main/java/org/apache/kafka/raft/KafkaRaftClient.java
  7. 12
      raft/src/main/java/org/apache/kafka/raft/OffsetAndEpoch.java
  8. 6
      raft/src/main/java/org/apache/kafka/raft/QuorumState.java
  9. 16
      raft/src/main/java/org/apache/kafka/raft/ReplicatedLog.java
  10. 4
      raft/src/main/java/org/apache/kafka/raft/internals/KafkaRaftMetrics.java
  11. 4
      raft/src/main/java/org/apache/kafka/snapshot/RecordsSnapshotReader.java
  12. 8
      raft/src/main/java/org/apache/kafka/snapshot/RecordsSnapshotWriter.java
  13. 2
      raft/src/main/java/org/apache/kafka/snapshot/Snapshots.java
  14. 206
      raft/src/test/java/org/apache/kafka/raft/KafkaRaftClientSnapshotTest.java
  15. 32
      raft/src/test/java/org/apache/kafka/raft/MockLog.java
  16. 30
      raft/src/test/java/org/apache/kafka/raft/MockLogTest.java
  17. 4
      raft/src/test/java/org/apache/kafka/raft/RaftClientTestContext.java
  18. 16
      raft/src/test/java/org/apache/kafka/raft/RaftEventSimulationTest.java
  19. 14
      raft/src/test/java/org/apache/kafka/snapshot/SnapshotWriterReaderTest.java

4
metadata/src/main/java/org/apache/kafka/image/MetadataDelta.java

@ -71,8 +71,8 @@ public final class MetadataDelta { @@ -71,8 +71,8 @@ public final class MetadataDelta {
public MetadataDelta(MetadataImage image) {
this.image = image;
this.highestOffset = image.highestOffsetAndEpoch().offset;
this.highestEpoch = image.highestOffsetAndEpoch().epoch;
this.highestOffset = image.highestOffsetAndEpoch().offset();
this.highestEpoch = image.highestOffsetAndEpoch().epoch();
}
public MetadataImage image() {

4
metadata/src/test/java/org/apache/kafka/image/MetadataImageTest.java

@ -91,8 +91,8 @@ public class MetadataImageTest { @@ -91,8 +91,8 @@ public class MetadataImageTest {
image.write(writer, new ImageWriterOptions.Builder().build());
MetadataDelta delta = new MetadataDelta(MetadataImage.EMPTY);
RecordTestUtils.replayAll(delta,
image.highestOffsetAndEpoch().offset,
image.highestOffsetAndEpoch().epoch,
image.highestOffsetAndEpoch().offset(),
image.highestOffsetAndEpoch().epoch(),
writer.records());
MetadataImage nextImage = delta.apply();
assertEquals(image, nextImage);

4
metadata/src/test/java/org/apache/kafka/image/writer/RaftSnapshotWriterTest.java

@ -48,12 +48,12 @@ public class RaftSnapshotWriterTest { @@ -48,12 +48,12 @@ public class RaftSnapshotWriterTest {
@Override
public long lastContainedLogOffset() {
return snapshotId().offset;
return snapshotId().offset();
}
@Override
public int lastContainedLogEpoch() {
return snapshotId().epoch;
return snapshotId().epoch();
}
@Override

4
metadata/src/test/java/org/apache/kafka/metadata/RecordTestUtils.java

@ -60,8 +60,8 @@ public class RecordTestUtils { @@ -60,8 +60,8 @@ public class RecordTestUtils {
if (target instanceof MetadataDelta) {
MetadataDelta delta = (MetadataDelta) target;
replayAll(delta,
delta.image().highestOffsetAndEpoch().offset,
delta.image().highestOffsetAndEpoch().epoch,
delta.image().highestOffsetAndEpoch().offset(),
delta.image().highestOffsetAndEpoch().epoch(),
recordsAndVersions);
return;
}

6
metadata/src/test/java/org/apache/kafka/metalog/LocalLogManager.java

@ -198,7 +198,7 @@ public final class LocalLogManager implements RaftClient<ApiMessageAndVersion>, @@ -198,7 +198,7 @@ public final class LocalLogManager implements RaftClient<ApiMessageAndVersion>,
public SharedLogData(Optional<RawSnapshotReader> snapshot) {
if (snapshot.isPresent()) {
RawSnapshotReader initialSnapshot = snapshot.get();
prevOffset = initialSnapshot.snapshotId().offset - 1;
prevOffset = initialSnapshot.snapshotId().offset() - 1;
snapshots.put(prevOffset, initialSnapshot);
} else {
prevOffset = -1;
@ -305,14 +305,14 @@ public final class LocalLogManager implements RaftClient<ApiMessageAndVersion>, @@ -305,14 +305,14 @@ public final class LocalLogManager implements RaftClient<ApiMessageAndVersion>,
* Stores a new snapshot and notifies all threads waiting for a snapshot.
*/
synchronized void addSnapshot(RawSnapshotReader newSnapshot) {
if (newSnapshot.snapshotId().offset - 1 > prevOffset) {
if (newSnapshot.snapshotId().offset() - 1 > prevOffset) {
log.error(
"Ignored attempt to add a snapshot {} that is greater than the latest offset {}",
newSnapshot,
prevOffset
);
} else {
snapshots.put(newSnapshot.snapshotId().offset - 1, newSnapshot);
snapshots.put(newSnapshot.snapshotId().offset() - 1, newSnapshot);
this.notifyAll();
}
}

30
raft/src/main/java/org/apache/kafka/raft/KafkaRaftClient.java

@ -262,7 +262,7 @@ public class KafkaRaftClient<T> implements RaftClient<T> { @@ -262,7 +262,7 @@ public class KafkaRaftClient<T> implements RaftClient<T> {
OptionalLong highWatermarkOpt
) {
highWatermarkOpt.ifPresent(highWatermark -> {
long newHighWatermark = Math.min(endOffset().offset, highWatermark);
long newHighWatermark = Math.min(endOffset().offset(), highWatermark);
if (state.updateHighWatermark(OptionalLong.of(newHighWatermark))) {
logger.debug("Follower high watermark updated to {}", newHighWatermark);
log.updateHighWatermark(new LogOffsetMetadata(newHighWatermark));
@ -884,13 +884,13 @@ public class KafkaRaftClient<T> implements RaftClient<T> { @@ -884,13 +884,13 @@ public class KafkaRaftClient<T> implements RaftClient<T> {
switch (validOffsetAndEpoch.kind()) {
case DIVERGING:
partitionData.divergingEpoch()
.setEpoch(validOffsetAndEpoch.offsetAndEpoch().epoch)
.setEndOffset(validOffsetAndEpoch.offsetAndEpoch().offset);
.setEpoch(validOffsetAndEpoch.offsetAndEpoch().epoch())
.setEndOffset(validOffsetAndEpoch.offsetAndEpoch().offset());
break;
case SNAPSHOT:
partitionData.snapshotId()
.setEpoch(validOffsetAndEpoch.offsetAndEpoch().epoch)
.setEndOffset(validOffsetAndEpoch.offsetAndEpoch().offset);
.setEpoch(validOffsetAndEpoch.offsetAndEpoch().epoch())
.setEndOffset(validOffsetAndEpoch.offsetAndEpoch().offset());
break;
default:
}
@ -1081,9 +1081,9 @@ public class KafkaRaftClient<T> implements RaftClient<T> { @@ -1081,9 +1081,9 @@ public class KafkaRaftClient<T> implements RaftClient<T> {
divergingEpoch.endOffset(), divergingEpoch.epoch());
state.highWatermark().ifPresent(highWatermark -> {
if (divergingOffsetAndEpoch.offset < highWatermark.offset) {
if (divergingOffsetAndEpoch.offset() < highWatermark.offset) {
throw new KafkaException("The leader requested truncation to offset " +
divergingOffsetAndEpoch.offset + ", which is below the current high watermark" +
divergingOffsetAndEpoch.offset() + ", which is below the current high watermark" +
" " + highWatermark);
}
});
@ -1288,8 +1288,8 @@ public class KafkaRaftClient<T> implements RaftClient<T> { @@ -1288,8 +1288,8 @@ public class KafkaRaftClient<T> implements RaftClient<T> {
responsePartitionSnapshot -> {
addQuorumLeader(responsePartitionSnapshot)
.snapshotId()
.setEndOffset(snapshotId.offset)
.setEpoch(snapshotId.epoch);
.setEndOffset(snapshotId.offset())
.setEpoch(snapshotId.epoch());
return responsePartitionSnapshot
.setSize(snapshotSize)
@ -1771,8 +1771,8 @@ public class KafkaRaftClient<T> implements RaftClient<T> { @@ -1771,8 +1771,8 @@ public class KafkaRaftClient<T> implements RaftClient<T> {
clusterId,
quorum.epoch(),
quorum.localIdOrThrow(),
endOffset.epoch,
endOffset.offset
endOffset.epoch(),
endOffset.offset()
);
}
@ -1805,8 +1805,8 @@ public class KafkaRaftClient<T> implements RaftClient<T> { @@ -1805,8 +1805,8 @@ public class KafkaRaftClient<T> implements RaftClient<T> {
private FetchSnapshotRequestData buildFetchSnapshotRequest(OffsetAndEpoch snapshotId, long snapshotSize) {
FetchSnapshotRequestData.SnapshotId requestSnapshotId = new FetchSnapshotRequestData.SnapshotId()
.setEpoch(snapshotId.epoch)
.setEndOffset(snapshotId.offset);
.setEpoch(snapshotId.epoch())
.setEndOffset(snapshotId.offset());
FetchSnapshotRequestData request = FetchSnapshotRequest.singleton(
clusterId,
@ -1852,7 +1852,7 @@ public class KafkaRaftClient<T> implements RaftClient<T> { @@ -1852,7 +1852,7 @@ public class KafkaRaftClient<T> implements RaftClient<T> {
LogAppendInfo info = appendAsLeader(batch.data);
OffsetAndEpoch offsetAndEpoch = new OffsetAndEpoch(info.lastOffset, epoch);
CompletableFuture<Long> future = appendPurgatory.await(
offsetAndEpoch.offset + 1, Integer.MAX_VALUE);
offsetAndEpoch.offset() + 1, Integer.MAX_VALUE);
future.whenComplete((commitTimeMs, exception) -> {
if (exception != null) {
@ -2494,7 +2494,7 @@ public class KafkaRaftClient<T> implements RaftClient<T> { @@ -2494,7 +2494,7 @@ public class KafkaRaftClient<T> implements RaftClient<T> {
*/
private void fireHandleSnapshot(SnapshotReader<T> reader) {
synchronized (this) {
nextOffset = reader.snapshotId().offset;
nextOffset = reader.snapshotId().offset();
lastSent = null;
}

12
raft/src/main/java/org/apache/kafka/raft/OffsetAndEpoch.java

@ -17,14 +17,22 @@ @@ -17,14 +17,22 @@
package org.apache.kafka.raft;
public class OffsetAndEpoch implements Comparable<OffsetAndEpoch> {
public final long offset;
public final int epoch;
private final long offset;
private final int epoch;
public OffsetAndEpoch(long offset, int epoch) {
this.offset = offset;
this.epoch = epoch;
}
public long offset() {
return offset;
}
public int epoch() {
return epoch;
}
@Override
public boolean equals(Object o) {
if (this == o) return true;

6
raft/src/main/java/org/apache/kafka/raft/QuorumState.java

@ -139,13 +139,13 @@ public class QuorumState { @@ -139,13 +139,13 @@ public class QuorumState {
throw new IllegalStateException("Initialized quorum state " + election
+ " with a voted candidate, which indicates this node was previously "
+ " a voter, but the local id " + localIdDescription);
} else if (election.epoch < logEndOffsetAndEpoch.epoch) {
} else if (election.epoch < logEndOffsetAndEpoch.epoch()) {
log.warn("Epoch from quorum-state file is {}, which is " +
"smaller than last written epoch {} in the log",
election.epoch, logEndOffsetAndEpoch.epoch);
election.epoch, logEndOffsetAndEpoch.epoch());
initialState = new UnattachedState(
time,
logEndOffsetAndEpoch.epoch,
logEndOffsetAndEpoch.epoch(),
voters,
Optional.empty(),
randomElectionTimeoutMs(),

16
raft/src/main/java/org/apache/kafka/raft/ReplicatedLog.java

@ -88,8 +88,8 @@ public interface ReplicatedLog extends AutoCloseable { @@ -88,8 +88,8 @@ public interface ReplicatedLog extends AutoCloseable {
Optional<OffsetAndEpoch> earliestSnapshotId = earliestSnapshotId();
if (earliestSnapshotId.isPresent() &&
((offset < startOffset()) ||
(offset == startOffset() && epoch != earliestSnapshotId.get().epoch) ||
(epoch < earliestSnapshotId.get().epoch))
(offset == startOffset() && epoch != earliestSnapshotId.get().epoch()) ||
(epoch < earliestSnapshotId.get().epoch()))
) {
/* Send a snapshot if the leader has a snapshot at the log start offset and
* 1. the fetch offset is less than the log start offset or
@ -108,7 +108,7 @@ public interface ReplicatedLog extends AutoCloseable { @@ -108,7 +108,7 @@ public interface ReplicatedLog extends AutoCloseable {
} else {
OffsetAndEpoch endOffsetAndEpoch = endOffsetForEpoch(epoch);
if (endOffsetAndEpoch.epoch != epoch || endOffsetAndEpoch.offset < offset) {
if (endOffsetAndEpoch.epoch() != epoch || endOffsetAndEpoch.offset() < offset) {
return ValidOffsetAndEpoch.diverging(endOffsetAndEpoch);
} else {
return ValidOffsetAndEpoch.valid(new OffsetAndEpoch(offset, epoch));
@ -217,15 +217,15 @@ public interface ReplicatedLog extends AutoCloseable { @@ -217,15 +217,15 @@ public interface ReplicatedLog extends AutoCloseable {
*/
default long truncateToEndOffset(OffsetAndEpoch endOffset) {
final long truncationOffset;
int leaderEpoch = endOffset.epoch;
int leaderEpoch = endOffset.epoch();
if (leaderEpoch == 0) {
truncationOffset = Math.min(endOffset.offset, endOffset().offset);
truncationOffset = Math.min(endOffset.offset(), endOffset().offset);
} else {
OffsetAndEpoch localEndOffset = endOffsetForEpoch(leaderEpoch);
if (localEndOffset.epoch == leaderEpoch) {
truncationOffset = Math.min(localEndOffset.offset, endOffset.offset);
if (localEndOffset.epoch() == leaderEpoch) {
truncationOffset = Math.min(localEndOffset.offset(), endOffset.offset());
} else {
truncationOffset = localEndOffset.offset;
truncationOffset = localEndOffset.offset();
}
}

4
raft/src/main/java/org/apache/kafka/raft/internals/KafkaRaftMetrics.java

@ -102,10 +102,10 @@ public class KafkaRaftMetrics implements AutoCloseable { @@ -102,10 +102,10 @@ public class KafkaRaftMetrics implements AutoCloseable {
metrics.addMetric(this.highWatermarkMetricName, (mConfig, currentTimeMs) -> state.highWatermark().map(hw -> hw.offset).orElse(-1L));
this.logEndOffsetMetricName = metrics.metricName("log-end-offset", metricGroupName, "The current raft log end offset.");
metrics.addMetric(this.logEndOffsetMetricName, (mConfig, currentTimeMs) -> logEndOffset.offset);
metrics.addMetric(this.logEndOffsetMetricName, (mConfig, currentTimeMs) -> logEndOffset.offset());
this.logEndEpochMetricName = metrics.metricName("log-end-epoch", metricGroupName, "The current raft log end epoch.");
metrics.addMetric(this.logEndEpochMetricName, (mConfig, currentTimeMs) -> logEndOffset.epoch);
metrics.addMetric(this.logEndEpochMetricName, (mConfig, currentTimeMs) -> logEndOffset.epoch());
this.numUnknownVoterConnectionsMetricName = metrics.metricName("number-unknown-voter-connections", metricGroupName,
"Number of unknown voters whose connection information is not cached; would never be larger than quorum-size.");

4
raft/src/main/java/org/apache/kafka/snapshot/RecordsSnapshotReader.java

@ -49,12 +49,12 @@ public final class RecordsSnapshotReader<T> implements SnapshotReader<T> { @@ -49,12 +49,12 @@ public final class RecordsSnapshotReader<T> implements SnapshotReader<T> {
@Override
public long lastContainedLogOffset() {
return snapshotId.offset - 1;
return snapshotId.offset() - 1;
}
@Override
public int lastContainedLogEpoch() {
return snapshotId.epoch;
return snapshotId.epoch();
}
@Override

8
raft/src/main/java/org/apache/kafka/snapshot/RecordsSnapshotWriter.java

@ -52,7 +52,7 @@ final public class RecordsSnapshotWriter<T> implements SnapshotWriter<T> { @@ -52,7 +52,7 @@ final public class RecordsSnapshotWriter<T> implements SnapshotWriter<T> {
this.lastContainedLogTimestamp = lastContainedLogTimestamp;
this.accumulator = new BatchAccumulator<>(
snapshot.snapshotId().epoch,
snapshot.snapshotId().epoch(),
0,
Integer.MAX_VALUE,
maxBatchSize,
@ -140,12 +140,12 @@ final public class RecordsSnapshotWriter<T> implements SnapshotWriter<T> { @@ -140,12 +140,12 @@ final public class RecordsSnapshotWriter<T> implements SnapshotWriter<T> {
@Override
public long lastContainedLogOffset() {
return snapshot.snapshotId().offset - 1;
return snapshot.snapshotId().offset() - 1;
}
@Override
public int lastContainedLogEpoch() {
return snapshot.snapshotId().epoch;
return snapshot.snapshotId().epoch();
}
@Override
@ -164,7 +164,7 @@ final public class RecordsSnapshotWriter<T> implements SnapshotWriter<T> { @@ -164,7 +164,7 @@ final public class RecordsSnapshotWriter<T> implements SnapshotWriter<T> {
throw new IllegalStateException(message);
}
accumulator.append(snapshot.snapshotId().epoch, records);
accumulator.append(snapshot.snapshotId().epoch(), records);
if (accumulator.needsDrain(time.milliseconds())) {
appendBatches(accumulator.drain());

2
raft/src/main/java/org/apache/kafka/snapshot/Snapshots.java

@ -53,7 +53,7 @@ public final class Snapshots { @@ -53,7 +53,7 @@ public final class Snapshots {
}
static String filenameFromSnapshotId(OffsetAndEpoch snapshotId) {
return String.format("%s-%s", OFFSET_FORMATTER.format(snapshotId.offset), EPOCH_FORMATTER.format(snapshotId.epoch));
return String.format("%s-%s", OFFSET_FORMATTER.format(snapshotId.offset()), EPOCH_FORMATTER.format(snapshotId.epoch()));
}
static Path moveRename(Path source, OffsetAndEpoch snapshotId) {

206
raft/src/test/java/org/apache/kafka/raft/KafkaRaftClientSnapshotTest.java

@ -60,8 +60,8 @@ final public class KafkaRaftClientSnapshotTest { @@ -60,8 +60,8 @@ final public class KafkaRaftClientSnapshotTest {
OffsetAndEpoch snapshotId = new OffsetAndEpoch(3, 1);
RaftClientTestContext context = new RaftClientTestContext.Builder(localId, voters)
.appendToLog(snapshotId.epoch, Arrays.asList("a", "b", "c"))
.appendToLog(snapshotId.epoch, Arrays.asList("d", "e", "f"))
.appendToLog(snapshotId.epoch(), Arrays.asList("a", "b", "c"))
.appendToLog(snapshotId.epoch(), Arrays.asList("d", "e", "f"))
.withEmptySnapshot(snapshotId)
.deleteBeforeSnapshot(snapshotId)
.build();
@ -92,8 +92,8 @@ final public class KafkaRaftClientSnapshotTest { @@ -92,8 +92,8 @@ final public class KafkaRaftClientSnapshotTest {
OffsetAndEpoch snapshotId = new OffsetAndEpoch(3, 1);
RaftClientTestContext context = new RaftClientTestContext.Builder(localId, voters)
.appendToLog(snapshotId.epoch, Arrays.asList("a", "b", "c"))
.appendToLog(snapshotId.epoch, Arrays.asList("d", "e", "f"))
.appendToLog(snapshotId.epoch(), Arrays.asList("a", "b", "c"))
.appendToLog(snapshotId.epoch(), Arrays.asList("d", "e", "f"))
.withEmptySnapshot(snapshotId)
.deleteBeforeSnapshot(snapshotId)
.withElectedLeader(epoch, leaderId)
@ -103,7 +103,7 @@ final public class KafkaRaftClientSnapshotTest { @@ -103,7 +103,7 @@ final public class KafkaRaftClientSnapshotTest {
long localLogEndOffset = context.log.endOffset().offset;
context.pollUntilRequest();
RaftRequest.Outbound fetchRequest = context.assertSentFetchRequest();
context.assertFetchRequestData(fetchRequest, epoch, localLogEndOffset, snapshotId.epoch);
context.assertFetchRequestData(fetchRequest, epoch, localLogEndOffset, snapshotId.epoch());
context.deliverResponse(
fetchRequest.correlationId,
fetchRequest.destinationId(),
@ -111,7 +111,7 @@ final public class KafkaRaftClientSnapshotTest { @@ -111,7 +111,7 @@ final public class KafkaRaftClientSnapshotTest {
);
context.pollUntilRequest();
context.assertSentFetchRequest(epoch, localLogEndOffset, snapshotId.epoch);
context.assertSentFetchRequest(epoch, localLogEndOffset, snapshotId.epoch());
// Check that listener was notified of the new snapshot
try (SnapshotReader<String> snapshot = context.listener.drainHandledSnapshot().get()) {
@ -129,8 +129,8 @@ final public class KafkaRaftClientSnapshotTest { @@ -129,8 +129,8 @@ final public class KafkaRaftClientSnapshotTest {
OffsetAndEpoch snapshotId = new OffsetAndEpoch(3, 1);
RaftClientTestContext context = new RaftClientTestContext.Builder(localId, voters)
.appendToLog(snapshotId.epoch, Arrays.asList("a", "b", "c"))
.appendToLog(snapshotId.epoch, Arrays.asList("d", "e", "f"))
.appendToLog(snapshotId.epoch(), Arrays.asList("a", "b", "c"))
.appendToLog(snapshotId.epoch(), Arrays.asList("d", "e", "f"))
.withEmptySnapshot(snapshotId)
.deleteBeforeSnapshot(snapshotId)
.withElectedLeader(epoch, leaderId)
@ -140,7 +140,7 @@ final public class KafkaRaftClientSnapshotTest { @@ -140,7 +140,7 @@ final public class KafkaRaftClientSnapshotTest {
long localLogEndOffset = context.log.endOffset().offset;
context.pollUntilRequest();
RaftRequest.Outbound fetchRequest = context.assertSentFetchRequest();
context.assertFetchRequestData(fetchRequest, epoch, localLogEndOffset, snapshotId.epoch);
context.assertFetchRequestData(fetchRequest, epoch, localLogEndOffset, snapshotId.epoch());
context.deliverResponse(
fetchRequest.correlationId,
fetchRequest.destinationId(),
@ -148,7 +148,7 @@ final public class KafkaRaftClientSnapshotTest { @@ -148,7 +148,7 @@ final public class KafkaRaftClientSnapshotTest {
);
context.pollUntilRequest();
context.assertSentFetchRequest(epoch, localLogEndOffset, snapshotId.epoch);
context.assertSentFetchRequest(epoch, localLogEndOffset, snapshotId.epoch());
RaftClientTestContext.MockListener secondListener = new RaftClientTestContext.MockListener(OptionalInt.of(localId));
context.client.register(secondListener);
@ -169,9 +169,9 @@ final public class KafkaRaftClientSnapshotTest { @@ -169,9 +169,9 @@ final public class KafkaRaftClientSnapshotTest {
OffsetAndEpoch snapshotId = new OffsetAndEpoch(3, 1);
RaftClientTestContext context = new RaftClientTestContext.Builder(localId, voters)
.appendToLog(snapshotId.epoch, Arrays.asList("a", "b", "c"))
.appendToLog(snapshotId.epoch, Arrays.asList("d", "e", "f"))
.appendToLog(snapshotId.epoch, Arrays.asList("g", "h", "i"))
.appendToLog(snapshotId.epoch(), Arrays.asList("a", "b", "c"))
.appendToLog(snapshotId.epoch(), Arrays.asList("d", "e", "f"))
.appendToLog(snapshotId.epoch(), Arrays.asList("g", "h", "i"))
.withEmptySnapshot(snapshotId)
.deleteBeforeSnapshot(snapshotId)
.build();
@ -197,7 +197,7 @@ final public class KafkaRaftClientSnapshotTest { @@ -197,7 +197,7 @@ final public class KafkaRaftClientSnapshotTest {
// Generate a new snapshot
OffsetAndEpoch secondSnapshotId = new OffsetAndEpoch(localLogEndOffset, epoch);
try (SnapshotWriter<String> snapshot = context.client.createSnapshot(secondSnapshotId.offset - 1, secondSnapshotId.epoch, 0).get()) {
try (SnapshotWriter<String> snapshot = context.client.createSnapshot(secondSnapshotId.offset() - 1, secondSnapshotId.epoch(), 0).get()) {
assertEquals(secondSnapshotId, snapshot.snapshotId());
snapshot.freeze();
}
@ -243,7 +243,7 @@ final public class KafkaRaftClientSnapshotTest { @@ -243,7 +243,7 @@ final public class KafkaRaftClientSnapshotTest {
context.advanceLocalLeaderHighWatermarkToLogEndOffset();
OffsetAndEpoch snapshotId = new OffsetAndEpoch(localLogEndOffset, epoch);
try (SnapshotWriter<String> snapshot = context.client.createSnapshot(snapshotId.offset - 1, snapshotId.epoch, 0).get()) {
try (SnapshotWriter<String> snapshot = context.client.createSnapshot(snapshotId.offset() - 1, snapshotId.epoch(), 0).get()) {
assertEquals(snapshotId, snapshot.snapshotId());
snapshot.freeze();
}
@ -257,8 +257,8 @@ final public class KafkaRaftClientSnapshotTest { @@ -257,8 +257,8 @@ final public class KafkaRaftClientSnapshotTest {
assertEquals(Errors.NONE, Errors.forCode(partitionResponse.errorCode()));
assertEquals(epoch, partitionResponse.currentLeader().leaderEpoch());
assertEquals(localId, partitionResponse.currentLeader().leaderId());
assertEquals(snapshotId.epoch, partitionResponse.snapshotId().epoch());
assertEquals(snapshotId.offset, partitionResponse.snapshotId().endOffset());
assertEquals(snapshotId.epoch(), partitionResponse.snapshotId().epoch());
assertEquals(snapshotId.offset(), partitionResponse.snapshotId().endOffset());
}
@Test
@ -270,20 +270,20 @@ final public class KafkaRaftClientSnapshotTest { @@ -270,20 +270,20 @@ final public class KafkaRaftClientSnapshotTest {
OffsetAndEpoch oldestSnapshotId = new OffsetAndEpoch(3, 2);
RaftClientTestContext context = new RaftClientTestContext.Builder(localId, voters)
.appendToLog(oldestSnapshotId.epoch, Arrays.asList("a", "b", "c"))
.appendToLog(oldestSnapshotId.epoch, Arrays.asList("d", "e", "f"))
.appendToLog(oldestSnapshotId.epoch(), Arrays.asList("a", "b", "c"))
.appendToLog(oldestSnapshotId.epoch(), Arrays.asList("d", "e", "f"))
.withAppendLingerMs(1)
.build();
context.becomeLeader();
int epoch = context.currentEpoch();
assertEquals(oldestSnapshotId.epoch + 1, epoch);
assertEquals(oldestSnapshotId.epoch() + 1, epoch);
// Advance the highWatermark
context.advanceLocalLeaderHighWatermarkToLogEndOffset();
// Create a snapshot at the high watermark
try (SnapshotWriter<String> snapshot = context.client.createSnapshot(oldestSnapshotId.offset - 1, oldestSnapshotId.epoch, 0).get()) {
try (SnapshotWriter<String> snapshot = context.client.createSnapshot(oldestSnapshotId.offset() - 1, oldestSnapshotId.epoch(), 0).get()) {
assertEquals(oldestSnapshotId, snapshot.snapshotId());
snapshot.freeze();
}
@ -294,7 +294,7 @@ final public class KafkaRaftClientSnapshotTest { @@ -294,7 +294,7 @@ final public class KafkaRaftClientSnapshotTest {
context.client.poll();
// It is an invalid request to send an last fetched epoch greater than the current epoch
context.deliverRequest(context.fetchRequest(epoch, otherNodeId, oldestSnapshotId.offset + 1, epoch + 1, 0));
context.deliverRequest(context.fetchRequest(epoch, otherNodeId, oldestSnapshotId.offset() + 1, epoch + 1, 0));
context.pollUntilResponse();
context.assertSentFetchPartitionResponse(Errors.INVALID_REQUEST, epoch, OptionalInt.of(localId));
}
@ -309,20 +309,20 @@ final public class KafkaRaftClientSnapshotTest { @@ -309,20 +309,20 @@ final public class KafkaRaftClientSnapshotTest {
OffsetAndEpoch oldestSnapshotId = new OffsetAndEpoch(3, 2);
RaftClientTestContext context = new RaftClientTestContext.Builder(localId, voters)
.appendToLog(oldestSnapshotId.epoch, Arrays.asList("a", "b", "c"))
.appendToLog(oldestSnapshotId.epoch + 2, Arrays.asList("d", "e", "f"))
.appendToLog(oldestSnapshotId.epoch(), Arrays.asList("a", "b", "c"))
.appendToLog(oldestSnapshotId.epoch() + 2, Arrays.asList("d", "e", "f"))
.withAppendLingerMs(1)
.build();
context.becomeLeader();
int epoch = context.currentEpoch();
assertEquals(oldestSnapshotId.epoch + 2 + 1, epoch);
assertEquals(oldestSnapshotId.epoch() + 2 + 1, epoch);
// Advance the highWatermark
context.advanceLocalLeaderHighWatermarkToLogEndOffset();
// Create a snapshot at the high watermark
try (SnapshotWriter<String> snapshot = context.client.createSnapshot(oldestSnapshotId.offset - 1, oldestSnapshotId.epoch, 0).get()) {
try (SnapshotWriter<String> snapshot = context.client.createSnapshot(oldestSnapshotId.offset() - 1, oldestSnapshotId.epoch(), 0).get()) {
assertEquals(oldestSnapshotId, snapshot.snapshotId());
snapshot.freeze();
}
@ -330,15 +330,15 @@ final public class KafkaRaftClientSnapshotTest { @@ -330,15 +330,15 @@ final public class KafkaRaftClientSnapshotTest {
// This should truncate to the old snapshot
context.deliverRequest(
context.fetchRequest(epoch, otherNodeId, oldestSnapshotId.offset + 1, oldestSnapshotId.epoch + 1, 0)
context.fetchRequest(epoch, otherNodeId, oldestSnapshotId.offset() + 1, oldestSnapshotId.epoch() + 1, 0)
);
context.pollUntilResponse();
FetchResponseData.PartitionData partitionResponse = context.assertSentFetchPartitionResponse();
assertEquals(Errors.NONE, Errors.forCode(partitionResponse.errorCode()));
assertEquals(epoch, partitionResponse.currentLeader().leaderEpoch());
assertEquals(localId, partitionResponse.currentLeader().leaderId());
assertEquals(oldestSnapshotId.epoch, partitionResponse.divergingEpoch().epoch());
assertEquals(oldestSnapshotId.offset, partitionResponse.divergingEpoch().endOffset());
assertEquals(oldestSnapshotId.epoch(), partitionResponse.divergingEpoch().epoch());
assertEquals(oldestSnapshotId.offset(), partitionResponse.divergingEpoch().endOffset());
}
@Test
@ -351,21 +351,21 @@ final public class KafkaRaftClientSnapshotTest { @@ -351,21 +351,21 @@ final public class KafkaRaftClientSnapshotTest {
OffsetAndEpoch oldestSnapshotId = new OffsetAndEpoch(3, 2);
RaftClientTestContext context = new RaftClientTestContext.Builder(localId, voters)
.appendToLog(oldestSnapshotId.epoch, Arrays.asList("a", "b", "c"))
.appendToLog(oldestSnapshotId.epoch, Arrays.asList("d", "e", "f"))
.appendToLog(oldestSnapshotId.epoch + 2, Arrays.asList("g", "h", "i"))
.appendToLog(oldestSnapshotId.epoch(), Arrays.asList("a", "b", "c"))
.appendToLog(oldestSnapshotId.epoch(), Arrays.asList("d", "e", "f"))
.appendToLog(oldestSnapshotId.epoch() + 2, Arrays.asList("g", "h", "i"))
.withAppendLingerMs(1)
.build();
context.becomeLeader();
int epoch = context.currentEpoch();
assertEquals(oldestSnapshotId.epoch + 2 + 1, epoch);
assertEquals(oldestSnapshotId.epoch() + 2 + 1, epoch);
// Advance the highWatermark
context.advanceLocalLeaderHighWatermarkToLogEndOffset();
// Create a snapshot at the high watermark
try (SnapshotWriter<String> snapshot = context.client.createSnapshot(oldestSnapshotId.offset - 1, oldestSnapshotId.epoch, 0).get()) {
try (SnapshotWriter<String> snapshot = context.client.createSnapshot(oldestSnapshotId.offset() - 1, oldestSnapshotId.epoch(), 0).get()) {
assertEquals(oldestSnapshotId, snapshot.snapshotId());
snapshot.freeze();
}
@ -373,7 +373,7 @@ final public class KafkaRaftClientSnapshotTest { @@ -373,7 +373,7 @@ final public class KafkaRaftClientSnapshotTest {
// Send fetch request at log start offset with valid last fetched epoch
context.deliverRequest(
context.fetchRequest(epoch, otherNodeId, oldestSnapshotId.offset, oldestSnapshotId.epoch, 0)
context.fetchRequest(epoch, otherNodeId, oldestSnapshotId.offset(), oldestSnapshotId.epoch(), 0)
);
context.pollUntilResponse();
context.assertSentFetchPartitionResponse(Errors.NONE, epoch, OptionalInt.of(localId));
@ -389,21 +389,21 @@ final public class KafkaRaftClientSnapshotTest { @@ -389,21 +389,21 @@ final public class KafkaRaftClientSnapshotTest {
OffsetAndEpoch oldestSnapshotId = new OffsetAndEpoch(3, 2);
RaftClientTestContext context = new RaftClientTestContext.Builder(localId, voters)
.appendToLog(oldestSnapshotId.epoch, Arrays.asList("a", "b", "c"))
.appendToLog(oldestSnapshotId.epoch, Arrays.asList("d", "e", "f"))
.appendToLog(oldestSnapshotId.epoch + 2, Arrays.asList("g", "h", "i"))
.appendToLog(oldestSnapshotId.epoch(), Arrays.asList("a", "b", "c"))
.appendToLog(oldestSnapshotId.epoch(), Arrays.asList("d", "e", "f"))
.appendToLog(oldestSnapshotId.epoch() + 2, Arrays.asList("g", "h", "i"))
.withAppendLingerMs(1)
.build();
context.becomeLeader();
int epoch = context.currentEpoch();
assertEquals(oldestSnapshotId.epoch + 2 + 1, epoch);
assertEquals(oldestSnapshotId.epoch() + 2 + 1, epoch);
// Advance the highWatermark
context.advanceLocalLeaderHighWatermarkToLogEndOffset();
// Create a snapshot at the high watermark
try (SnapshotWriter<String> snapshot = context.client.createSnapshot(oldestSnapshotId.offset - 1, oldestSnapshotId.epoch, 0).get()) {
try (SnapshotWriter<String> snapshot = context.client.createSnapshot(oldestSnapshotId.offset() - 1, oldestSnapshotId.epoch(), 0).get()) {
assertEquals(oldestSnapshotId, snapshot.snapshotId());
snapshot.freeze();
}
@ -412,15 +412,15 @@ final public class KafkaRaftClientSnapshotTest { @@ -412,15 +412,15 @@ final public class KafkaRaftClientSnapshotTest {
// Send fetch with log start offset and invalid last fetched epoch
context.deliverRequest(
context.fetchRequest(epoch, otherNodeId, oldestSnapshotId.offset, oldestSnapshotId.epoch + 1, 0)
context.fetchRequest(epoch, otherNodeId, oldestSnapshotId.offset(), oldestSnapshotId.epoch() + 1, 0)
);
context.pollUntilResponse();
FetchResponseData.PartitionData partitionResponse = context.assertSentFetchPartitionResponse();
assertEquals(Errors.NONE, Errors.forCode(partitionResponse.errorCode()));
assertEquals(epoch, partitionResponse.currentLeader().leaderEpoch());
assertEquals(localId, partitionResponse.currentLeader().leaderId());
assertEquals(oldestSnapshotId.epoch, partitionResponse.snapshotId().epoch());
assertEquals(oldestSnapshotId.offset, partitionResponse.snapshotId().endOffset());
assertEquals(oldestSnapshotId.epoch(), partitionResponse.snapshotId().epoch());
assertEquals(oldestSnapshotId.offset(), partitionResponse.snapshotId().endOffset());
}
@Test
@ -433,21 +433,21 @@ final public class KafkaRaftClientSnapshotTest { @@ -433,21 +433,21 @@ final public class KafkaRaftClientSnapshotTest {
OffsetAndEpoch oldestSnapshotId = new OffsetAndEpoch(3, 2);
RaftClientTestContext context = new RaftClientTestContext.Builder(localId, voters)
.appendToLog(oldestSnapshotId.epoch, Arrays.asList("a", "b", "c"))
.appendToLog(oldestSnapshotId.epoch, Arrays.asList("d", "e", "f"))
.appendToLog(oldestSnapshotId.epoch + 2, Arrays.asList("g", "h", "i"))
.appendToLog(oldestSnapshotId.epoch(), Arrays.asList("a", "b", "c"))
.appendToLog(oldestSnapshotId.epoch(), Arrays.asList("d", "e", "f"))
.appendToLog(oldestSnapshotId.epoch() + 2, Arrays.asList("g", "h", "i"))
.withAppendLingerMs(1)
.build();
context.becomeLeader();
int epoch = context.currentEpoch();
assertEquals(oldestSnapshotId.epoch + 2 + 1, epoch);
assertEquals(oldestSnapshotId.epoch() + 2 + 1, epoch);
// Advance the highWatermark
context.advanceLocalLeaderHighWatermarkToLogEndOffset();
// Create a snapshot at the high watermark
try (SnapshotWriter<String> snapshot = context.client.createSnapshot(oldestSnapshotId.offset - 1, oldestSnapshotId.epoch, 0).get()) {
try (SnapshotWriter<String> snapshot = context.client.createSnapshot(oldestSnapshotId.offset() - 1, oldestSnapshotId.epoch(), 0).get()) {
assertEquals(oldestSnapshotId, snapshot.snapshotId());
snapshot.freeze();
}
@ -459,7 +459,7 @@ final public class KafkaRaftClientSnapshotTest { @@ -459,7 +459,7 @@ final public class KafkaRaftClientSnapshotTest {
epoch,
otherNodeId,
context.log.endOffset().offset,
oldestSnapshotId.epoch - 1,
oldestSnapshotId.epoch() - 1,
0
)
);
@ -468,8 +468,8 @@ final public class KafkaRaftClientSnapshotTest { @@ -468,8 +468,8 @@ final public class KafkaRaftClientSnapshotTest {
assertEquals(Errors.NONE, Errors.forCode(partitionResponse.errorCode()));
assertEquals(epoch, partitionResponse.currentLeader().leaderEpoch());
assertEquals(localId, partitionResponse.currentLeader().leaderId());
assertEquals(oldestSnapshotId.epoch, partitionResponse.snapshotId().epoch());
assertEquals(oldestSnapshotId.offset, partitionResponse.snapshotId().endOffset());
assertEquals(oldestSnapshotId.epoch(), partitionResponse.snapshotId().epoch());
assertEquals(oldestSnapshotId.offset(), partitionResponse.snapshotId().endOffset());
}
@Test
@ -529,7 +529,7 @@ final public class KafkaRaftClientSnapshotTest { @@ -529,7 +529,7 @@ final public class KafkaRaftClientSnapshotTest {
List<String> records = Arrays.asList("foo", "bar");
RaftClientTestContext context = new RaftClientTestContext.Builder(localId, voters)
.appendToLog(snapshotId.epoch, Arrays.asList("a"))
.appendToLog(snapshotId.epoch(), Arrays.asList("a"))
.build();
context.becomeLeader();
@ -537,7 +537,7 @@ final public class KafkaRaftClientSnapshotTest { @@ -537,7 +537,7 @@ final public class KafkaRaftClientSnapshotTest {
context.advanceLocalLeaderHighWatermarkToLogEndOffset();
try (SnapshotWriter<String> snapshot = context.client.createSnapshot(snapshotId.offset - 1, snapshotId.epoch, 0).get()) {
try (SnapshotWriter<String> snapshot = context.client.createSnapshot(snapshotId.offset() - 1, snapshotId.epoch(), 0).get()) {
assertEquals(snapshotId, snapshot.snapshotId());
snapshot.append(records);
snapshot.freeze();
@ -578,7 +578,7 @@ final public class KafkaRaftClientSnapshotTest { @@ -578,7 +578,7 @@ final public class KafkaRaftClientSnapshotTest {
List<String> records = Arrays.asList("foo", "bar");
RaftClientTestContext context = new RaftClientTestContext.Builder(localId, voters)
.appendToLog(snapshotId.epoch, records)
.appendToLog(snapshotId.epoch(), records)
.build();
context.becomeLeader();
@ -586,7 +586,7 @@ final public class KafkaRaftClientSnapshotTest { @@ -586,7 +586,7 @@ final public class KafkaRaftClientSnapshotTest {
context.advanceLocalLeaderHighWatermarkToLogEndOffset();
try (SnapshotWriter<String> snapshot = context.client.createSnapshot(snapshotId.offset - 1, snapshotId.epoch, 0).get()) {
try (SnapshotWriter<String> snapshot = context.client.createSnapshot(snapshotId.offset() - 1, snapshotId.epoch(), 0).get()) {
assertEquals(snapshotId, snapshot.snapshotId());
snapshot.append(records);
snapshot.freeze();
@ -687,7 +687,7 @@ final public class KafkaRaftClientSnapshotTest { @@ -687,7 +687,7 @@ final public class KafkaRaftClientSnapshotTest {
List<String> records = Arrays.asList("foo", "bar");
RaftClientTestContext context = new RaftClientTestContext.Builder(localId, voters)
.appendToLog(snapshotId.epoch, Arrays.asList("a"))
.appendToLog(snapshotId.epoch(), Arrays.asList("a"))
.build();
context.becomeLeader();
@ -695,7 +695,7 @@ final public class KafkaRaftClientSnapshotTest { @@ -695,7 +695,7 @@ final public class KafkaRaftClientSnapshotTest {
context.advanceLocalLeaderHighWatermarkToLogEndOffset();
try (SnapshotWriter<String> snapshot = context.client.createSnapshot(snapshotId.offset - 1, snapshotId.epoch, 0).get()) {
try (SnapshotWriter<String> snapshot = context.client.createSnapshot(snapshotId.offset() - 1, snapshotId.epoch(), 0).get()) {
assertEquals(snapshotId, snapshot.snapshotId());
snapshot.append(records);
snapshot.freeze();
@ -882,8 +882,8 @@ final public class KafkaRaftClientSnapshotTest { @@ -882,8 +882,8 @@ final public class KafkaRaftClientSnapshotTest {
localId,
Integer.MAX_VALUE
).get();
assertEquals(snapshotId.offset, request.snapshotId().endOffset());
assertEquals(snapshotId.epoch, request.snapshotId().epoch());
assertEquals(snapshotId.offset(), request.snapshotId().endOffset());
assertEquals(snapshotId.epoch(), request.snapshotId().epoch());
assertEquals(0, request.position());
List<String> records = Arrays.asList("foo", "bar");
@ -909,7 +909,7 @@ final public class KafkaRaftClientSnapshotTest { @@ -909,7 +909,7 @@ final public class KafkaRaftClientSnapshotTest {
context.pollUntilRequest();
fetchRequest = context.assertSentFetchRequest();
context.assertFetchRequestData(fetchRequest, epoch, snapshotId.offset, snapshotId.epoch);
context.assertFetchRequestData(fetchRequest, epoch, snapshotId.offset(), snapshotId.epoch());
// Check that the snapshot was written to the log
RawSnapshotReader snapshot = context.log.readSnapshot(snapshotId).get();
@ -953,8 +953,8 @@ final public class KafkaRaftClientSnapshotTest { @@ -953,8 +953,8 @@ final public class KafkaRaftClientSnapshotTest {
localId,
Integer.MAX_VALUE
).get();
assertEquals(snapshotId.offset, request.snapshotId().endOffset());
assertEquals(snapshotId.epoch, request.snapshotId().epoch());
assertEquals(snapshotId.offset(), request.snapshotId().endOffset());
assertEquals(snapshotId.epoch(), request.snapshotId().epoch());
assertEquals(0, request.position());
List<String> records = Arrays.asList("foo", "bar");
@ -989,8 +989,8 @@ final public class KafkaRaftClientSnapshotTest { @@ -989,8 +989,8 @@ final public class KafkaRaftClientSnapshotTest {
localId,
Integer.MAX_VALUE
).get();
assertEquals(snapshotId.offset, request.snapshotId().endOffset());
assertEquals(snapshotId.epoch, request.snapshotId().epoch());
assertEquals(snapshotId.offset(), request.snapshotId().endOffset());
assertEquals(snapshotId.epoch(), request.snapshotId().epoch());
assertEquals(sendingBuffer.limit(), request.position());
sendingBuffer = memorySnapshot.buffer().slice();
@ -1012,7 +1012,7 @@ final public class KafkaRaftClientSnapshotTest { @@ -1012,7 +1012,7 @@ final public class KafkaRaftClientSnapshotTest {
context.pollUntilRequest();
fetchRequest = context.assertSentFetchRequest();
context.assertFetchRequestData(fetchRequest, epoch, snapshotId.offset, snapshotId.epoch);
context.assertFetchRequestData(fetchRequest, epoch, snapshotId.offset(), snapshotId.epoch());
// Check that the snapshot was written to the log
RawSnapshotReader snapshot = context.log.readSnapshot(snapshotId).get();
@ -1056,8 +1056,8 @@ final public class KafkaRaftClientSnapshotTest { @@ -1056,8 +1056,8 @@ final public class KafkaRaftClientSnapshotTest {
localId,
Integer.MAX_VALUE
).get();
assertEquals(snapshotId.offset, request.snapshotId().endOffset());
assertEquals(snapshotId.epoch, request.snapshotId().epoch());
assertEquals(snapshotId.offset(), request.snapshotId().endOffset());
assertEquals(snapshotId.epoch(), request.snapshotId().epoch());
assertEquals(0, request.position());
// Reply with a snapshot not found error
@ -1114,8 +1114,8 @@ final public class KafkaRaftClientSnapshotTest { @@ -1114,8 +1114,8 @@ final public class KafkaRaftClientSnapshotTest {
localId,
Integer.MAX_VALUE
).get();
assertEquals(snapshotId.offset, request.snapshotId().endOffset());
assertEquals(snapshotId.epoch, request.snapshotId().epoch());
assertEquals(snapshotId.offset(), request.snapshotId().endOffset());
assertEquals(snapshotId.epoch(), request.snapshotId().epoch());
assertEquals(0, request.position());
// Reply with new leader response
@ -1171,8 +1171,8 @@ final public class KafkaRaftClientSnapshotTest { @@ -1171,8 +1171,8 @@ final public class KafkaRaftClientSnapshotTest {
localId,
Integer.MAX_VALUE
).get();
assertEquals(snapshotId.offset, request.snapshotId().endOffset());
assertEquals(snapshotId.epoch, request.snapshotId().epoch());
assertEquals(snapshotId.offset(), request.snapshotId().endOffset());
assertEquals(snapshotId.epoch(), request.snapshotId().epoch());
assertEquals(0, request.position());
// Reply with new leader epoch
@ -1228,8 +1228,8 @@ final public class KafkaRaftClientSnapshotTest { @@ -1228,8 +1228,8 @@ final public class KafkaRaftClientSnapshotTest {
localId,
Integer.MAX_VALUE
).get();
assertEquals(snapshotId.offset, request.snapshotId().endOffset());
assertEquals(snapshotId.epoch, request.snapshotId().epoch());
assertEquals(snapshotId.offset(), request.snapshotId().endOffset());
assertEquals(snapshotId.epoch(), request.snapshotId().epoch());
assertEquals(0, request.position());
// Reply with unknown leader epoch
@ -1260,8 +1260,8 @@ final public class KafkaRaftClientSnapshotTest { @@ -1260,8 +1260,8 @@ final public class KafkaRaftClientSnapshotTest {
localId,
Integer.MAX_VALUE
).get();
assertEquals(snapshotId.offset, request.snapshotId().endOffset());
assertEquals(snapshotId.epoch, request.snapshotId().epoch());
assertEquals(snapshotId.offset(), request.snapshotId().endOffset());
assertEquals(snapshotId.epoch(), request.snapshotId().epoch());
assertEquals(0, request.position());
}
@ -1295,8 +1295,8 @@ final public class KafkaRaftClientSnapshotTest { @@ -1295,8 +1295,8 @@ final public class KafkaRaftClientSnapshotTest {
localId,
Integer.MAX_VALUE
).get();
assertEquals(snapshotId.offset, request.snapshotId().endOffset());
assertEquals(snapshotId.epoch, request.snapshotId().epoch());
assertEquals(snapshotId.offset(), request.snapshotId().endOffset());
assertEquals(snapshotId.epoch(), request.snapshotId().epoch());
assertEquals(0, request.position());
// Reply with an invalid snapshot id endOffset
@ -1314,7 +1314,7 @@ final public class KafkaRaftClientSnapshotTest { @@ -1314,7 +1314,7 @@ final public class KafkaRaftClientSnapshotTest {
responsePartitionSnapshot
.snapshotId()
.setEndOffset(-1)
.setEpoch(snapshotId.epoch);
.setEpoch(snapshotId.epoch());
return responsePartitionSnapshot;
}
@ -1342,8 +1342,8 @@ final public class KafkaRaftClientSnapshotTest { @@ -1342,8 +1342,8 @@ final public class KafkaRaftClientSnapshotTest {
localId,
Integer.MAX_VALUE
).get();
assertEquals(snapshotId.offset, request.snapshotId().endOffset());
assertEquals(snapshotId.epoch, request.snapshotId().epoch());
assertEquals(snapshotId.offset(), request.snapshotId().endOffset());
assertEquals(snapshotId.epoch(), request.snapshotId().epoch());
assertEquals(0, request.position());
// Reply with an invalid snapshot id epoch
@ -1360,7 +1360,7 @@ final public class KafkaRaftClientSnapshotTest { @@ -1360,7 +1360,7 @@ final public class KafkaRaftClientSnapshotTest {
responsePartitionSnapshot
.snapshotId()
.setEndOffset(snapshotId.offset)
.setEndOffset(snapshotId.offset())
.setEpoch(-1);
return responsePartitionSnapshot;
@ -1406,8 +1406,8 @@ final public class KafkaRaftClientSnapshotTest { @@ -1406,8 +1406,8 @@ final public class KafkaRaftClientSnapshotTest {
localId,
Integer.MAX_VALUE
).get();
assertEquals(snapshotId.offset, request.snapshotId().endOffset());
assertEquals(snapshotId.epoch, request.snapshotId().epoch());
assertEquals(snapshotId.offset(), request.snapshotId().endOffset());
assertEquals(snapshotId.epoch(), request.snapshotId().epoch());
assertEquals(0, request.position());
// Sleeping for fetch timeout should transition to candidate
@ -1432,8 +1432,8 @@ final public class KafkaRaftClientSnapshotTest { @@ -1432,8 +1432,8 @@ final public class KafkaRaftClientSnapshotTest {
responsePartitionSnapshot
.snapshotId()
.setEndOffset(snapshotId.offset)
.setEpoch(snapshotId.epoch);
.setEndOffset(snapshotId.offset())
.setEpoch(snapshotId.epoch());
return responsePartitionSnapshot;
}
@ -1532,7 +1532,7 @@ final public class KafkaRaftClientSnapshotTest { @@ -1532,7 +1532,7 @@ final public class KafkaRaftClientSnapshotTest {
// When leader creating snapshot:
// 1.1 high watermark cannot be empty
assertEquals(OptionalLong.empty(), context.client.highWatermark());
assertThrows(IllegalArgumentException.class, () -> context.client.createSnapshot(invalidSnapshotId1.offset, invalidSnapshotId1.epoch, 0));
assertThrows(IllegalArgumentException.class, () -> context.client.createSnapshot(invalidSnapshotId1.offset(), invalidSnapshotId1.epoch(), 0));
// 1.2 high watermark must larger than or equal to the snapshotId's endOffset
context.advanceLocalLeaderHighWatermarkToLogEndOffset();
@ -1544,17 +1544,17 @@ final public class KafkaRaftClientSnapshotTest { @@ -1544,17 +1544,17 @@ final public class KafkaRaftClientSnapshotTest {
assertEquals(context.log.endOffset().offset, context.client.highWatermark().getAsLong() + newRecords.size());
OffsetAndEpoch invalidSnapshotId2 = new OffsetAndEpoch(context.client.highWatermark().getAsLong() + 1, currentEpoch);
assertThrows(IllegalArgumentException.class, () -> context.client.createSnapshot(invalidSnapshotId2.offset, invalidSnapshotId2.epoch, 0));
assertThrows(IllegalArgumentException.class, () -> context.client.createSnapshot(invalidSnapshotId2.offset(), invalidSnapshotId2.epoch(), 0));
// 2 the quorum epoch must larger than or equal to the snapshotId's epoch
OffsetAndEpoch invalidSnapshotId3 = new OffsetAndEpoch(context.client.highWatermark().getAsLong() - 2, currentEpoch + 1);
assertThrows(IllegalArgumentException.class, () -> context.client.createSnapshot(invalidSnapshotId3.offset, invalidSnapshotId3.epoch, 0));
assertThrows(IllegalArgumentException.class, () -> context.client.createSnapshot(invalidSnapshotId3.offset(), invalidSnapshotId3.epoch(), 0));
// 3 the snapshotId should be validated against endOffsetForEpoch
OffsetAndEpoch endOffsetForEpoch = context.log.endOffsetForEpoch(epoch);
assertEquals(epoch, endOffsetForEpoch.epoch);
OffsetAndEpoch invalidSnapshotId4 = new OffsetAndEpoch(endOffsetForEpoch.offset + 1, epoch);
assertThrows(IllegalArgumentException.class, () -> context.client.createSnapshot(invalidSnapshotId4.offset, invalidSnapshotId4.epoch, 0));
assertEquals(epoch, endOffsetForEpoch.epoch());
OffsetAndEpoch invalidSnapshotId4 = new OffsetAndEpoch(endOffsetForEpoch.offset() + 1, epoch);
assertThrows(IllegalArgumentException.class, () -> context.client.createSnapshot(invalidSnapshotId4.offset(), invalidSnapshotId4.epoch(), 0));
}
@Test
@ -1574,7 +1574,7 @@ final public class KafkaRaftClientSnapshotTest { @@ -1574,7 +1574,7 @@ final public class KafkaRaftClientSnapshotTest {
// 1) The high watermark cannot be empty
assertEquals(OptionalLong.empty(), context.client.highWatermark());
OffsetAndEpoch invalidSnapshotId1 = new OffsetAndEpoch(0, 0);
assertThrows(IllegalArgumentException.class, () -> context.client.createSnapshot(invalidSnapshotId1.offset, invalidSnapshotId1.epoch, 0));
assertThrows(IllegalArgumentException.class, () -> context.client.createSnapshot(invalidSnapshotId1.offset(), invalidSnapshotId1.epoch(), 0));
// Poll for our first fetch request
context.pollUntilRequest();
@ -1592,11 +1592,11 @@ final public class KafkaRaftClientSnapshotTest { @@ -1592,11 +1592,11 @@ final public class KafkaRaftClientSnapshotTest {
// 2) The high watermark must be larger than or equal to the snapshotId's endOffset
int currentEpoch = context.currentEpoch();
OffsetAndEpoch invalidSnapshotId2 = new OffsetAndEpoch(context.client.highWatermark().getAsLong() + 1, currentEpoch);
assertThrows(IllegalArgumentException.class, () -> context.client.createSnapshot(invalidSnapshotId2.offset, invalidSnapshotId2.epoch, 0));
assertThrows(IllegalArgumentException.class, () -> context.client.createSnapshot(invalidSnapshotId2.offset(), invalidSnapshotId2.epoch(), 0));
// 3) The quorum epoch must be larger than or equal to the snapshotId's epoch
OffsetAndEpoch invalidSnapshotId3 = new OffsetAndEpoch(context.client.highWatermark().getAsLong(), currentEpoch + 1);
assertThrows(IllegalArgumentException.class, () -> context.client.createSnapshot(invalidSnapshotId3.offset, invalidSnapshotId3.epoch, 0));
assertThrows(IllegalArgumentException.class, () -> context.client.createSnapshot(invalidSnapshotId3.offset(), invalidSnapshotId3.epoch(), 0));
// The high watermark advances to be larger than log.endOffsetForEpoch(3), to test the case 3
context.pollUntilRequest();
@ -1613,9 +1613,9 @@ final public class KafkaRaftClientSnapshotTest { @@ -1613,9 +1613,9 @@ final public class KafkaRaftClientSnapshotTest {
// 4) The snapshotId should be validated against endOffsetForEpoch
OffsetAndEpoch endOffsetForEpoch = context.log.endOffsetForEpoch(3);
assertEquals(3, endOffsetForEpoch.epoch);
OffsetAndEpoch invalidSnapshotId4 = new OffsetAndEpoch(endOffsetForEpoch.offset + 1, epoch);
assertThrows(IllegalArgumentException.class, () -> context.client.createSnapshot(invalidSnapshotId4.offset, invalidSnapshotId4.epoch, 0));
assertEquals(3, endOffsetForEpoch.epoch());
OffsetAndEpoch invalidSnapshotId4 = new OffsetAndEpoch(endOffsetForEpoch.offset() + 1, epoch);
assertThrows(IllegalArgumentException.class, () -> context.client.createSnapshot(invalidSnapshotId4.offset(), invalidSnapshotId4.epoch(), 0));
}
private static FetchSnapshotRequestData fetchSnapshotRequest(
@ -1637,8 +1637,8 @@ final public class KafkaRaftClientSnapshotTest { @@ -1637,8 +1637,8 @@ final public class KafkaRaftClientSnapshotTest {
long position
) {
FetchSnapshotRequestData.SnapshotId snapshotId = new FetchSnapshotRequestData.SnapshotId()
.setEndOffset(offsetAndEpoch.offset)
.setEpoch(offsetAndEpoch.epoch);
.setEndOffset(offsetAndEpoch.offset())
.setEpoch(offsetAndEpoch.epoch());
FetchSnapshotRequestData request = FetchSnapshotRequest.singleton(
clusterId,
@ -1671,8 +1671,8 @@ final public class KafkaRaftClientSnapshotTest { @@ -1671,8 +1671,8 @@ final public class KafkaRaftClientSnapshotTest {
.setLeaderId(leaderId);
partitionSnapshot.snapshotId()
.setEndOffset(snapshotId.offset)
.setEpoch(snapshotId.epoch);
.setEndOffset(snapshotId.offset())
.setEpoch(snapshotId.epoch());
return partitionSnapshot
.setSize(size)
@ -1698,8 +1698,8 @@ final public class KafkaRaftClientSnapshotTest { @@ -1698,8 +1698,8 @@ final public class KafkaRaftClientSnapshotTest {
.setLeaderId(leaderId);
partitionData.snapshotId()
.setEpoch(snapshotId.epoch)
.setEndOffset(snapshotId.offset);
.setEpoch(snapshotId.epoch())
.setEndOffset(snapshotId.offset());
});
}

32
raft/src/test/java/org/apache/kafka/raft/MockLog.java

@ -92,14 +92,14 @@ public class MockLog implements ReplicatedLog { @@ -92,14 +92,14 @@ public class MockLog implements ReplicatedLog {
public boolean truncateToLatestSnapshot() {
AtomicBoolean truncated = new AtomicBoolean(false);
latestSnapshotId().ifPresent(snapshotId -> {
if (snapshotId.epoch > logLastFetchedEpoch().orElse(0) ||
(snapshotId.epoch == logLastFetchedEpoch().orElse(0) &&
snapshotId.offset > endOffset().offset)) {
if (snapshotId.epoch() > logLastFetchedEpoch().orElse(0) ||
(snapshotId.epoch() == logLastFetchedEpoch().orElse(0) &&
snapshotId.offset() > endOffset().offset)) {
batches.clear();
epochStartOffsets.clear();
snapshots.headMap(snapshotId, false).clear();
updateHighWatermark(new LogOffsetMetadata(snapshotId.offset));
updateHighWatermark(new LogOffsetMetadata(snapshotId.offset()));
flush(false);
truncated.set(true);
@ -188,7 +188,7 @@ public class MockLog implements ReplicatedLog { @@ -188,7 +188,7 @@ public class MockLog implements ReplicatedLog {
@Override
public int lastFetchedEpoch() {
return logLastFetchedEpoch().orElseGet(() -> latestSnapshotId().map(id -> id.epoch).orElse(0));
return logLastFetchedEpoch().orElseGet(() -> latestSnapshotId().map(OffsetAndEpoch::epoch).orElse(0));
}
@Override
@ -201,7 +201,7 @@ public class MockLog implements ReplicatedLog { @@ -201,7 +201,7 @@ public class MockLog implements ReplicatedLog {
}
private OffsetAndEpoch lastOffsetAndEpochFiltered(Predicate<EpochStartOffset> predicate) {
int epochLowerBound = earliestSnapshotId().map(id -> id.epoch).orElse(0);
int epochLowerBound = earliestSnapshotId().map(OffsetAndEpoch::epoch).orElse(0);
for (EpochStartOffset epochStartOffset : epochStartOffsets) {
if (!predicate.test(epochStartOffset)) {
return new OffsetAndEpoch(epochStartOffset.startOffset, epochLowerBound);
@ -230,7 +230,7 @@ public class MockLog implements ReplicatedLog { @@ -230,7 +230,7 @@ public class MockLog implements ReplicatedLog {
.map(entry -> entry.offset + 1)
.orElse(
latestSnapshotId()
.map(id -> id.offset)
.map(OffsetAndEpoch::offset)
.orElse(0L)
);
return new LogOffsetMetadata(nextOffset, Optional.of(new MockOffsetMetadata(nextId)));
@ -242,7 +242,7 @@ public class MockLog implements ReplicatedLog { @@ -242,7 +242,7 @@ public class MockLog implements ReplicatedLog {
.map(entry -> entry.offset)
.orElse(
earliestSnapshotId()
.map(id -> id.offset)
.map(OffsetAndEpoch::offset)
.orElse(0L)
);
}
@ -436,7 +436,7 @@ public class MockLog implements ReplicatedLog { @@ -436,7 +436,7 @@ public class MockLog implements ReplicatedLog {
@Override
public Optional<RawSnapshotWriter> createNewSnapshot(OffsetAndEpoch snapshotId) {
if (snapshotId.offset < startOffset()) {
if (snapshotId.offset() < startOffset()) {
logger.info(
"Cannot create a snapshot with an id ({}) less than the log start offset ({})",
snapshotId,
@ -447,7 +447,7 @@ public class MockLog implements ReplicatedLog { @@ -447,7 +447,7 @@ public class MockLog implements ReplicatedLog {
}
long highWatermarkOffset = highWatermark().offset;
if (snapshotId.offset > highWatermarkOffset) {
if (snapshotId.offset() > highWatermarkOffset) {
throw new IllegalArgumentException(
String.format(
"Cannot create a snapshot with an id (%s) greater than the high-watermark (%s)",
@ -457,7 +457,7 @@ public class MockLog implements ReplicatedLog { @@ -457,7 +457,7 @@ public class MockLog implements ReplicatedLog {
);
}
ValidOffsetAndEpoch validOffsetAndEpoch = validateOffsetAndEpoch(snapshotId.offset, snapshotId.epoch);
ValidOffsetAndEpoch validOffsetAndEpoch = validateOffsetAndEpoch(snapshotId.offset(), snapshotId.epoch());
if (validOffsetAndEpoch.kind() != ValidOffsetAndEpoch.Kind.VALID) {
throw new IllegalArgumentException(
String.format(
@ -511,7 +511,7 @@ public class MockLog implements ReplicatedLog { @@ -511,7 +511,7 @@ public class MockLog implements ReplicatedLog {
@Override
public boolean deleteBeforeSnapshot(OffsetAndEpoch snapshotId) {
if (startOffset() > snapshotId.offset) {
if (startOffset() > snapshotId.offset()) {
throw new OffsetOutOfRangeException(
String.format(
"New log start (%s) is less than the curent log start offset (%s)",
@ -520,7 +520,7 @@ public class MockLog implements ReplicatedLog { @@ -520,7 +520,7 @@ public class MockLog implements ReplicatedLog {
)
);
}
if (highWatermark.offset < snapshotId.offset) {
if (highWatermark.offset < snapshotId.offset()) {
throw new OffsetOutOfRangeException(
String.format(
"New log start (%s) is greater than the high watermark (%s)",
@ -534,11 +534,11 @@ public class MockLog implements ReplicatedLog { @@ -534,11 +534,11 @@ public class MockLog implements ReplicatedLog {
if (snapshots.containsKey(snapshotId)) {
snapshots.headMap(snapshotId, false).clear();
batches.removeIf(entry -> entry.lastOffset() < snapshotId.offset);
batches.removeIf(entry -> entry.lastOffset() < snapshotId.offset());
AtomicReference<Optional<EpochStartOffset>> last = new AtomicReference<>(Optional.empty());
epochStartOffsets.removeIf(epochStartOffset -> {
if (epochStartOffset.startOffset <= snapshotId.offset) {
if (epochStartOffset.startOffset <= snapshotId.offset()) {
last.set(Optional.of(epochStartOffset));
return true;
}
@ -549,7 +549,7 @@ public class MockLog implements ReplicatedLog { @@ -549,7 +549,7 @@ public class MockLog implements ReplicatedLog {
last.get().ifPresent(epochStartOffset -> {
epochStartOffsets.add(
0,
new EpochStartOffset(epochStartOffset.epoch, snapshotId.offset)
new EpochStartOffset(epochStartOffset.epoch, snapshotId.offset())
);
});

30
raft/src/test/java/org/apache/kafka/raft/MockLogTest.java

@ -509,7 +509,7 @@ public class MockLogTest { @@ -509,7 +509,7 @@ public class MockLogTest {
}
assertTrue(log.deleteBeforeSnapshot(snapshotId));
assertEquals(snapshotId.offset, log.startOffset());
assertEquals(snapshotId.offset(), log.startOffset());
assertEquals(Optional.empty(), log.createNewSnapshot(new OffsetAndEpoch(numberOfRecords - 1, epoch)));
}
@ -528,7 +528,7 @@ public class MockLogTest { @@ -528,7 +528,7 @@ public class MockLogTest {
}
assertTrue(log.deleteBeforeSnapshot(snapshotId));
assertEquals(snapshotId.offset, log.startOffset());
assertEquals(snapshotId.offset(), log.startOffset());
assertThrows(
IllegalArgumentException.class,
@ -578,7 +578,7 @@ public class MockLogTest { @@ -578,7 +578,7 @@ public class MockLogTest {
}
assertTrue(log.deleteBeforeSnapshot(snapshotId));
assertEquals(snapshotId.offset, log.startOffset());
assertEquals(snapshotId.offset(), log.startOffset());
assertEquals(Optional.empty(), log.createNewSnapshot(snapshotId));
}
@ -665,10 +665,10 @@ public class MockLogTest { @@ -665,10 +665,10 @@ public class MockLogTest {
}
assertTrue(log.truncateToLatestSnapshot());
assertEquals(sameEpochSnapshotId.offset, log.startOffset());
assertEquals(sameEpochSnapshotId.epoch, log.lastFetchedEpoch());
assertEquals(sameEpochSnapshotId.offset, log.endOffset().offset);
assertEquals(sameEpochSnapshotId.offset, log.highWatermark().offset);
assertEquals(sameEpochSnapshotId.offset(), log.startOffset());
assertEquals(sameEpochSnapshotId.epoch(), log.lastFetchedEpoch());
assertEquals(sameEpochSnapshotId.offset(), log.endOffset().offset);
assertEquals(sameEpochSnapshotId.offset(), log.highWatermark().offset);
OffsetAndEpoch greaterEpochSnapshotId = new OffsetAndEpoch(3 * numberOfRecords, epoch + 1);
@ -679,10 +679,10 @@ public class MockLogTest { @@ -679,10 +679,10 @@ public class MockLogTest {
}
assertTrue(log.truncateToLatestSnapshot());
assertEquals(greaterEpochSnapshotId.offset, log.startOffset());
assertEquals(greaterEpochSnapshotId.epoch, log.lastFetchedEpoch());
assertEquals(greaterEpochSnapshotId.offset, log.endOffset().offset);
assertEquals(greaterEpochSnapshotId.offset, log.highWatermark().offset);
assertEquals(greaterEpochSnapshotId.offset(), log.startOffset());
assertEquals(greaterEpochSnapshotId.epoch(), log.lastFetchedEpoch());
assertEquals(greaterEpochSnapshotId.offset(), log.endOffset().offset);
assertEquals(greaterEpochSnapshotId.offset(), log.highWatermark().offset);
}
@Test
@ -716,7 +716,7 @@ public class MockLogTest { @@ -716,7 +716,7 @@ public class MockLogTest {
OffsetAndEpoch sameEpochSnapshotId = new OffsetAndEpoch(numberOfRecords, epoch);
appendBatch(numberOfRecords, epoch);
log.updateHighWatermark(new LogOffsetMetadata(sameEpochSnapshotId.offset));
log.updateHighWatermark(new LogOffsetMetadata(sameEpochSnapshotId.offset()));
try (RawSnapshotWriter snapshot = log.createNewSnapshot(sameEpochSnapshotId).get()) {
snapshot.freeze();
@ -740,15 +740,15 @@ public class MockLogTest { @@ -740,15 +740,15 @@ public class MockLogTest {
OffsetAndEpoch sameEpochSnapshotId = new OffsetAndEpoch(numberOfRecords, epoch);
appendBatch(numberOfRecords, epoch);
log.updateHighWatermark(new LogOffsetMetadata(sameEpochSnapshotId.offset));
log.updateHighWatermark(new LogOffsetMetadata(sameEpochSnapshotId.offset()));
try (RawSnapshotWriter snapshot = log.createNewSnapshot(sameEpochSnapshotId).get()) {
snapshot.freeze();
}
OffsetAndEpoch greaterEpochSnapshotId = new OffsetAndEpoch(2 * numberOfRecords, epoch + 1);
appendBatch(numberOfRecords, greaterEpochSnapshotId.epoch);
log.updateHighWatermark(new LogOffsetMetadata(greaterEpochSnapshotId.offset));
appendBatch(numberOfRecords, greaterEpochSnapshotId.epoch());
log.updateHighWatermark(new LogOffsetMetadata(greaterEpochSnapshotId.offset()));
try (RawSnapshotWriter snapshot = log.createNewSnapshot(greaterEpochSnapshotId).get()) {
snapshot.freeze();

4
raft/src/test/java/org/apache/kafka/raft/RaftClientTestContext.java

@ -196,8 +196,8 @@ public final class RaftClientTestContext { @@ -196,8 +196,8 @@ public final class RaftClientTestContext {
}
Builder deleteBeforeSnapshot(OffsetAndEpoch snapshotId) throws IOException {
if (snapshotId.offset > log.highWatermark().offset) {
log.updateHighWatermark(new LogOffsetMetadata(snapshotId.offset));
if (snapshotId.offset() > log.highWatermark().offset) {
log.updateHighWatermark(new LogOffsetMetadata(snapshotId.offset()));
}
log.deleteBeforeSnapshot(snapshotId);

16
raft/src/test/java/org/apache/kafka/raft/RaftEventSimulationTest.java

@ -1024,16 +1024,16 @@ public class RaftEventSimulationTest { @@ -1024,16 +1024,16 @@ public class RaftEventSimulationTest {
log.earliestSnapshotId().ifPresent(earliestSnapshotId -> {
long logStartOffset = log.startOffset();
ValidOffsetAndEpoch validateOffsetAndEpoch = log.validateOffsetAndEpoch(
earliestSnapshotId.offset,
earliestSnapshotId.epoch
earliestSnapshotId.offset(),
earliestSnapshotId.epoch()
);
assertTrue(
logStartOffset <= earliestSnapshotId.offset,
logStartOffset <= earliestSnapshotId.offset(),
() -> String.format(
"invalid log start offset (%s) and snapshotId offset (%s): nodeId = %s",
logStartOffset,
earliestSnapshotId.offset,
earliestSnapshotId.offset(),
nodeId
)
);
@ -1046,7 +1046,7 @@ public class RaftEventSimulationTest { @@ -1046,7 +1046,7 @@ public class RaftEventSimulationTest {
if (logStartOffset > 0) {
assertEquals(
logStartOffset,
earliestSnapshotId.offset,
earliestSnapshotId.offset(),
() -> String.format("mising snapshot at log start offset: nodeId = %s", nodeId)
);
}
@ -1108,8 +1108,8 @@ public class RaftEventSimulationTest { @@ -1108,8 +1108,8 @@ public class RaftEventSimulationTest {
AtomicLong startOffset = new AtomicLong(0);
log.earliestSnapshotId().ifPresent(snapshotId -> {
assertTrue(snapshotId.offset <= highWatermark.getAsLong());
startOffset.set(snapshotId.offset);
assertTrue(snapshotId.offset() <= highWatermark.getAsLong());
startOffset.set(snapshotId.offset());
try (SnapshotReader<Integer> snapshot =
RecordsSnapshotReader.of(log.readSnapshot(snapshotId).get(), node.intSerde, BufferSupplier.create(), Integer.MAX_VALUE, true)) {
@ -1120,7 +1120,7 @@ public class RaftEventSimulationTest { @@ -1120,7 +1120,7 @@ public class RaftEventSimulationTest {
assertEquals(1, batch.records().size());
// The snapshotId offset is an "end offset"
long offset = snapshotId.offset - 1;
long offset = snapshotId.offset() - 1;
int sequence = batch.records().get(0);
committedSequenceNumbers.putIfAbsent(offset, sequence);

14
raft/src/test/java/org/apache/kafka/snapshot/SnapshotWriterReaderTest.java

@ -64,7 +64,7 @@ final public class SnapshotWriterReaderTest { @@ -64,7 +64,7 @@ final public class SnapshotWriterReaderTest {
context.advanceLocalLeaderHighWatermarkToLogEndOffset();
// Create an empty snapshot and freeze it immediately
try (SnapshotWriter<String> snapshot = context.client.createSnapshot(id.offset - 1, id.epoch, magicTimestamp).get()) {
try (SnapshotWriter<String> snapshot = context.client.createSnapshot(id.offset() - 1, id.epoch(), magicTimestamp).get()) {
assertEquals(id, snapshot.snapshotId());
snapshot.freeze();
}
@ -88,7 +88,7 @@ final public class SnapshotWriterReaderTest { @@ -88,7 +88,7 @@ final public class SnapshotWriterReaderTest {
RaftClientTestContext.Builder contextBuilder = new RaftClientTestContext.Builder(localId, voters);
for (List<String> batch : expected) {
contextBuilder.appendToLog(id.epoch, batch);
contextBuilder.appendToLog(id.epoch(), batch);
}
RaftClientTestContext context = contextBuilder.build();
@ -97,7 +97,7 @@ final public class SnapshotWriterReaderTest { @@ -97,7 +97,7 @@ final public class SnapshotWriterReaderTest {
context.advanceLocalLeaderHighWatermarkToLogEndOffset();
try (SnapshotWriter<String> snapshot = context.client.createSnapshot(id.offset - 1, id.epoch, magicTimestamp).get()) {
try (SnapshotWriter<String> snapshot = context.client.createSnapshot(id.offset() - 1, id.epoch(), magicTimestamp).get()) {
assertEquals(id, snapshot.snapshotId());
expected.forEach(batch -> assertDoesNotThrow(() -> snapshot.append(batch)));
snapshot.freeze();
@ -120,7 +120,7 @@ final public class SnapshotWriterReaderTest { @@ -120,7 +120,7 @@ final public class SnapshotWriterReaderTest {
RaftClientTestContext.Builder contextBuilder = new RaftClientTestContext.Builder(localId, voters);
for (List<String> batch : expected) {
contextBuilder.appendToLog(id.epoch, batch);
contextBuilder.appendToLog(id.epoch(), batch);
}
RaftClientTestContext context = contextBuilder.build();
@ -129,7 +129,7 @@ final public class SnapshotWriterReaderTest { @@ -129,7 +129,7 @@ final public class SnapshotWriterReaderTest {
context.advanceLocalLeaderHighWatermarkToLogEndOffset();
try (SnapshotWriter<String> snapshot = context.client.createSnapshot(id.offset - 1, id.epoch, 0).get()) {
try (SnapshotWriter<String> snapshot = context.client.createSnapshot(id.offset() - 1, id.epoch(), 0).get()) {
assertEquals(id, snapshot.snapshotId());
expected.forEach(batch -> {
assertDoesNotThrow(() -> snapshot.append(batch));
@ -148,7 +148,7 @@ final public class SnapshotWriterReaderTest { @@ -148,7 +148,7 @@ final public class SnapshotWriterReaderTest {
RaftClientTestContext.Builder contextBuilder = new RaftClientTestContext.Builder(localId, voters);
for (List<String> batch : expected) {
contextBuilder.appendToLog(id.epoch, batch);
contextBuilder.appendToLog(id.epoch(), batch);
}
RaftClientTestContext context = contextBuilder.build();
@ -157,7 +157,7 @@ final public class SnapshotWriterReaderTest { @@ -157,7 +157,7 @@ final public class SnapshotWriterReaderTest {
context.advanceLocalLeaderHighWatermarkToLogEndOffset();
try (SnapshotWriter<String> snapshot = context.client.createSnapshot(id.offset - 1, id.epoch, 0).get()) {
try (SnapshotWriter<String> snapshot = context.client.createSnapshot(id.offset() - 1, id.epoch(), 0).get()) {
assertEquals(id, snapshot.snapshotId());
expected.forEach(batch -> {
assertDoesNotThrow(() -> snapshot.append(batch));

Loading…
Cancel
Save